datafusion_common/scalar/
mod.rs

1// Licensed to the Apache Software Foundation (ASF) under one
2// or more contributor license agreements.  See the NOTICE file
3// distributed with this work for additional information
4// regarding copyright ownership.  The ASF licenses this file
5// to you under the Apache License, Version 2.0 (the
6// "License"); you may not use this file except in compliance
7// with the License.  You may obtain a copy of the License at
8//
9//   http://www.apache.org/licenses/LICENSE-2.0
10//
11// Unless required by applicable law or agreed to in writing,
12// software distributed under the License is distributed on an
13// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14// KIND, either express or implied.  See the License for the
15// specific language governing permissions and limitations
16// under the License.
17
18//! [`ScalarValue`]: stores single  values
19
20mod cache;
21mod consts;
22mod struct_builder;
23
24use std::borrow::Borrow;
25use std::cmp::Ordering;
26use std::collections::{HashSet, VecDeque};
27use std::convert::Infallible;
28use std::fmt;
29use std::hash::Hash;
30use std::hash::Hasher;
31use std::iter::repeat_n;
32use std::mem::{size_of, size_of_val};
33use std::str::FromStr;
34use std::sync::Arc;
35
36use crate::assert_or_internal_err;
37use crate::cast::{
38    as_binary_array, as_binary_view_array, as_boolean_array, as_date32_array,
39    as_date64_array, as_decimal32_array, as_decimal64_array, as_decimal128_array,
40    as_decimal256_array, as_dictionary_array, as_duration_microsecond_array,
41    as_duration_millisecond_array, as_duration_nanosecond_array,
42    as_duration_second_array, as_fixed_size_binary_array, as_fixed_size_list_array,
43    as_float16_array, as_float32_array, as_float64_array, as_int8_array, as_int16_array,
44    as_int32_array, as_int64_array, as_interval_dt_array, as_interval_mdn_array,
45    as_interval_ym_array, as_large_binary_array, as_large_list_array,
46    as_large_string_array, as_string_array, as_string_view_array,
47    as_time32_millisecond_array, as_time32_second_array, as_time64_microsecond_array,
48    as_time64_nanosecond_array, as_timestamp_microsecond_array,
49    as_timestamp_millisecond_array, as_timestamp_nanosecond_array,
50    as_timestamp_second_array, as_uint8_array, as_uint16_array, as_uint32_array,
51    as_uint64_array, as_union_array,
52};
53use crate::error::{_exec_err, _internal_err, _not_impl_err, DataFusionError, Result};
54use crate::format::DEFAULT_CAST_OPTIONS;
55use crate::hash_utils::create_hashes;
56use crate::utils::SingleRowListArrayBuilder;
57use crate::{_internal_datafusion_err, arrow_datafusion_err};
58use arrow::array::{
59    Array, ArrayData, ArrayRef, ArrowNativeTypeOp, ArrowPrimitiveType, AsArray,
60    BinaryArray, BinaryViewArray, BinaryViewBuilder, BooleanArray, Date32Array,
61    Date64Array, Decimal32Array, Decimal64Array, Decimal128Array, Decimal256Array,
62    DictionaryArray, DurationMicrosecondArray, DurationMillisecondArray,
63    DurationNanosecondArray, DurationSecondArray, FixedSizeBinaryArray,
64    FixedSizeBinaryBuilder, FixedSizeListArray, Float16Array, Float32Array, Float64Array,
65    GenericListArray, Int8Array, Int16Array, Int32Array, Int64Array,
66    IntervalDayTimeArray, IntervalMonthDayNanoArray, IntervalYearMonthArray,
67    LargeBinaryArray, LargeListArray, LargeStringArray, ListArray, MapArray,
68    MutableArrayData, OffsetSizeTrait, PrimitiveArray, Scalar, StringArray,
69    StringViewArray, StringViewBuilder, StructArray, Time32MillisecondArray,
70    Time32SecondArray, Time64MicrosecondArray, Time64NanosecondArray,
71    TimestampMicrosecondArray, TimestampMillisecondArray, TimestampNanosecondArray,
72    TimestampSecondArray, UInt8Array, UInt16Array, UInt32Array, UInt64Array, UnionArray,
73    new_empty_array, new_null_array,
74};
75use arrow::buffer::{BooleanBuffer, ScalarBuffer};
76use arrow::compute::kernels::cast::{CastOptions, cast_with_options};
77use arrow::compute::kernels::numeric::{
78    add, add_wrapping, div, mul, mul_wrapping, rem, sub, sub_wrapping,
79};
80use arrow::datatypes::{
81    ArrowDictionaryKeyType, ArrowNativeType, ArrowTimestampType, DataType, Date32Type,
82    Decimal32Type, Decimal64Type, Decimal128Type, Decimal256Type, DecimalType, Field,
83    Float32Type, Int8Type, Int16Type, Int32Type, Int64Type, IntervalDayTime,
84    IntervalDayTimeType, IntervalMonthDayNano, IntervalMonthDayNanoType, IntervalUnit,
85    IntervalYearMonthType, TimeUnit, TimestampMicrosecondType, TimestampMillisecondType,
86    TimestampNanosecondType, TimestampSecondType, UInt8Type, UInt16Type, UInt32Type,
87    UInt64Type, UnionFields, UnionMode, i256, validate_decimal_precision_and_scale,
88};
89use arrow::util::display::{ArrayFormatter, FormatOptions, array_value_to_string};
90use cache::{get_or_create_cached_key_array, get_or_create_cached_null_array};
91use chrono::{Duration, NaiveDate};
92use half::f16;
93pub use struct_builder::ScalarStructBuilder;
94
95const SECONDS_PER_DAY: i64 = 86_400;
96const MILLIS_PER_DAY: i64 = SECONDS_PER_DAY * 1_000;
97const MICROS_PER_DAY: i64 = MILLIS_PER_DAY * 1_000;
98const NANOS_PER_DAY: i64 = MICROS_PER_DAY * 1_000;
99const MICROS_PER_MILLISECOND: i64 = 1_000;
100const NANOS_PER_MILLISECOND: i64 = 1_000_000;
101
102/// Returns the multiplier that converts the input date representation into the
103/// desired timestamp unit, if the conversion requires a multiplication that can
104/// overflow an `i64`.
105pub fn date_to_timestamp_multiplier(
106    source_type: &DataType,
107    target_type: &DataType,
108) -> Option<i64> {
109    let DataType::Timestamp(target_unit, _) = target_type else {
110        return None;
111    };
112
113    // Only `Timestamp` target types have a time unit; otherwise no
114    // multiplier applies (handled above). The function returns `Some(m)`
115    // when converting the `source_type` to `target_type` requires a
116    // multiplication that could overflow `i64`. It returns `None` when
117    // the conversion is a division or otherwise doesn't require a
118    // multiplication (e.g. Date64 -> Second).
119    match source_type {
120        // Date32 stores days since epoch. Converting to any timestamp
121        // unit requires multiplying by the per-day factor (seconds,
122        // milliseconds, microseconds, nanoseconds).
123        DataType::Date32 => Some(match target_unit {
124            TimeUnit::Second => SECONDS_PER_DAY,
125            TimeUnit::Millisecond => MILLIS_PER_DAY,
126            TimeUnit::Microsecond => MICROS_PER_DAY,
127            TimeUnit::Nanosecond => NANOS_PER_DAY,
128        }),
129
130        // Date64 stores milliseconds since epoch. Converting to
131        // seconds is a division (no multiplication), so return `None`.
132        // Converting to milliseconds is 1:1 (multiplier 1). Converting
133        // to micro/nano requires multiplying by 1_000 / 1_000_000.
134        DataType::Date64 => match target_unit {
135            TimeUnit::Second => None,
136            // Converting Date64 (ms since epoch) to millisecond timestamps
137            // is an identity conversion and does not require multiplication.
138            // Returning `None` indicates no multiplication-based overflow
139            // check is necessary.
140            TimeUnit::Millisecond => None,
141            TimeUnit::Microsecond => Some(MICROS_PER_MILLISECOND),
142            TimeUnit::Nanosecond => Some(NANOS_PER_MILLISECOND),
143        },
144
145        _ => None,
146    }
147}
148
149/// Ensures the provided value can be represented as a timestamp with the given
150/// multiplier. Returns an [`DataFusionError::Execution`] when the converted
151/// value would overflow the timestamp range.
152pub fn ensure_timestamp_in_bounds(
153    value: i64,
154    multiplier: i64,
155    source_type: &DataType,
156    target_type: &DataType,
157) -> Result<()> {
158    if multiplier <= 1 {
159        return Ok(());
160    }
161
162    if value.checked_mul(multiplier).is_none() {
163        let target = format_timestamp_type_for_error(target_type);
164        _exec_err!(
165            "Cannot cast {} value {} to {}: converted value exceeds the representable i64 range",
166            source_type,
167            value,
168            target
169        )
170    } else {
171        Ok(())
172    }
173}
174
175/// Format a `DataType::Timestamp` into a short, stable string used in
176/// user-facing error messages.
177pub(crate) fn format_timestamp_type_for_error(target_type: &DataType) -> String {
178    match target_type {
179        DataType::Timestamp(unit, _) => {
180            let s = match unit {
181                TimeUnit::Second => "s",
182                TimeUnit::Millisecond => "ms",
183                TimeUnit::Microsecond => "us",
184                TimeUnit::Nanosecond => "ns",
185            };
186            format!("Timestamp({s})")
187        }
188        other => format!("{other}"),
189    }
190}
191
192/// A dynamically typed, nullable single value.
193///
194/// While an arrow  [`Array`]) stores one or more values of the same type, in a
195/// single column, a `ScalarValue` stores a single value of a single type, the
196/// equivalent of 1 row and one column.
197///
198/// ```text
199///  ┌────────┐
200///  │ value1 │
201///  │ value2 │                  ┌────────┐
202///  │ value3 │                  │ value2 │
203///  │  ...   │                  └────────┘
204///  │ valueN │
205///  └────────┘
206///
207///    Array                     ScalarValue
208///
209/// stores multiple,             stores a single,
210/// possibly null, values of     possible null, value
211/// the same type
212/// ```
213///
214/// # Performance
215///
216/// In general, performance will be better using arrow [`Array`]s rather than
217/// [`ScalarValue`], as it is far more efficient to process multiple values at
218/// once (vectorized processing).
219///
220/// # Example
221/// ```
222/// # use datafusion_common::ScalarValue;
223/// // Create single scalar value for an Int32 value
224/// let s1 = ScalarValue::Int32(Some(10));
225///
226/// // You can also create values using the From impl:
227/// let s2 = ScalarValue::from(10i32);
228/// assert_eq!(s1, s2);
229/// ```
230///
231/// # Null Handling
232///
233/// `ScalarValue` represents null values in the same way as Arrow. Nulls are
234/// "typed" in the sense that a null value in an [`Int32Array`] is different
235/// from a null value in a [`Float64Array`], and is different from the values in
236/// a [`NullArray`].
237///
238/// ```
239/// # fn main() -> datafusion_common::Result<()> {
240/// # use std::collections::hash_set::Difference;
241/// # use datafusion_common::ScalarValue;
242/// # use arrow::datatypes::DataType;
243/// // You can create a 'null' Int32 value directly:
244/// let s1 = ScalarValue::Int32(None);
245///
246/// // You can also create a null value for a given datatype:
247/// let s2 = ScalarValue::try_from(&DataType::Int32)?;
248/// assert_eq!(s1, s2);
249///
250/// // Note that this is DIFFERENT than a `ScalarValue::Null`
251/// let s3 = ScalarValue::Null;
252/// assert_ne!(s1, s3);
253/// # Ok(())
254/// # }
255/// ```
256///
257/// # Nested Types
258///
259/// `List` / `LargeList` / `FixedSizeList` / `Struct` / `Map` are represented as a
260/// single element array of the corresponding type.
261///
262/// ## Example: Creating [`ScalarValue::Struct`] using [`ScalarStructBuilder`]
263/// ```
264/// # use std::sync::Arc;
265/// # use arrow::datatypes::{DataType, Field};
266/// # use datafusion_common::{ScalarValue, scalar::ScalarStructBuilder};
267/// // Build a struct like: {a: 1, b: "foo"}
268/// let field_a = Field::new("a", DataType::Int32, false);
269/// let field_b = Field::new("b", DataType::Utf8, false);
270///
271/// let s1 = ScalarStructBuilder::new()
272///     .with_scalar(field_a, ScalarValue::from(1i32))
273///     .with_scalar(field_b, ScalarValue::from("foo"))
274///     .build();
275/// ```
276///
277/// ## Example: Creating a null [`ScalarValue::Struct`] using [`ScalarStructBuilder`]
278/// ```
279/// # use std::sync::Arc;
280/// # use arrow::datatypes::{DataType, Field};
281/// # use datafusion_common::{ScalarValue, scalar::ScalarStructBuilder};
282/// // Build a struct representing a NULL value
283/// let fields = vec![
284///     Field::new("a", DataType::Int32, false),
285///     Field::new("b", DataType::Utf8, false),
286/// ];
287///
288/// let s1 = ScalarStructBuilder::new_null(fields);
289/// ```
290///
291/// ## Example: Creating [`ScalarValue::Struct`] directly
292/// ```
293/// # use std::sync::Arc;
294/// # use arrow::datatypes::{DataType, Field, Fields};
295/// # use arrow::array::{ArrayRef, Int32Array, StructArray, StringArray};
296/// # use datafusion_common::ScalarValue;
297/// // Build a struct like: {a: 1, b: "foo"}
298/// // Field description
299/// let fields = Fields::from(vec![
300///     Field::new("a", DataType::Int32, false),
301///     Field::new("b", DataType::Utf8, false),
302/// ]);
303/// // one row arrays for each field
304/// let arrays: Vec<ArrayRef> = vec![
305///     Arc::new(Int32Array::from(vec![1])),
306///     Arc::new(StringArray::from(vec!["foo"])),
307/// ];
308/// // no nulls for this array
309/// let nulls = None;
310/// let arr = StructArray::new(fields, arrays, nulls);
311///
312/// // Create a ScalarValue::Struct directly
313/// let s1 = ScalarValue::Struct(Arc::new(arr));
314/// ```
315///
316///
317/// # Further Reading
318/// See [datatypes](https://arrow.apache.org/docs/python/api/datatypes.html) for
319/// details on datatypes and the [format](https://github.com/apache/arrow/blob/master/format/Schema.fbs#L354-L375)
320/// for the definitive reference.
321///
322/// [`NullArray`]: arrow::array::NullArray
323#[derive(Clone)]
324pub enum ScalarValue {
325    /// represents `DataType::Null` (castable to/from any other type)
326    Null,
327    /// true or false value
328    Boolean(Option<bool>),
329    /// 16bit float
330    Float16(Option<f16>),
331    /// 32bit float
332    Float32(Option<f32>),
333    /// 64bit float
334    Float64(Option<f64>),
335    /// 32bit decimal, using the i32 to represent the decimal, precision scale
336    Decimal32(Option<i32>, u8, i8),
337    /// 64bit decimal, using the i64 to represent the decimal, precision scale
338    Decimal64(Option<i64>, u8, i8),
339    /// 128bit decimal, using the i128 to represent the decimal, precision scale
340    Decimal128(Option<i128>, u8, i8),
341    /// 256bit decimal, using the i256 to represent the decimal, precision scale
342    Decimal256(Option<i256>, u8, i8),
343    /// signed 8bit int
344    Int8(Option<i8>),
345    /// signed 16bit int
346    Int16(Option<i16>),
347    /// signed 32bit int
348    Int32(Option<i32>),
349    /// signed 64bit int
350    Int64(Option<i64>),
351    /// unsigned 8bit int
352    UInt8(Option<u8>),
353    /// unsigned 16bit int
354    UInt16(Option<u16>),
355    /// unsigned 32bit int
356    UInt32(Option<u32>),
357    /// unsigned 64bit int
358    UInt64(Option<u64>),
359    /// utf-8 encoded string.
360    Utf8(Option<String>),
361    /// utf-8 encoded string but from view types.
362    Utf8View(Option<String>),
363    /// utf-8 encoded string representing a LargeString's arrow type.
364    LargeUtf8(Option<String>),
365    /// binary
366    Binary(Option<Vec<u8>>),
367    /// binary but from view types.
368    BinaryView(Option<Vec<u8>>),
369    /// fixed size binary
370    FixedSizeBinary(i32, Option<Vec<u8>>),
371    /// large binary
372    LargeBinary(Option<Vec<u8>>),
373    /// Fixed size list scalar.
374    ///
375    /// The array must be a FixedSizeListArray with length 1.
376    FixedSizeList(Arc<FixedSizeListArray>),
377    /// Represents a single element of a [`ListArray`] as an [`ArrayRef`]
378    ///
379    /// The array must be a ListArray with length 1.
380    List(Arc<ListArray>),
381    /// The array must be a LargeListArray with length 1.
382    LargeList(Arc<LargeListArray>),
383    /// Represents a single element [`StructArray`] as an [`ArrayRef`]. See
384    /// [`ScalarValue`] for examples of how to create instances of this type.
385    Struct(Arc<StructArray>),
386    /// Represents a single element [`MapArray`] as an [`ArrayRef`].
387    Map(Arc<MapArray>),
388    /// Date stored as a signed 32bit int days since UNIX epoch 1970-01-01
389    Date32(Option<i32>),
390    /// Date stored as a signed 64bit int milliseconds since UNIX epoch 1970-01-01
391    Date64(Option<i64>),
392    /// Time stored as a signed 32bit int as seconds since midnight
393    Time32Second(Option<i32>),
394    /// Time stored as a signed 32bit int as milliseconds since midnight
395    Time32Millisecond(Option<i32>),
396    /// Time stored as a signed 64bit int as microseconds since midnight
397    Time64Microsecond(Option<i64>),
398    /// Time stored as a signed 64bit int as nanoseconds since midnight
399    Time64Nanosecond(Option<i64>),
400    /// Timestamp Second
401    TimestampSecond(Option<i64>, Option<Arc<str>>),
402    /// Timestamp Milliseconds
403    TimestampMillisecond(Option<i64>, Option<Arc<str>>),
404    /// Timestamp Microseconds
405    TimestampMicrosecond(Option<i64>, Option<Arc<str>>),
406    /// Timestamp Nanoseconds
407    TimestampNanosecond(Option<i64>, Option<Arc<str>>),
408    /// Number of elapsed whole months
409    IntervalYearMonth(Option<i32>),
410    /// Number of elapsed days and milliseconds (no leap seconds)
411    /// stored as 2 contiguous 32-bit signed integers
412    IntervalDayTime(Option<IntervalDayTime>),
413    /// A triple of the number of elapsed months, days, and nanoseconds.
414    /// Months and days are encoded as 32-bit signed integers.
415    /// Nanoseconds is encoded as a 64-bit signed integer (no leap seconds).
416    IntervalMonthDayNano(Option<IntervalMonthDayNano>),
417    /// Duration in seconds
418    DurationSecond(Option<i64>),
419    /// Duration in milliseconds
420    DurationMillisecond(Option<i64>),
421    /// Duration in microseconds
422    DurationMicrosecond(Option<i64>),
423    /// Duration in nanoseconds
424    DurationNanosecond(Option<i64>),
425    /// A nested datatype that can represent slots of differing types. Components:
426    /// `.0`: a tuple of union `type_id` and the single value held by this Scalar
427    /// `.1`: the list of fields, zero-to-one of which will by set in `.0`
428    /// `.2`: the physical storage of the source/destination UnionArray from which this Scalar came
429    Union(Option<(i8, Box<ScalarValue>)>, UnionFields, UnionMode),
430    /// Dictionary type: index type and value
431    Dictionary(Box<DataType>, Box<ScalarValue>),
432}
433
434impl Hash for Fl<f16> {
435    fn hash<H: Hasher>(&self, state: &mut H) {
436        self.0.to_bits().hash(state);
437    }
438}
439
440// manual implementation of `PartialEq`
441impl PartialEq for ScalarValue {
442    fn eq(&self, other: &Self) -> bool {
443        use ScalarValue::*;
444        // This purposely doesn't have a catch-all "(_, _)" so that
445        // any newly added enum variant will require editing this list
446        // or else face a compile error
447        match (self, other) {
448            (Decimal32(v1, p1, s1), Decimal32(v2, p2, s2)) => {
449                v1.eq(v2) && p1.eq(p2) && s1.eq(s2)
450            }
451            (Decimal32(_, _, _), _) => false,
452            (Decimal64(v1, p1, s1), Decimal64(v2, p2, s2)) => {
453                v1.eq(v2) && p1.eq(p2) && s1.eq(s2)
454            }
455            (Decimal64(_, _, _), _) => false,
456            (Decimal128(v1, p1, s1), Decimal128(v2, p2, s2)) => {
457                v1.eq(v2) && p1.eq(p2) && s1.eq(s2)
458            }
459            (Decimal128(_, _, _), _) => false,
460            (Decimal256(v1, p1, s1), Decimal256(v2, p2, s2)) => {
461                v1.eq(v2) && p1.eq(p2) && s1.eq(s2)
462            }
463            (Decimal256(_, _, _), _) => false,
464            (Boolean(v1), Boolean(v2)) => v1.eq(v2),
465            (Boolean(_), _) => false,
466            (Float32(v1), Float32(v2)) => match (v1, v2) {
467                (Some(f1), Some(f2)) => f1.to_bits() == f2.to_bits(),
468                _ => v1.eq(v2),
469            },
470            (Float16(v1), Float16(v2)) => match (v1, v2) {
471                (Some(f1), Some(f2)) => f1.to_bits() == f2.to_bits(),
472                _ => v1.eq(v2),
473            },
474            (Float32(_), _) => false,
475            (Float16(_), _) => false,
476            (Float64(v1), Float64(v2)) => match (v1, v2) {
477                (Some(f1), Some(f2)) => f1.to_bits() == f2.to_bits(),
478                _ => v1.eq(v2),
479            },
480            (Float64(_), _) => false,
481            (Int8(v1), Int8(v2)) => v1.eq(v2),
482            (Int8(_), _) => false,
483            (Int16(v1), Int16(v2)) => v1.eq(v2),
484            (Int16(_), _) => false,
485            (Int32(v1), Int32(v2)) => v1.eq(v2),
486            (Int32(_), _) => false,
487            (Int64(v1), Int64(v2)) => v1.eq(v2),
488            (Int64(_), _) => false,
489            (UInt8(v1), UInt8(v2)) => v1.eq(v2),
490            (UInt8(_), _) => false,
491            (UInt16(v1), UInt16(v2)) => v1.eq(v2),
492            (UInt16(_), _) => false,
493            (UInt32(v1), UInt32(v2)) => v1.eq(v2),
494            (UInt32(_), _) => false,
495            (UInt64(v1), UInt64(v2)) => v1.eq(v2),
496            (UInt64(_), _) => false,
497            (Utf8(v1), Utf8(v2)) => v1.eq(v2),
498            (Utf8(_), _) => false,
499            (Utf8View(v1), Utf8View(v2)) => v1.eq(v2),
500            (Utf8View(_), _) => false,
501            (LargeUtf8(v1), LargeUtf8(v2)) => v1.eq(v2),
502            (LargeUtf8(_), _) => false,
503            (Binary(v1), Binary(v2)) => v1.eq(v2),
504            (Binary(_), _) => false,
505            (BinaryView(v1), BinaryView(v2)) => v1.eq(v2),
506            (BinaryView(_), _) => false,
507            (FixedSizeBinary(_, v1), FixedSizeBinary(_, v2)) => v1.eq(v2),
508            (FixedSizeBinary(_, _), _) => false,
509            (LargeBinary(v1), LargeBinary(v2)) => v1.eq(v2),
510            (LargeBinary(_), _) => false,
511            (FixedSizeList(v1), FixedSizeList(v2)) => v1.eq(v2),
512            (FixedSizeList(_), _) => false,
513            (List(v1), List(v2)) => v1.eq(v2),
514            (List(_), _) => false,
515            (LargeList(v1), LargeList(v2)) => v1.eq(v2),
516            (LargeList(_), _) => false,
517            (Struct(v1), Struct(v2)) => v1.eq(v2),
518            (Struct(_), _) => false,
519            (Map(v1), Map(v2)) => v1.eq(v2),
520            (Map(_), _) => false,
521            (Date32(v1), Date32(v2)) => v1.eq(v2),
522            (Date32(_), _) => false,
523            (Date64(v1), Date64(v2)) => v1.eq(v2),
524            (Date64(_), _) => false,
525            (Time32Second(v1), Time32Second(v2)) => v1.eq(v2),
526            (Time32Second(_), _) => false,
527            (Time32Millisecond(v1), Time32Millisecond(v2)) => v1.eq(v2),
528            (Time32Millisecond(_), _) => false,
529            (Time64Microsecond(v1), Time64Microsecond(v2)) => v1.eq(v2),
530            (Time64Microsecond(_), _) => false,
531            (Time64Nanosecond(v1), Time64Nanosecond(v2)) => v1.eq(v2),
532            (Time64Nanosecond(_), _) => false,
533            (TimestampSecond(v1, _), TimestampSecond(v2, _)) => v1.eq(v2),
534            (TimestampSecond(_, _), _) => false,
535            (TimestampMillisecond(v1, _), TimestampMillisecond(v2, _)) => v1.eq(v2),
536            (TimestampMillisecond(_, _), _) => false,
537            (TimestampMicrosecond(v1, _), TimestampMicrosecond(v2, _)) => v1.eq(v2),
538            (TimestampMicrosecond(_, _), _) => false,
539            (TimestampNanosecond(v1, _), TimestampNanosecond(v2, _)) => v1.eq(v2),
540            (TimestampNanosecond(_, _), _) => false,
541            (DurationSecond(v1), DurationSecond(v2)) => v1.eq(v2),
542            (DurationSecond(_), _) => false,
543            (DurationMillisecond(v1), DurationMillisecond(v2)) => v1.eq(v2),
544            (DurationMillisecond(_), _) => false,
545            (DurationMicrosecond(v1), DurationMicrosecond(v2)) => v1.eq(v2),
546            (DurationMicrosecond(_), _) => false,
547            (DurationNanosecond(v1), DurationNanosecond(v2)) => v1.eq(v2),
548            (DurationNanosecond(_), _) => false,
549            (IntervalYearMonth(v1), IntervalYearMonth(v2)) => v1.eq(v2),
550            (IntervalYearMonth(_), _) => false,
551            (IntervalDayTime(v1), IntervalDayTime(v2)) => v1.eq(v2),
552            (IntervalDayTime(_), _) => false,
553            (IntervalMonthDayNano(v1), IntervalMonthDayNano(v2)) => v1.eq(v2),
554            (IntervalMonthDayNano(_), _) => false,
555            (Union(val1, fields1, mode1), Union(val2, fields2, mode2)) => {
556                val1.eq(val2) && fields1.eq(fields2) && mode1.eq(mode2)
557            }
558            (Union(_, _, _), _) => false,
559            (Dictionary(k1, v1), Dictionary(k2, v2)) => k1.eq(k2) && v1.eq(v2),
560            (Dictionary(_, _), _) => false,
561            (Null, Null) => true,
562            (Null, _) => false,
563        }
564    }
565}
566
567// manual implementation of `PartialOrd`
568impl PartialOrd for ScalarValue {
569    fn partial_cmp(&self, other: &Self) -> Option<Ordering> {
570        use ScalarValue::*;
571        // This purposely doesn't have a catch-all "(_, _)" so that
572        // any newly added enum variant will require editing this list
573        // or else face a compile error
574        match (self, other) {
575            (Decimal32(v1, p1, s1), Decimal32(v2, p2, s2)) => {
576                if p1.eq(p2) && s1.eq(s2) {
577                    v1.partial_cmp(v2)
578                } else {
579                    // Two decimal values can be compared if they have the same precision and scale.
580                    None
581                }
582            }
583            (Decimal32(_, _, _), _) => None,
584            (Decimal64(v1, p1, s1), Decimal64(v2, p2, s2)) => {
585                if p1.eq(p2) && s1.eq(s2) {
586                    v1.partial_cmp(v2)
587                } else {
588                    // Two decimal values can be compared if they have the same precision and scale.
589                    None
590                }
591            }
592            (Decimal64(_, _, _), _) => None,
593            (Decimal128(v1, p1, s1), Decimal128(v2, p2, s2)) => {
594                if p1.eq(p2) && s1.eq(s2) {
595                    v1.partial_cmp(v2)
596                } else {
597                    // Two decimal values can be compared if they have the same precision and scale.
598                    None
599                }
600            }
601            (Decimal128(_, _, _), _) => None,
602            (Decimal256(v1, p1, s1), Decimal256(v2, p2, s2)) => {
603                if p1.eq(p2) && s1.eq(s2) {
604                    v1.partial_cmp(v2)
605                } else {
606                    // Two decimal values can be compared if they have the same precision and scale.
607                    None
608                }
609            }
610            (Decimal256(_, _, _), _) => None,
611            (Boolean(v1), Boolean(v2)) => v1.partial_cmp(v2),
612            (Boolean(_), _) => None,
613            (Float32(v1), Float32(v2)) => match (v1, v2) {
614                (Some(f1), Some(f2)) => Some(f1.total_cmp(f2)),
615                _ => v1.partial_cmp(v2),
616            },
617            (Float16(v1), Float16(v2)) => match (v1, v2) {
618                (Some(f1), Some(f2)) => Some(f1.total_cmp(f2)),
619                _ => v1.partial_cmp(v2),
620            },
621            (Float32(_), _) => None,
622            (Float16(_), _) => None,
623            (Float64(v1), Float64(v2)) => match (v1, v2) {
624                (Some(f1), Some(f2)) => Some(f1.total_cmp(f2)),
625                _ => v1.partial_cmp(v2),
626            },
627            (Float64(_), _) => None,
628            (Int8(v1), Int8(v2)) => v1.partial_cmp(v2),
629            (Int8(_), _) => None,
630            (Int16(v1), Int16(v2)) => v1.partial_cmp(v2),
631            (Int16(_), _) => None,
632            (Int32(v1), Int32(v2)) => v1.partial_cmp(v2),
633            (Int32(_), _) => None,
634            (Int64(v1), Int64(v2)) => v1.partial_cmp(v2),
635            (Int64(_), _) => None,
636            (UInt8(v1), UInt8(v2)) => v1.partial_cmp(v2),
637            (UInt8(_), _) => None,
638            (UInt16(v1), UInt16(v2)) => v1.partial_cmp(v2),
639            (UInt16(_), _) => None,
640            (UInt32(v1), UInt32(v2)) => v1.partial_cmp(v2),
641            (UInt32(_), _) => None,
642            (UInt64(v1), UInt64(v2)) => v1.partial_cmp(v2),
643            (UInt64(_), _) => None,
644            (Utf8(v1), Utf8(v2)) => v1.partial_cmp(v2),
645            (Utf8(_), _) => None,
646            (LargeUtf8(v1), LargeUtf8(v2)) => v1.partial_cmp(v2),
647            (LargeUtf8(_), _) => None,
648            (Utf8View(v1), Utf8View(v2)) => v1.partial_cmp(v2),
649            (Utf8View(_), _) => None,
650            (Binary(v1), Binary(v2)) => v1.partial_cmp(v2),
651            (Binary(_), _) => None,
652            (BinaryView(v1), BinaryView(v2)) => v1.partial_cmp(v2),
653            (BinaryView(_), _) => None,
654            (FixedSizeBinary(_, v1), FixedSizeBinary(_, v2)) => v1.partial_cmp(v2),
655            (FixedSizeBinary(_, _), _) => None,
656            (LargeBinary(v1), LargeBinary(v2)) => v1.partial_cmp(v2),
657            (LargeBinary(_), _) => None,
658            // ScalarValue::List / ScalarValue::FixedSizeList / ScalarValue::LargeList are ensure to have length 1
659            (List(arr1), List(arr2)) => partial_cmp_list(arr1.as_ref(), arr2.as_ref()),
660            (FixedSizeList(arr1), FixedSizeList(arr2)) => {
661                partial_cmp_list(arr1.as_ref(), arr2.as_ref())
662            }
663            (LargeList(arr1), LargeList(arr2)) => {
664                partial_cmp_list(arr1.as_ref(), arr2.as_ref())
665            }
666            (List(_), _) | (LargeList(_), _) | (FixedSizeList(_), _) => None,
667            (Struct(struct_arr1), Struct(struct_arr2)) => {
668                partial_cmp_struct(struct_arr1.as_ref(), struct_arr2.as_ref())
669            }
670            (Struct(_), _) => None,
671            (Map(map_arr1), Map(map_arr2)) => partial_cmp_map(map_arr1, map_arr2),
672            (Map(_), _) => None,
673            (Date32(v1), Date32(v2)) => v1.partial_cmp(v2),
674            (Date32(_), _) => None,
675            (Date64(v1), Date64(v2)) => v1.partial_cmp(v2),
676            (Date64(_), _) => None,
677            (Time32Second(v1), Time32Second(v2)) => v1.partial_cmp(v2),
678            (Time32Second(_), _) => None,
679            (Time32Millisecond(v1), Time32Millisecond(v2)) => v1.partial_cmp(v2),
680            (Time32Millisecond(_), _) => None,
681            (Time64Microsecond(v1), Time64Microsecond(v2)) => v1.partial_cmp(v2),
682            (Time64Microsecond(_), _) => None,
683            (Time64Nanosecond(v1), Time64Nanosecond(v2)) => v1.partial_cmp(v2),
684            (Time64Nanosecond(_), _) => None,
685            (TimestampSecond(v1, _), TimestampSecond(v2, _)) => v1.partial_cmp(v2),
686            (TimestampSecond(_, _), _) => None,
687            (TimestampMillisecond(v1, _), TimestampMillisecond(v2, _)) => {
688                v1.partial_cmp(v2)
689            }
690            (TimestampMillisecond(_, _), _) => None,
691            (TimestampMicrosecond(v1, _), TimestampMicrosecond(v2, _)) => {
692                v1.partial_cmp(v2)
693            }
694            (TimestampMicrosecond(_, _), _) => None,
695            (TimestampNanosecond(v1, _), TimestampNanosecond(v2, _)) => {
696                v1.partial_cmp(v2)
697            }
698            (TimestampNanosecond(_, _), _) => None,
699            (IntervalYearMonth(v1), IntervalYearMonth(v2)) => v1.partial_cmp(v2),
700            (IntervalYearMonth(_), _) => None,
701            (IntervalDayTime(v1), IntervalDayTime(v2)) => v1.partial_cmp(v2),
702            (IntervalDayTime(_), _) => None,
703            (IntervalMonthDayNano(v1), IntervalMonthDayNano(v2)) => v1.partial_cmp(v2),
704            (IntervalMonthDayNano(_), _) => None,
705            (DurationSecond(v1), DurationSecond(v2)) => v1.partial_cmp(v2),
706            (DurationSecond(_), _) => None,
707            (DurationMillisecond(v1), DurationMillisecond(v2)) => v1.partial_cmp(v2),
708            (DurationMillisecond(_), _) => None,
709            (DurationMicrosecond(v1), DurationMicrosecond(v2)) => v1.partial_cmp(v2),
710            (DurationMicrosecond(_), _) => None,
711            (DurationNanosecond(v1), DurationNanosecond(v2)) => v1.partial_cmp(v2),
712            (DurationNanosecond(_), _) => None,
713            (Union(v1, t1, m1), Union(v2, t2, m2)) => {
714                if t1.eq(t2) && m1.eq(m2) {
715                    v1.partial_cmp(v2)
716                } else {
717                    None
718                }
719            }
720            (Union(_, _, _), _) => None,
721            (Dictionary(k1, v1), Dictionary(k2, v2)) => {
722                // Don't compare if the key types don't match (it is effectively a different datatype)
723                if k1 == k2 { v1.partial_cmp(v2) } else { None }
724            }
725            (Dictionary(_, _), _) => None,
726            (Null, Null) => Some(Ordering::Equal),
727            (Null, _) => None,
728        }
729    }
730}
731
732/// List/LargeList/FixedSizeList scalars always have a single element
733/// array. This function returns that array
734fn first_array_for_list(arr: &dyn Array) -> ArrayRef {
735    assert_eq!(arr.len(), 1);
736    if let Some(arr) = arr.as_list_opt::<i32>() {
737        arr.value(0)
738    } else if let Some(arr) = arr.as_list_opt::<i64>() {
739        arr.value(0)
740    } else if let Some(arr) = arr.as_fixed_size_list_opt() {
741        arr.value(0)
742    } else {
743        unreachable!(
744            "Since only List / LargeList / FixedSizeList are supported, this should never happen"
745        )
746    }
747}
748
749/// Compares two List/LargeList/FixedSizeList scalars
750fn partial_cmp_list(arr1: &dyn Array, arr2: &dyn Array) -> Option<Ordering> {
751    if arr1.data_type() != arr2.data_type() {
752        return None;
753    }
754    let arr1 = first_array_for_list(arr1);
755    let arr2 = first_array_for_list(arr2);
756
757    let min_length = arr1.len().min(arr2.len());
758    let arr1_trimmed = arr1.slice(0, min_length);
759    let arr2_trimmed = arr2.slice(0, min_length);
760
761    let lt_res = arrow::compute::kernels::cmp::lt(&arr1_trimmed, &arr2_trimmed).ok()?;
762    let eq_res = arrow::compute::kernels::cmp::eq(&arr1_trimmed, &arr2_trimmed).ok()?;
763
764    for j in 0..lt_res.len() {
765        // In Postgres, NULL values in lists are always considered to be greater than non-NULL values:
766        //
767        // $ SELECT ARRAY[NULL]::integer[] > ARRAY[1]
768        // true
769        //
770        // These next two if statements are introduced for replicating Postgres behavior, as
771        // arrow::compute does not account for this.
772        if arr1_trimmed.is_null(j) && !arr2_trimmed.is_null(j) {
773            return Some(Ordering::Greater);
774        }
775        if !arr1_trimmed.is_null(j) && arr2_trimmed.is_null(j) {
776            return Some(Ordering::Less);
777        }
778
779        if lt_res.is_valid(j) && lt_res.value(j) {
780            return Some(Ordering::Less);
781        }
782        if eq_res.is_valid(j) && !eq_res.value(j) {
783            return Some(Ordering::Greater);
784        }
785    }
786
787    Some(arr1.len().cmp(&arr2.len()))
788}
789
790fn flatten<'a>(array: &'a StructArray, columns: &mut Vec<&'a ArrayRef>) {
791    for i in 0..array.num_columns() {
792        let column = array.column(i);
793        if let Some(nested_struct) = column.as_any().downcast_ref::<StructArray>() {
794            // If it's a nested struct, recursively expand
795            flatten(nested_struct, columns);
796        } else {
797            // If it's a primitive type, add directly
798            columns.push(column);
799        }
800    }
801}
802
803pub fn partial_cmp_struct(s1: &StructArray, s2: &StructArray) -> Option<Ordering> {
804    if s1.len() != s2.len() {
805        return None;
806    }
807
808    if s1.data_type() != s2.data_type() {
809        return None;
810    }
811
812    let mut expanded_columns1 = Vec::with_capacity(s1.num_columns());
813    let mut expanded_columns2 = Vec::with_capacity(s2.num_columns());
814
815    flatten(s1, &mut expanded_columns1);
816    flatten(s2, &mut expanded_columns2);
817
818    for col_index in 0..expanded_columns1.len() {
819        let arr1 = expanded_columns1[col_index];
820        let arr2 = expanded_columns2[col_index];
821
822        let lt_res = arrow::compute::kernels::cmp::lt(arr1, arr2).ok()?;
823        let eq_res = arrow::compute::kernels::cmp::eq(arr1, arr2).ok()?;
824
825        for j in 0..lt_res.len() {
826            if lt_res.is_valid(j) && lt_res.value(j) {
827                return Some(Ordering::Less);
828            }
829            if eq_res.is_valid(j) && !eq_res.value(j) {
830                return Some(Ordering::Greater);
831            }
832        }
833    }
834    Some(Ordering::Equal)
835}
836
837fn partial_cmp_map(m1: &Arc<MapArray>, m2: &Arc<MapArray>) -> Option<Ordering> {
838    if m1.len() != m2.len() {
839        return None;
840    }
841
842    if m1.data_type() != m2.data_type() {
843        return None;
844    }
845
846    for col_index in 0..m1.len() {
847        let arr1 = m1.entries().column(col_index);
848        let arr2 = m2.entries().column(col_index);
849
850        let lt_res = arrow::compute::kernels::cmp::lt(arr1, arr2).ok()?;
851        let eq_res = arrow::compute::kernels::cmp::eq(arr1, arr2).ok()?;
852
853        for j in 0..lt_res.len() {
854            if lt_res.is_valid(j) && lt_res.value(j) {
855                return Some(Ordering::Less);
856            }
857            if eq_res.is_valid(j) && !eq_res.value(j) {
858                return Some(Ordering::Greater);
859            }
860        }
861    }
862    Some(Ordering::Equal)
863}
864
865impl Eq for ScalarValue {}
866
867//Float wrapper over f32/f64. Just because we cannot build std::hash::Hash for floats directly we have to do it through type wrapper
868struct Fl<T>(T);
869
870macro_rules! hash_float_value {
871    ($(($t:ty, $i:ty)),+) => {
872        $(impl std::hash::Hash for Fl<$t> {
873            #[inline]
874            fn hash<H: std::hash::Hasher>(&self, state: &mut H) {
875                state.write(&<$i>::from_ne_bytes(self.0.to_ne_bytes()).to_ne_bytes())
876            }
877        })+
878    };
879}
880
881hash_float_value!((f64, u64), (f32, u32));
882
883// manual implementation of `Hash`
884//
885// # Panics
886//
887// Panics if there is an error when creating hash values for rows
888impl Hash for ScalarValue {
889    fn hash<H: Hasher>(&self, state: &mut H) {
890        use ScalarValue::*;
891        match self {
892            Decimal32(v, p, s) => {
893                v.hash(state);
894                p.hash(state);
895                s.hash(state)
896            }
897            Decimal64(v, p, s) => {
898                v.hash(state);
899                p.hash(state);
900                s.hash(state)
901            }
902            Decimal128(v, p, s) => {
903                v.hash(state);
904                p.hash(state);
905                s.hash(state)
906            }
907            Decimal256(v, p, s) => {
908                v.hash(state);
909                p.hash(state);
910                s.hash(state)
911            }
912            Boolean(v) => v.hash(state),
913            Float16(v) => v.map(Fl).hash(state),
914            Float32(v) => v.map(Fl).hash(state),
915            Float64(v) => v.map(Fl).hash(state),
916            Int8(v) => v.hash(state),
917            Int16(v) => v.hash(state),
918            Int32(v) => v.hash(state),
919            Int64(v) => v.hash(state),
920            UInt8(v) => v.hash(state),
921            UInt16(v) => v.hash(state),
922            UInt32(v) => v.hash(state),
923            UInt64(v) => v.hash(state),
924            Utf8(v) | LargeUtf8(v) | Utf8View(v) => v.hash(state),
925            Binary(v) | FixedSizeBinary(_, v) | LargeBinary(v) | BinaryView(v) => {
926                v.hash(state)
927            }
928            List(arr) => {
929                hash_nested_array(arr.to_owned() as ArrayRef, state);
930            }
931            LargeList(arr) => {
932                hash_nested_array(arr.to_owned() as ArrayRef, state);
933            }
934            FixedSizeList(arr) => {
935                hash_nested_array(arr.to_owned() as ArrayRef, state);
936            }
937            Struct(arr) => {
938                hash_nested_array(arr.to_owned() as ArrayRef, state);
939            }
940            Map(arr) => {
941                hash_nested_array(arr.to_owned() as ArrayRef, state);
942            }
943            Date32(v) => v.hash(state),
944            Date64(v) => v.hash(state),
945            Time32Second(v) => v.hash(state),
946            Time32Millisecond(v) => v.hash(state),
947            Time64Microsecond(v) => v.hash(state),
948            Time64Nanosecond(v) => v.hash(state),
949            TimestampSecond(v, _) => v.hash(state),
950            TimestampMillisecond(v, _) => v.hash(state),
951            TimestampMicrosecond(v, _) => v.hash(state),
952            TimestampNanosecond(v, _) => v.hash(state),
953            DurationSecond(v) => v.hash(state),
954            DurationMillisecond(v) => v.hash(state),
955            DurationMicrosecond(v) => v.hash(state),
956            DurationNanosecond(v) => v.hash(state),
957            IntervalYearMonth(v) => v.hash(state),
958            IntervalDayTime(v) => v.hash(state),
959            IntervalMonthDayNano(v) => v.hash(state),
960            Union(v, t, m) => {
961                v.hash(state);
962                t.hash(state);
963                m.hash(state);
964            }
965            Dictionary(k, v) => {
966                k.hash(state);
967                v.hash(state);
968            }
969            // stable hash for Null value
970            Null => 1.hash(state),
971        }
972    }
973}
974
975fn hash_nested_array<H: Hasher>(arr: ArrayRef, state: &mut H) {
976    let len = arr.len();
977    let hashes_buffer = &mut vec![0; len];
978    let random_state = ahash::RandomState::with_seeds(0, 0, 0, 0);
979    let hashes = create_hashes(&[arr], &random_state, hashes_buffer)
980        .expect("hash_nested_array: failed to create row hashes");
981    // Hash back to std::hash::Hasher
982    hashes.hash(state);
983}
984
985/// Return a reference to the values array and the index into it for a
986/// dictionary array
987///
988/// # Errors
989///
990/// Errors if the array cannot be downcasted to DictionaryArray
991#[inline]
992pub fn get_dict_value<K: ArrowDictionaryKeyType>(
993    array: &dyn Array,
994    index: usize,
995) -> Result<(&ArrayRef, Option<usize>)> {
996    let dict_array = as_dictionary_array::<K>(array)?;
997    Ok((dict_array.values(), dict_array.key(index)))
998}
999
1000/// Create a dictionary array representing `value` repeated `size`
1001/// times
1002fn dict_from_scalar<K: ArrowDictionaryKeyType>(
1003    value: &ScalarValue,
1004    size: usize,
1005) -> Result<ArrayRef> {
1006    // values array is one element long (the value)
1007    let values_array = value.to_array_of_size(1)?;
1008
1009    // Create a key array with `size` elements, each of 0
1010    // Use cache to avoid repeated allocations for the same size
1011    let key_array: PrimitiveArray<K> =
1012        get_or_create_cached_key_array::<K>(size, value.is_null());
1013
1014    // create a new DictionaryArray
1015    //
1016    // Note: this path could be made faster by using the ArrayData
1017    // APIs and skipping validation, if it every comes up in
1018    // performance traces.
1019    Ok(Arc::new(
1020        DictionaryArray::<K>::try_new(key_array, values_array)?, // should always be valid by construction above
1021    ))
1022}
1023
1024/// Create a `DictionaryArray` from the provided values array.
1025///
1026/// Each element gets a unique key (`0..N-1`), without deduplication.
1027/// Useful for wrapping arrays in dictionary form.
1028///
1029/// # Input
1030/// ["alice", "bob", "alice", null, "carol"]
1031///
1032/// # Output
1033/// `DictionaryArray<Int32>`
1034/// {
1035///   keys:   [0, 1, 2, 3, 4],
1036///   values: ["alice", "bob", "alice", null, "carol"]
1037/// }
1038pub fn dict_from_values<K: ArrowDictionaryKeyType>(
1039    values_array: ArrayRef,
1040) -> Result<ArrayRef> {
1041    // Create a key array with `size` elements of 0..array_len for all
1042    // non-null value elements
1043    let key_array: PrimitiveArray<K> = (0..values_array.len())
1044        .map(|index| {
1045            if values_array.is_valid(index) {
1046                let native_index = K::Native::from_usize(index).ok_or_else(|| {
1047                    _internal_datafusion_err!(
1048                        "Can not create index of type {} from value {index}",
1049                        K::DATA_TYPE
1050                    )
1051                })?;
1052                Ok(Some(native_index))
1053            } else {
1054                Ok(None)
1055            }
1056        })
1057        .collect::<Result<Vec<_>>>()?
1058        .into_iter()
1059        .collect();
1060
1061    // create a new DictionaryArray
1062    //
1063    // Note: this path could be made faster by using the ArrayData
1064    // APIs and skipping validation, if it every comes up in
1065    // performance traces.
1066    let dict_array = DictionaryArray::<K>::try_new(key_array, values_array)?;
1067    Ok(Arc::new(dict_array))
1068}
1069
1070macro_rules! typed_cast_tz {
1071    ($array:expr, $index:expr, $array_cast:ident, $SCALAR:ident, $TZ:expr) => {{
1072        let array = $array_cast($array)?;
1073        Ok::<ScalarValue, DataFusionError>(ScalarValue::$SCALAR(
1074            match array.is_null($index) {
1075                true => None,
1076                false => Some(array.value($index).into()),
1077            },
1078            $TZ.clone(),
1079        ))
1080    }};
1081}
1082
1083macro_rules! typed_cast {
1084    ($array:expr, $index:expr, $array_cast:ident, $SCALAR:ident) => {{
1085        let array = $array_cast($array)?;
1086        Ok::<ScalarValue, DataFusionError>(ScalarValue::$SCALAR(
1087            match array.is_null($index) {
1088                true => None,
1089                false => Some(array.value($index).into()),
1090            },
1091        ))
1092    }};
1093}
1094
1095macro_rules! build_array_from_option {
1096    ($DATA_TYPE:ident, $ARRAY_TYPE:ident, $EXPR:expr, $SIZE:expr) => {{
1097        match $EXPR {
1098            Some(value) => Arc::new($ARRAY_TYPE::from_value(*value, $SIZE)),
1099            None => new_null_array(&DataType::$DATA_TYPE, $SIZE),
1100        }
1101    }};
1102    ($DATA_TYPE:ident, $ENUM:expr, $ARRAY_TYPE:ident, $EXPR:expr, $SIZE:expr) => {{
1103        match $EXPR {
1104            Some(value) => Arc::new($ARRAY_TYPE::from_value(*value, $SIZE)),
1105            None => new_null_array(&DataType::$DATA_TYPE($ENUM), $SIZE),
1106        }
1107    }};
1108}
1109
1110macro_rules! build_timestamp_array_from_option {
1111    ($TIME_UNIT:expr, $TZ:expr, $ARRAY_TYPE:ident, $EXPR:expr, $SIZE:expr) => {
1112        match $EXPR {
1113            Some(value) => {
1114                Arc::new($ARRAY_TYPE::from_value(*value, $SIZE).with_timezone_opt($TZ))
1115            }
1116            None => new_null_array(&DataType::Timestamp($TIME_UNIT, $TZ), $SIZE),
1117        }
1118    };
1119}
1120
1121macro_rules! eq_array_primitive {
1122    ($array:expr, $index:expr, $array_cast:ident, $VALUE:expr) => {{
1123        let array = $array_cast($array)?;
1124        let is_valid = array.is_valid($index);
1125        Ok::<bool, DataFusionError>(match $VALUE {
1126            Some(val) => is_valid && &array.value($index) == val,
1127            None => !is_valid,
1128        })
1129    }};
1130}
1131
1132impl ScalarValue {
1133    /// Create a [`Result<ScalarValue>`] with the provided value and datatype
1134    ///
1135    /// # Panics
1136    ///
1137    /// Panics if d is not compatible with T
1138    pub fn new_primitive<T: ArrowPrimitiveType>(
1139        a: Option<T::Native>,
1140        d: &DataType,
1141    ) -> Result<Self> {
1142        match a {
1143            None => d.try_into(),
1144            Some(v) => {
1145                let array = PrimitiveArray::<T>::new(vec![v].into(), None)
1146                    .with_data_type(d.clone());
1147                Self::try_from_array(&array, 0)
1148            }
1149        }
1150    }
1151
1152    /// Create a decimal Scalar from value/precision and scale.
1153    pub fn try_new_decimal128(value: i128, precision: u8, scale: i8) -> Result<Self> {
1154        Self::validate_decimal_or_internal_err::<Decimal128Type>(precision, scale)?;
1155        Ok(ScalarValue::Decimal128(Some(value), precision, scale))
1156    }
1157
1158    /// Create a Null instance of ScalarValue for this datatype
1159    ///
1160    /// Example
1161    /// ```
1162    /// use arrow::datatypes::DataType;
1163    /// use datafusion_common::ScalarValue;
1164    ///
1165    /// let scalar = ScalarValue::try_new_null(&DataType::Int32).unwrap();
1166    /// assert_eq!(scalar.is_null(), true);
1167    /// assert_eq!(scalar.data_type(), DataType::Int32);
1168    /// ```
1169    pub fn try_new_null(data_type: &DataType) -> Result<Self> {
1170        Ok(match data_type {
1171            DataType::Boolean => ScalarValue::Boolean(None),
1172            DataType::Float16 => ScalarValue::Float16(None),
1173            DataType::Float64 => ScalarValue::Float64(None),
1174            DataType::Float32 => ScalarValue::Float32(None),
1175            DataType::Int8 => ScalarValue::Int8(None),
1176            DataType::Int16 => ScalarValue::Int16(None),
1177            DataType::Int32 => ScalarValue::Int32(None),
1178            DataType::Int64 => ScalarValue::Int64(None),
1179            DataType::UInt8 => ScalarValue::UInt8(None),
1180            DataType::UInt16 => ScalarValue::UInt16(None),
1181            DataType::UInt32 => ScalarValue::UInt32(None),
1182            DataType::UInt64 => ScalarValue::UInt64(None),
1183            DataType::Decimal32(precision, scale) => {
1184                ScalarValue::Decimal32(None, *precision, *scale)
1185            }
1186            DataType::Decimal64(precision, scale) => {
1187                ScalarValue::Decimal64(None, *precision, *scale)
1188            }
1189            DataType::Decimal128(precision, scale) => {
1190                ScalarValue::Decimal128(None, *precision, *scale)
1191            }
1192            DataType::Decimal256(precision, scale) => {
1193                ScalarValue::Decimal256(None, *precision, *scale)
1194            }
1195            DataType::Utf8 => ScalarValue::Utf8(None),
1196            DataType::LargeUtf8 => ScalarValue::LargeUtf8(None),
1197            DataType::Utf8View => ScalarValue::Utf8View(None),
1198            DataType::Binary => ScalarValue::Binary(None),
1199            DataType::BinaryView => ScalarValue::BinaryView(None),
1200            DataType::FixedSizeBinary(len) => ScalarValue::FixedSizeBinary(*len, None),
1201            DataType::LargeBinary => ScalarValue::LargeBinary(None),
1202            DataType::Date32 => ScalarValue::Date32(None),
1203            DataType::Date64 => ScalarValue::Date64(None),
1204            DataType::Time32(TimeUnit::Second) => ScalarValue::Time32Second(None),
1205            DataType::Time32(TimeUnit::Millisecond) => {
1206                ScalarValue::Time32Millisecond(None)
1207            }
1208            DataType::Time64(TimeUnit::Microsecond) => {
1209                ScalarValue::Time64Microsecond(None)
1210            }
1211            DataType::Time64(TimeUnit::Nanosecond) => ScalarValue::Time64Nanosecond(None),
1212            DataType::Timestamp(TimeUnit::Second, tz_opt) => {
1213                ScalarValue::TimestampSecond(None, tz_opt.clone())
1214            }
1215            DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => {
1216                ScalarValue::TimestampMillisecond(None, tz_opt.clone())
1217            }
1218            DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => {
1219                ScalarValue::TimestampMicrosecond(None, tz_opt.clone())
1220            }
1221            DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => {
1222                ScalarValue::TimestampNanosecond(None, tz_opt.clone())
1223            }
1224            DataType::Interval(IntervalUnit::YearMonth) => {
1225                ScalarValue::IntervalYearMonth(None)
1226            }
1227            DataType::Interval(IntervalUnit::DayTime) => {
1228                ScalarValue::IntervalDayTime(None)
1229            }
1230            DataType::Interval(IntervalUnit::MonthDayNano) => {
1231                ScalarValue::IntervalMonthDayNano(None)
1232            }
1233            DataType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(None),
1234            DataType::Duration(TimeUnit::Millisecond) => {
1235                ScalarValue::DurationMillisecond(None)
1236            }
1237            DataType::Duration(TimeUnit::Microsecond) => {
1238                ScalarValue::DurationMicrosecond(None)
1239            }
1240            DataType::Duration(TimeUnit::Nanosecond) => {
1241                ScalarValue::DurationNanosecond(None)
1242            }
1243            DataType::Dictionary(index_type, value_type) => ScalarValue::Dictionary(
1244                index_type.clone(),
1245                Box::new(value_type.as_ref().try_into()?),
1246            ),
1247            // `ScalarValue::List` contains single element `ListArray`.
1248            DataType::List(field_ref) => ScalarValue::List(Arc::new(
1249                GenericListArray::new_null(Arc::clone(field_ref), 1),
1250            )),
1251            // `ScalarValue::LargeList` contains single element `LargeListArray`.
1252            DataType::LargeList(field_ref) => ScalarValue::LargeList(Arc::new(
1253                GenericListArray::new_null(Arc::clone(field_ref), 1),
1254            )),
1255            // `ScalarValue::FixedSizeList` contains single element `FixedSizeList`.
1256            DataType::FixedSizeList(field_ref, fixed_length) => {
1257                ScalarValue::FixedSizeList(Arc::new(FixedSizeListArray::new_null(
1258                    Arc::clone(field_ref),
1259                    *fixed_length,
1260                    1,
1261                )))
1262            }
1263            DataType::Struct(fields) => ScalarValue::Struct(
1264                new_null_array(&DataType::Struct(fields.to_owned()), 1)
1265                    .as_struct()
1266                    .to_owned()
1267                    .into(),
1268            ),
1269            DataType::Map(fields, sorted) => ScalarValue::Map(
1270                new_null_array(&DataType::Map(fields.to_owned(), sorted.to_owned()), 1)
1271                    .as_map()
1272                    .to_owned()
1273                    .into(),
1274            ),
1275            DataType::Union(fields, mode) => {
1276                ScalarValue::Union(None, fields.clone(), *mode)
1277            }
1278            DataType::Null => ScalarValue::Null,
1279            _ => {
1280                return _not_impl_err!(
1281                    "Can't create a null scalar from data_type \"{data_type}\""
1282                );
1283            }
1284        })
1285    }
1286
1287    /// Returns a [`ScalarValue::Utf8`] representing `val`
1288    pub fn new_utf8(val: impl Into<String>) -> Self {
1289        ScalarValue::from(val.into())
1290    }
1291
1292    /// Returns a [`ScalarValue::Utf8View`] representing `val`
1293    pub fn new_utf8view(val: impl Into<String>) -> Self {
1294        ScalarValue::Utf8View(Some(val.into()))
1295    }
1296
1297    /// Returns a [`ScalarValue::IntervalYearMonth`] representing
1298    /// `years` years and `months` months
1299    pub fn new_interval_ym(years: i32, months: i32) -> Self {
1300        let val = IntervalYearMonthType::make_value(years, months);
1301        ScalarValue::IntervalYearMonth(Some(val))
1302    }
1303
1304    /// Returns a [`ScalarValue::IntervalDayTime`] representing
1305    /// `days` days and `millis` milliseconds
1306    pub fn new_interval_dt(days: i32, millis: i32) -> Self {
1307        let val = IntervalDayTimeType::make_value(days, millis);
1308        Self::IntervalDayTime(Some(val))
1309    }
1310
1311    /// Returns a [`ScalarValue::IntervalMonthDayNano`] representing
1312    /// `months` months and `days` days, and `nanos` nanoseconds
1313    pub fn new_interval_mdn(months: i32, days: i32, nanos: i64) -> Self {
1314        let val = IntervalMonthDayNanoType::make_value(months, days, nanos);
1315        ScalarValue::IntervalMonthDayNano(Some(val))
1316    }
1317
1318    /// Returns a [`ScalarValue`] representing
1319    /// `value` and `tz_opt` timezone
1320    pub fn new_timestamp<T: ArrowTimestampType>(
1321        value: Option<i64>,
1322        tz_opt: Option<Arc<str>>,
1323    ) -> Self {
1324        match T::UNIT {
1325            TimeUnit::Second => ScalarValue::TimestampSecond(value, tz_opt),
1326            TimeUnit::Millisecond => ScalarValue::TimestampMillisecond(value, tz_opt),
1327            TimeUnit::Microsecond => ScalarValue::TimestampMicrosecond(value, tz_opt),
1328            TimeUnit::Nanosecond => ScalarValue::TimestampNanosecond(value, tz_opt),
1329        }
1330    }
1331
1332    /// Returns a [`ScalarValue`] representing PI
1333    pub fn new_pi(datatype: &DataType) -> Result<ScalarValue> {
1334        match datatype {
1335            DataType::Float16 => Ok(ScalarValue::from(f16::PI)),
1336            DataType::Float32 => Ok(ScalarValue::from(std::f32::consts::PI)),
1337            DataType::Float64 => Ok(ScalarValue::from(std::f64::consts::PI)),
1338            _ => _internal_err!("PI is not supported for data type: {}", datatype),
1339        }
1340    }
1341
1342    /// Returns a [`ScalarValue`] representing PI's upper bound
1343    pub fn new_pi_upper(datatype: &DataType) -> Result<ScalarValue> {
1344        match datatype {
1345            DataType::Float16 => Ok(ScalarValue::Float16(Some(consts::PI_UPPER_F16))),
1346            DataType::Float32 => Ok(ScalarValue::from(consts::PI_UPPER_F32)),
1347            DataType::Float64 => Ok(ScalarValue::from(consts::PI_UPPER_F64)),
1348            _ => {
1349                _internal_err!("PI_UPPER is not supported for data type: {}", datatype)
1350            }
1351        }
1352    }
1353
1354    /// Returns a [`ScalarValue`] representing -PI's lower bound
1355    pub fn new_negative_pi_lower(datatype: &DataType) -> Result<ScalarValue> {
1356        match datatype {
1357            DataType::Float16 => {
1358                Ok(ScalarValue::Float16(Some(consts::NEGATIVE_PI_LOWER_F16)))
1359            }
1360            DataType::Float32 => Ok(ScalarValue::from(consts::NEGATIVE_PI_LOWER_F32)),
1361            DataType::Float64 => Ok(ScalarValue::from(consts::NEGATIVE_PI_LOWER_F64)),
1362            _ => {
1363                _internal_err!("-PI_LOWER is not supported for data type: {}", datatype)
1364            }
1365        }
1366    }
1367
1368    /// Returns a [`ScalarValue`] representing FRAC_PI_2's upper bound
1369    pub fn new_frac_pi_2_upper(datatype: &DataType) -> Result<ScalarValue> {
1370        match datatype {
1371            DataType::Float16 => {
1372                Ok(ScalarValue::Float16(Some(consts::FRAC_PI_2_UPPER_F16)))
1373            }
1374            DataType::Float32 => Ok(ScalarValue::from(consts::FRAC_PI_2_UPPER_F32)),
1375            DataType::Float64 => Ok(ScalarValue::from(consts::FRAC_PI_2_UPPER_F64)),
1376            _ => {
1377                _internal_err!("PI_UPPER/2 is not supported for data type: {}", datatype)
1378            }
1379        }
1380    }
1381
1382    // Returns a [`ScalarValue`] representing FRAC_PI_2's lower bound
1383    pub fn new_neg_frac_pi_2_lower(datatype: &DataType) -> Result<ScalarValue> {
1384        match datatype {
1385            DataType::Float16 => Ok(ScalarValue::Float16(Some(
1386                consts::NEGATIVE_FRAC_PI_2_LOWER_F16,
1387            ))),
1388            DataType::Float32 => {
1389                Ok(ScalarValue::from(consts::NEGATIVE_FRAC_PI_2_LOWER_F32))
1390            }
1391            DataType::Float64 => {
1392                Ok(ScalarValue::from(consts::NEGATIVE_FRAC_PI_2_LOWER_F64))
1393            }
1394            _ => {
1395                _internal_err!("-PI/2_LOWER is not supported for data type: {}", datatype)
1396            }
1397        }
1398    }
1399
1400    /// Returns a [`ScalarValue`] representing -PI
1401    pub fn new_negative_pi(datatype: &DataType) -> Result<ScalarValue> {
1402        match datatype {
1403            DataType::Float16 => Ok(ScalarValue::from(-f16::PI)),
1404            DataType::Float32 => Ok(ScalarValue::from(-std::f32::consts::PI)),
1405            DataType::Float64 => Ok(ScalarValue::from(-std::f64::consts::PI)),
1406            _ => _internal_err!("-PI is not supported for data type: {}", datatype),
1407        }
1408    }
1409
1410    /// Returns a [`ScalarValue`] representing PI/2
1411    pub fn new_frac_pi_2(datatype: &DataType) -> Result<ScalarValue> {
1412        match datatype {
1413            DataType::Float16 => Ok(ScalarValue::from(f16::FRAC_PI_2)),
1414            DataType::Float32 => Ok(ScalarValue::from(std::f32::consts::FRAC_PI_2)),
1415            DataType::Float64 => Ok(ScalarValue::from(std::f64::consts::FRAC_PI_2)),
1416            _ => _internal_err!("PI/2 is not supported for data type: {}", datatype),
1417        }
1418    }
1419
1420    /// Returns a [`ScalarValue`] representing -PI/2
1421    pub fn new_neg_frac_pi_2(datatype: &DataType) -> Result<ScalarValue> {
1422        match datatype {
1423            DataType::Float16 => Ok(ScalarValue::from(-f16::FRAC_PI_2)),
1424            DataType::Float32 => Ok(ScalarValue::from(-std::f32::consts::FRAC_PI_2)),
1425            DataType::Float64 => Ok(ScalarValue::from(-std::f64::consts::FRAC_PI_2)),
1426            _ => _internal_err!("-PI/2 is not supported for data type: {}", datatype),
1427        }
1428    }
1429
1430    /// Returns a [`ScalarValue`] representing infinity
1431    pub fn new_infinity(datatype: &DataType) -> Result<ScalarValue> {
1432        match datatype {
1433            DataType::Float16 => Ok(ScalarValue::from(f16::INFINITY)),
1434            DataType::Float32 => Ok(ScalarValue::from(f32::INFINITY)),
1435            DataType::Float64 => Ok(ScalarValue::from(f64::INFINITY)),
1436            _ => {
1437                _internal_err!("Infinity is not supported for data type: {}", datatype)
1438            }
1439        }
1440    }
1441
1442    /// Returns a [`ScalarValue`] representing negative infinity
1443    pub fn new_neg_infinity(datatype: &DataType) -> Result<ScalarValue> {
1444        match datatype {
1445            DataType::Float16 => Ok(ScalarValue::from(f16::NEG_INFINITY)),
1446            DataType::Float32 => Ok(ScalarValue::from(f32::NEG_INFINITY)),
1447            DataType::Float64 => Ok(ScalarValue::from(f64::NEG_INFINITY)),
1448            _ => {
1449                _internal_err!(
1450                    "Negative Infinity is not supported for data type: {}",
1451                    datatype
1452                )
1453            }
1454        }
1455    }
1456
1457    /// Create a zero value in the given type.
1458    pub fn new_zero(datatype: &DataType) -> Result<ScalarValue> {
1459        Ok(match datatype {
1460            DataType::Boolean => ScalarValue::Boolean(Some(false)),
1461            DataType::Int8 => ScalarValue::Int8(Some(0)),
1462            DataType::Int16 => ScalarValue::Int16(Some(0)),
1463            DataType::Int32 => ScalarValue::Int32(Some(0)),
1464            DataType::Int64 => ScalarValue::Int64(Some(0)),
1465            DataType::UInt8 => ScalarValue::UInt8(Some(0)),
1466            DataType::UInt16 => ScalarValue::UInt16(Some(0)),
1467            DataType::UInt32 => ScalarValue::UInt32(Some(0)),
1468            DataType::UInt64 => ScalarValue::UInt64(Some(0)),
1469            DataType::Float16 => ScalarValue::Float16(Some(f16::ZERO)),
1470            DataType::Float32 => ScalarValue::Float32(Some(0.0)),
1471            DataType::Float64 => ScalarValue::Float64(Some(0.0)),
1472            DataType::Decimal32(precision, scale) => {
1473                ScalarValue::Decimal32(Some(0), *precision, *scale)
1474            }
1475            DataType::Decimal64(precision, scale) => {
1476                ScalarValue::Decimal64(Some(0), *precision, *scale)
1477            }
1478            DataType::Decimal128(precision, scale) => {
1479                ScalarValue::Decimal128(Some(0), *precision, *scale)
1480            }
1481            DataType::Decimal256(precision, scale) => {
1482                ScalarValue::Decimal256(Some(i256::ZERO), *precision, *scale)
1483            }
1484            DataType::Timestamp(TimeUnit::Second, tz) => {
1485                ScalarValue::TimestampSecond(Some(0), tz.clone())
1486            }
1487            DataType::Timestamp(TimeUnit::Millisecond, tz) => {
1488                ScalarValue::TimestampMillisecond(Some(0), tz.clone())
1489            }
1490            DataType::Timestamp(TimeUnit::Microsecond, tz) => {
1491                ScalarValue::TimestampMicrosecond(Some(0), tz.clone())
1492            }
1493            DataType::Timestamp(TimeUnit::Nanosecond, tz) => {
1494                ScalarValue::TimestampNanosecond(Some(0), tz.clone())
1495            }
1496            DataType::Time32(TimeUnit::Second) => ScalarValue::Time32Second(Some(0)),
1497            DataType::Time32(TimeUnit::Millisecond) => {
1498                ScalarValue::Time32Millisecond(Some(0))
1499            }
1500            DataType::Time64(TimeUnit::Microsecond) => {
1501                ScalarValue::Time64Microsecond(Some(0))
1502            }
1503            DataType::Time64(TimeUnit::Nanosecond) => {
1504                ScalarValue::Time64Nanosecond(Some(0))
1505            }
1506            DataType::Interval(IntervalUnit::YearMonth) => {
1507                ScalarValue::IntervalYearMonth(Some(0))
1508            }
1509            DataType::Interval(IntervalUnit::DayTime) => {
1510                ScalarValue::IntervalDayTime(Some(IntervalDayTime::ZERO))
1511            }
1512            DataType::Interval(IntervalUnit::MonthDayNano) => {
1513                ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano::ZERO))
1514            }
1515            DataType::Duration(TimeUnit::Second) => ScalarValue::DurationSecond(Some(0)),
1516            DataType::Duration(TimeUnit::Millisecond) => {
1517                ScalarValue::DurationMillisecond(Some(0))
1518            }
1519            DataType::Duration(TimeUnit::Microsecond) => {
1520                ScalarValue::DurationMicrosecond(Some(0))
1521            }
1522            DataType::Duration(TimeUnit::Nanosecond) => {
1523                ScalarValue::DurationNanosecond(Some(0))
1524            }
1525            DataType::Date32 => ScalarValue::Date32(Some(0)),
1526            DataType::Date64 => ScalarValue::Date64(Some(0)),
1527            _ => {
1528                return _not_impl_err!(
1529                    "Can't create a zero scalar from data_type \"{datatype}\""
1530                );
1531            }
1532        })
1533    }
1534
1535    /// Returns a default value for the given `DataType`.
1536    ///
1537    /// This function is useful when you need to initialize a column with
1538    /// non-null values in a DataFrame or when you need a "zero" value
1539    /// for a specific data type.
1540    ///
1541    /// # Default Values
1542    ///
1543    /// - **Numeric types**: Returns zero (via [`new_zero`])
1544    /// - **String types**: Returns empty string (`""`)
1545    /// - **Binary types**: Returns empty byte array
1546    /// - **Temporal types**: Returns zero/epoch value
1547    /// - **List types**: Returns empty list
1548    /// - **Struct types**: Returns struct with all fields set to their defaults
1549    /// - **Dictionary types**: Returns dictionary with default value
1550    /// - **Map types**: Returns empty map
1551    /// - **Union types**: Returns first variant with default value
1552    ///
1553    /// # Errors
1554    ///
1555    /// Returns an error for data types that don't have a clear default value
1556    /// or are not yet supported (e.g., `RunEndEncoded`).
1557    ///
1558    /// [`new_zero`]: Self::new_zero
1559    pub fn new_default(datatype: &DataType) -> Result<ScalarValue> {
1560        match datatype {
1561            // Null type
1562            DataType::Null => Ok(ScalarValue::Null),
1563
1564            // Numeric types
1565            DataType::Boolean
1566            | DataType::Int8
1567            | DataType::Int16
1568            | DataType::Int32
1569            | DataType::Int64
1570            | DataType::UInt8
1571            | DataType::UInt16
1572            | DataType::UInt32
1573            | DataType::UInt64
1574            | DataType::Float16
1575            | DataType::Float32
1576            | DataType::Float64
1577            | DataType::Decimal128(_, _)
1578            | DataType::Decimal256(_, _)
1579            | DataType::Timestamp(_, _)
1580            | DataType::Time32(_)
1581            | DataType::Time64(_)
1582            | DataType::Interval(_)
1583            | DataType::Duration(_)
1584            | DataType::Date32
1585            | DataType::Date64 => ScalarValue::new_zero(datatype),
1586
1587            // String types
1588            DataType::Utf8 => Ok(ScalarValue::Utf8(Some("".to_string()))),
1589            DataType::LargeUtf8 => Ok(ScalarValue::LargeUtf8(Some("".to_string()))),
1590            DataType::Utf8View => Ok(ScalarValue::Utf8View(Some("".to_string()))),
1591
1592            // Binary types
1593            DataType::Binary => Ok(ScalarValue::Binary(Some(vec![]))),
1594            DataType::LargeBinary => Ok(ScalarValue::LargeBinary(Some(vec![]))),
1595            DataType::BinaryView => Ok(ScalarValue::BinaryView(Some(vec![]))),
1596
1597            // Fixed-size binary
1598            DataType::FixedSizeBinary(size) => Ok(ScalarValue::FixedSizeBinary(
1599                *size,
1600                Some(vec![0; *size as usize]),
1601            )),
1602
1603            // List types
1604            DataType::List(field) => {
1605                let list =
1606                    ScalarValue::new_list(&[], field.data_type(), field.is_nullable());
1607                Ok(ScalarValue::List(list))
1608            }
1609            DataType::FixedSizeList(field, _size) => {
1610                let empty_arr = new_empty_array(field.data_type());
1611                let values = Arc::new(
1612                    SingleRowListArrayBuilder::new(empty_arr)
1613                        .with_nullable(field.is_nullable())
1614                        .build_fixed_size_list_array(0),
1615                );
1616                Ok(ScalarValue::FixedSizeList(values))
1617            }
1618            DataType::LargeList(field) => {
1619                let list = ScalarValue::new_large_list(&[], field.data_type());
1620                Ok(ScalarValue::LargeList(list))
1621            }
1622
1623            // Struct types
1624            DataType::Struct(fields) => {
1625                let values = fields
1626                    .iter()
1627                    .map(|f| ScalarValue::new_default(f.data_type()))
1628                    .collect::<Result<Vec<_>>>()?;
1629                Ok(ScalarValue::Struct(Arc::new(StructArray::new(
1630                    fields.clone(),
1631                    values
1632                        .into_iter()
1633                        .map(|v| v.to_array())
1634                        .collect::<Result<_>>()?,
1635                    None,
1636                ))))
1637            }
1638
1639            // Dictionary types
1640            DataType::Dictionary(key_type, value_type) => Ok(ScalarValue::Dictionary(
1641                key_type.clone(),
1642                Box::new(ScalarValue::new_default(value_type)?),
1643            )),
1644
1645            // Map types
1646            DataType::Map(field, _) => Ok(ScalarValue::Map(Arc::new(MapArray::from(
1647                ArrayData::new_empty(field.data_type()),
1648            )))),
1649
1650            // Union types - return first variant with default value
1651            DataType::Union(fields, mode) => {
1652                if let Some((type_id, field)) = fields.iter().next() {
1653                    let default_value = ScalarValue::new_default(field.data_type())?;
1654                    Ok(ScalarValue::Union(
1655                        Some((type_id, Box::new(default_value))),
1656                        fields.clone(),
1657                        *mode,
1658                    ))
1659                } else {
1660                    _internal_err!("Union type must have at least one field")
1661                }
1662            }
1663
1664            // Unsupported types for now
1665            _ => {
1666                _not_impl_err!(
1667                    "Default value for data_type \"{datatype}\" is not implemented yet"
1668                )
1669            }
1670        }
1671    }
1672
1673    /// Create an one value in the given type.
1674    pub fn new_one(datatype: &DataType) -> Result<ScalarValue> {
1675        Ok(match datatype {
1676            DataType::Int8 => ScalarValue::Int8(Some(1)),
1677            DataType::Int16 => ScalarValue::Int16(Some(1)),
1678            DataType::Int32 => ScalarValue::Int32(Some(1)),
1679            DataType::Int64 => ScalarValue::Int64(Some(1)),
1680            DataType::UInt8 => ScalarValue::UInt8(Some(1)),
1681            DataType::UInt16 => ScalarValue::UInt16(Some(1)),
1682            DataType::UInt32 => ScalarValue::UInt32(Some(1)),
1683            DataType::UInt64 => ScalarValue::UInt64(Some(1)),
1684            DataType::Float16 => ScalarValue::Float16(Some(f16::ONE)),
1685            DataType::Float32 => ScalarValue::Float32(Some(1.0)),
1686            DataType::Float64 => ScalarValue::Float64(Some(1.0)),
1687            DataType::Decimal32(precision, scale) => {
1688                Self::validate_decimal_or_internal_err::<Decimal32Type>(
1689                    *precision, *scale,
1690                )?;
1691                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1692                match 10_i32.checked_pow(*scale as u32) {
1693                    Some(value) => {
1694                        ScalarValue::Decimal32(Some(value), *precision, *scale)
1695                    }
1696                    None => return _internal_err!("Unsupported scale {scale}"),
1697                }
1698            }
1699            DataType::Decimal64(precision, scale) => {
1700                Self::validate_decimal_or_internal_err::<Decimal64Type>(
1701                    *precision, *scale,
1702                )?;
1703                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1704                match i64::from(10).checked_pow(*scale as u32) {
1705                    Some(value) => {
1706                        ScalarValue::Decimal64(Some(value), *precision, *scale)
1707                    }
1708                    None => return _internal_err!("Unsupported scale {scale}"),
1709                }
1710            }
1711            DataType::Decimal128(precision, scale) => {
1712                Self::validate_decimal_or_internal_err::<Decimal128Type>(
1713                    *precision, *scale,
1714                )?;
1715                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1716                match i128::from(10).checked_pow(*scale as u32) {
1717                    Some(value) => {
1718                        ScalarValue::Decimal128(Some(value), *precision, *scale)
1719                    }
1720                    None => return _internal_err!("Unsupported scale {scale}"),
1721                }
1722            }
1723            DataType::Decimal256(precision, scale) => {
1724                Self::validate_decimal_or_internal_err::<Decimal256Type>(
1725                    *precision, *scale,
1726                )?;
1727                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1728                match i256::from(10).checked_pow(*scale as u32) {
1729                    Some(value) => {
1730                        ScalarValue::Decimal256(Some(value), *precision, *scale)
1731                    }
1732                    None => return _internal_err!("Unsupported scale {scale}"),
1733                }
1734            }
1735            _ => {
1736                return _not_impl_err!(
1737                    "Can't create an one scalar from data_type \"{datatype}\""
1738                );
1739            }
1740        })
1741    }
1742
1743    /// Create a negative one value in the given type.
1744    pub fn new_negative_one(datatype: &DataType) -> Result<ScalarValue> {
1745        Ok(match datatype {
1746            DataType::Int8 | DataType::UInt8 => ScalarValue::Int8(Some(-1)),
1747            DataType::Int16 | DataType::UInt16 => ScalarValue::Int16(Some(-1)),
1748            DataType::Int32 | DataType::UInt32 => ScalarValue::Int32(Some(-1)),
1749            DataType::Int64 | DataType::UInt64 => ScalarValue::Int64(Some(-1)),
1750            DataType::Float16 => ScalarValue::Float16(Some(f16::NEG_ONE)),
1751            DataType::Float32 => ScalarValue::Float32(Some(-1.0)),
1752            DataType::Float64 => ScalarValue::Float64(Some(-1.0)),
1753            DataType::Decimal32(precision, scale) => {
1754                Self::validate_decimal_or_internal_err::<Decimal32Type>(
1755                    *precision, *scale,
1756                )?;
1757                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1758                match 10_i32.checked_pow(*scale as u32) {
1759                    Some(value) => {
1760                        ScalarValue::Decimal32(Some(-value), *precision, *scale)
1761                    }
1762                    None => return _internal_err!("Unsupported scale {scale}"),
1763                }
1764            }
1765            DataType::Decimal64(precision, scale) => {
1766                Self::validate_decimal_or_internal_err::<Decimal64Type>(
1767                    *precision, *scale,
1768                )?;
1769                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1770                match i64::from(10).checked_pow(*scale as u32) {
1771                    Some(value) => {
1772                        ScalarValue::Decimal64(Some(-value), *precision, *scale)
1773                    }
1774                    None => return _internal_err!("Unsupported scale {scale}"),
1775                }
1776            }
1777            DataType::Decimal128(precision, scale) => {
1778                Self::validate_decimal_or_internal_err::<Decimal128Type>(
1779                    *precision, *scale,
1780                )?;
1781                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1782                match i128::from(10).checked_pow(*scale as u32) {
1783                    Some(value) => {
1784                        ScalarValue::Decimal128(Some(-value), *precision, *scale)
1785                    }
1786                    None => return _internal_err!("Unsupported scale {scale}"),
1787                }
1788            }
1789            DataType::Decimal256(precision, scale) => {
1790                Self::validate_decimal_or_internal_err::<Decimal256Type>(
1791                    *precision, *scale,
1792                )?;
1793                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1794                match i256::from(10).checked_pow(*scale as u32) {
1795                    Some(value) => {
1796                        ScalarValue::Decimal256(Some(-value), *precision, *scale)
1797                    }
1798                    None => return _internal_err!("Unsupported scale {scale}"),
1799                }
1800            }
1801            _ => {
1802                return _not_impl_err!(
1803                    "Can't create a negative one scalar from data_type \"{datatype}\""
1804                );
1805            }
1806        })
1807    }
1808
1809    pub fn new_ten(datatype: &DataType) -> Result<ScalarValue> {
1810        Ok(match datatype {
1811            DataType::Int8 => ScalarValue::Int8(Some(10)),
1812            DataType::Int16 => ScalarValue::Int16(Some(10)),
1813            DataType::Int32 => ScalarValue::Int32(Some(10)),
1814            DataType::Int64 => ScalarValue::Int64(Some(10)),
1815            DataType::UInt8 => ScalarValue::UInt8(Some(10)),
1816            DataType::UInt16 => ScalarValue::UInt16(Some(10)),
1817            DataType::UInt32 => ScalarValue::UInt32(Some(10)),
1818            DataType::UInt64 => ScalarValue::UInt64(Some(10)),
1819            DataType::Float16 => ScalarValue::Float16(Some(f16::from_f32(10.0))),
1820            DataType::Float32 => ScalarValue::Float32(Some(10.0)),
1821            DataType::Float64 => ScalarValue::Float64(Some(10.0)),
1822            DataType::Decimal32(precision, scale) => {
1823                Self::validate_decimal_or_internal_err::<Decimal32Type>(
1824                    *precision, *scale,
1825                )?;
1826                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1827                match 10_i32.checked_pow((*scale + 1) as u32) {
1828                    Some(value) => {
1829                        ScalarValue::Decimal32(Some(value), *precision, *scale)
1830                    }
1831                    None => return _internal_err!("Unsupported scale {scale}"),
1832                }
1833            }
1834            DataType::Decimal64(precision, scale) => {
1835                Self::validate_decimal_or_internal_err::<Decimal64Type>(
1836                    *precision, *scale,
1837                )?;
1838                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1839                match i64::from(10).checked_pow((*scale + 1) as u32) {
1840                    Some(value) => {
1841                        ScalarValue::Decimal64(Some(value), *precision, *scale)
1842                    }
1843                    None => return _internal_err!("Unsupported scale {scale}"),
1844                }
1845            }
1846            DataType::Decimal128(precision, scale) => {
1847                Self::validate_decimal_or_internal_err::<Decimal128Type>(
1848                    *precision, *scale,
1849                )?;
1850                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1851                match i128::from(10).checked_pow((*scale + 1) as u32) {
1852                    Some(value) => {
1853                        ScalarValue::Decimal128(Some(value), *precision, *scale)
1854                    }
1855                    None => return _internal_err!("Unsupported scale {scale}"),
1856                }
1857            }
1858            DataType::Decimal256(precision, scale) => {
1859                Self::validate_decimal_or_internal_err::<Decimal256Type>(
1860                    *precision, *scale,
1861                )?;
1862                assert_or_internal_err!(*scale >= 0, "Negative scale is not supported");
1863                match i256::from(10).checked_pow((*scale + 1) as u32) {
1864                    Some(value) => {
1865                        ScalarValue::Decimal256(Some(value), *precision, *scale)
1866                    }
1867                    None => return _internal_err!("Unsupported scale {scale}"),
1868                }
1869            }
1870            _ => {
1871                return _not_impl_err!(
1872                    "Can't create a ten scalar from data_type \"{datatype}\""
1873                );
1874            }
1875        })
1876    }
1877
1878    /// return the [`DataType`] of this `ScalarValue`
1879    pub fn data_type(&self) -> DataType {
1880        match self {
1881            ScalarValue::Boolean(_) => DataType::Boolean,
1882            ScalarValue::UInt8(_) => DataType::UInt8,
1883            ScalarValue::UInt16(_) => DataType::UInt16,
1884            ScalarValue::UInt32(_) => DataType::UInt32,
1885            ScalarValue::UInt64(_) => DataType::UInt64,
1886            ScalarValue::Int8(_) => DataType::Int8,
1887            ScalarValue::Int16(_) => DataType::Int16,
1888            ScalarValue::Int32(_) => DataType::Int32,
1889            ScalarValue::Int64(_) => DataType::Int64,
1890            ScalarValue::Decimal32(_, precision, scale) => {
1891                DataType::Decimal32(*precision, *scale)
1892            }
1893            ScalarValue::Decimal64(_, precision, scale) => {
1894                DataType::Decimal64(*precision, *scale)
1895            }
1896            ScalarValue::Decimal128(_, precision, scale) => {
1897                DataType::Decimal128(*precision, *scale)
1898            }
1899            ScalarValue::Decimal256(_, precision, scale) => {
1900                DataType::Decimal256(*precision, *scale)
1901            }
1902            ScalarValue::TimestampSecond(_, tz_opt) => {
1903                DataType::Timestamp(TimeUnit::Second, tz_opt.clone())
1904            }
1905            ScalarValue::TimestampMillisecond(_, tz_opt) => {
1906                DataType::Timestamp(TimeUnit::Millisecond, tz_opt.clone())
1907            }
1908            ScalarValue::TimestampMicrosecond(_, tz_opt) => {
1909                DataType::Timestamp(TimeUnit::Microsecond, tz_opt.clone())
1910            }
1911            ScalarValue::TimestampNanosecond(_, tz_opt) => {
1912                DataType::Timestamp(TimeUnit::Nanosecond, tz_opt.clone())
1913            }
1914            ScalarValue::Float16(_) => DataType::Float16,
1915            ScalarValue::Float32(_) => DataType::Float32,
1916            ScalarValue::Float64(_) => DataType::Float64,
1917            ScalarValue::Utf8(_) => DataType::Utf8,
1918            ScalarValue::LargeUtf8(_) => DataType::LargeUtf8,
1919            ScalarValue::Utf8View(_) => DataType::Utf8View,
1920            ScalarValue::Binary(_) => DataType::Binary,
1921            ScalarValue::BinaryView(_) => DataType::BinaryView,
1922            ScalarValue::FixedSizeBinary(sz, _) => DataType::FixedSizeBinary(*sz),
1923            ScalarValue::LargeBinary(_) => DataType::LargeBinary,
1924            ScalarValue::List(arr) => arr.data_type().to_owned(),
1925            ScalarValue::LargeList(arr) => arr.data_type().to_owned(),
1926            ScalarValue::FixedSizeList(arr) => arr.data_type().to_owned(),
1927            ScalarValue::Struct(arr) => arr.data_type().to_owned(),
1928            ScalarValue::Map(arr) => arr.data_type().to_owned(),
1929            ScalarValue::Date32(_) => DataType::Date32,
1930            ScalarValue::Date64(_) => DataType::Date64,
1931            ScalarValue::Time32Second(_) => DataType::Time32(TimeUnit::Second),
1932            ScalarValue::Time32Millisecond(_) => DataType::Time32(TimeUnit::Millisecond),
1933            ScalarValue::Time64Microsecond(_) => DataType::Time64(TimeUnit::Microsecond),
1934            ScalarValue::Time64Nanosecond(_) => DataType::Time64(TimeUnit::Nanosecond),
1935            ScalarValue::IntervalYearMonth(_) => {
1936                DataType::Interval(IntervalUnit::YearMonth)
1937            }
1938            ScalarValue::IntervalDayTime(_) => DataType::Interval(IntervalUnit::DayTime),
1939            ScalarValue::IntervalMonthDayNano(_) => {
1940                DataType::Interval(IntervalUnit::MonthDayNano)
1941            }
1942            ScalarValue::DurationSecond(_) => DataType::Duration(TimeUnit::Second),
1943            ScalarValue::DurationMillisecond(_) => {
1944                DataType::Duration(TimeUnit::Millisecond)
1945            }
1946            ScalarValue::DurationMicrosecond(_) => {
1947                DataType::Duration(TimeUnit::Microsecond)
1948            }
1949            ScalarValue::DurationNanosecond(_) => {
1950                DataType::Duration(TimeUnit::Nanosecond)
1951            }
1952            ScalarValue::Union(_, fields, mode) => DataType::Union(fields.clone(), *mode),
1953            ScalarValue::Dictionary(k, v) => {
1954                DataType::Dictionary(k.clone(), Box::new(v.data_type()))
1955            }
1956            ScalarValue::Null => DataType::Null,
1957        }
1958    }
1959
1960    /// Calculate arithmetic negation for a scalar value
1961    pub fn arithmetic_negate(&self) -> Result<Self> {
1962        fn neg_checked_with_ctx<T: ArrowNativeTypeOp>(
1963            v: T,
1964            ctx: impl Fn() -> String,
1965        ) -> Result<T> {
1966            v.neg_checked()
1967                .map_err(|e| arrow_datafusion_err!(e).context(ctx()))
1968        }
1969        match self {
1970            ScalarValue::Int8(None)
1971            | ScalarValue::Int16(None)
1972            | ScalarValue::Int32(None)
1973            | ScalarValue::Int64(None)
1974            | ScalarValue::Float16(None)
1975            | ScalarValue::Float32(None)
1976            | ScalarValue::Float64(None) => Ok(self.clone()),
1977            ScalarValue::Float16(Some(v)) => Ok(ScalarValue::Float16(Some(-v))),
1978            ScalarValue::Float64(Some(v)) => Ok(ScalarValue::Float64(Some(-v))),
1979            ScalarValue::Float32(Some(v)) => Ok(ScalarValue::Float32(Some(-v))),
1980            ScalarValue::Int8(Some(v)) => Ok(ScalarValue::Int8(Some(v.neg_checked()?))),
1981            ScalarValue::Int16(Some(v)) => Ok(ScalarValue::Int16(Some(v.neg_checked()?))),
1982            ScalarValue::Int32(Some(v)) => Ok(ScalarValue::Int32(Some(v.neg_checked()?))),
1983            ScalarValue::Int64(Some(v)) => Ok(ScalarValue::Int64(Some(v.neg_checked()?))),
1984            ScalarValue::IntervalYearMonth(Some(v)) => Ok(
1985                ScalarValue::IntervalYearMonth(Some(neg_checked_with_ctx(*v, || {
1986                    format!("In negation of IntervalYearMonth({v})")
1987                })?)),
1988            ),
1989            ScalarValue::IntervalDayTime(Some(v)) => {
1990                let (days, ms) = IntervalDayTimeType::to_parts(*v);
1991                let val = IntervalDayTimeType::make_value(
1992                    neg_checked_with_ctx(days, || {
1993                        format!("In negation of days {days} in IntervalDayTime")
1994                    })?,
1995                    neg_checked_with_ctx(ms, || {
1996                        format!("In negation of milliseconds {ms} in IntervalDayTime")
1997                    })?,
1998                );
1999                Ok(ScalarValue::IntervalDayTime(Some(val)))
2000            }
2001            ScalarValue::IntervalMonthDayNano(Some(v)) => {
2002                let (months, days, nanos) = IntervalMonthDayNanoType::to_parts(*v);
2003                let val = IntervalMonthDayNanoType::make_value(
2004                    neg_checked_with_ctx(months, || {
2005                        format!("In negation of months {months} of IntervalMonthDayNano")
2006                    })?,
2007                    neg_checked_with_ctx(days, || {
2008                        format!("In negation of days {days} of IntervalMonthDayNano")
2009                    })?,
2010                    neg_checked_with_ctx(nanos, || {
2011                        format!("In negation of nanos {nanos} of IntervalMonthDayNano")
2012                    })?,
2013                );
2014                Ok(ScalarValue::IntervalMonthDayNano(Some(val)))
2015            }
2016            ScalarValue::Decimal32(Some(v), precision, scale) => {
2017                Ok(ScalarValue::Decimal32(
2018                    Some(neg_checked_with_ctx(*v, || {
2019                        format!("In negation of Decimal32({v}, {precision}, {scale})")
2020                    })?),
2021                    *precision,
2022                    *scale,
2023                ))
2024            }
2025            ScalarValue::Decimal64(Some(v), precision, scale) => {
2026                Ok(ScalarValue::Decimal64(
2027                    Some(neg_checked_with_ctx(*v, || {
2028                        format!("In negation of Decimal64({v}, {precision}, {scale})")
2029                    })?),
2030                    *precision,
2031                    *scale,
2032                ))
2033            }
2034            ScalarValue::Decimal128(Some(v), precision, scale) => {
2035                Ok(ScalarValue::Decimal128(
2036                    Some(neg_checked_with_ctx(*v, || {
2037                        format!("In negation of Decimal128({v}, {precision}, {scale})")
2038                    })?),
2039                    *precision,
2040                    *scale,
2041                ))
2042            }
2043            ScalarValue::Decimal256(Some(v), precision, scale) => {
2044                Ok(ScalarValue::Decimal256(
2045                    Some(neg_checked_with_ctx(*v, || {
2046                        format!("In negation of Decimal256({v}, {precision}, {scale})")
2047                    })?),
2048                    *precision,
2049                    *scale,
2050                ))
2051            }
2052            ScalarValue::TimestampSecond(Some(v), tz) => {
2053                Ok(ScalarValue::TimestampSecond(
2054                    Some(neg_checked_with_ctx(*v, || {
2055                        format!("In negation of TimestampSecond({v})")
2056                    })?),
2057                    tz.clone(),
2058                ))
2059            }
2060            ScalarValue::TimestampNanosecond(Some(v), tz) => {
2061                Ok(ScalarValue::TimestampNanosecond(
2062                    Some(neg_checked_with_ctx(*v, || {
2063                        format!("In negation of TimestampNanoSecond({v})")
2064                    })?),
2065                    tz.clone(),
2066                ))
2067            }
2068            ScalarValue::TimestampMicrosecond(Some(v), tz) => {
2069                Ok(ScalarValue::TimestampMicrosecond(
2070                    Some(neg_checked_with_ctx(*v, || {
2071                        format!("In negation of TimestampMicroSecond({v})")
2072                    })?),
2073                    tz.clone(),
2074                ))
2075            }
2076            ScalarValue::TimestampMillisecond(Some(v), tz) => {
2077                Ok(ScalarValue::TimestampMillisecond(
2078                    Some(neg_checked_with_ctx(*v, || {
2079                        format!("In negation of TimestampMilliSecond({v})")
2080                    })?),
2081                    tz.clone(),
2082                ))
2083            }
2084            value => _internal_err!(
2085                "Can not run arithmetic negative on scalar value {value:?}"
2086            ),
2087        }
2088    }
2089
2090    /// Wrapping addition of `ScalarValue`
2091    ///
2092    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2093    /// should operate on Arrays directly, using vectorized array kernels
2094    pub fn add<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2095        let r = add_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2096        Self::try_from_array(r.as_ref(), 0)
2097    }
2098
2099    /// Checked addition of `ScalarValue`
2100    ///
2101    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2102    /// should operate on Arrays directly, using vectorized array kernels
2103    pub fn add_checked<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2104        let r = add(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2105        Self::try_from_array(r.as_ref(), 0)
2106    }
2107
2108    /// Wrapping subtraction of `ScalarValue`
2109    ///
2110    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2111    /// should operate on Arrays directly, using vectorized array kernels
2112    pub fn sub<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2113        let r = sub_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2114        Self::try_from_array(r.as_ref(), 0)
2115    }
2116
2117    /// Checked subtraction of `ScalarValue`
2118    ///
2119    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2120    /// should operate on Arrays directly, using vectorized array kernels
2121    pub fn sub_checked<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2122        let r = sub(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2123        Self::try_from_array(r.as_ref(), 0)
2124    }
2125
2126    /// Wrapping multiplication of `ScalarValue`
2127    ///
2128    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2129    /// should operate on Arrays directly, using vectorized array kernels.
2130    pub fn mul<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2131        let r = mul_wrapping(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2132        Self::try_from_array(r.as_ref(), 0)
2133    }
2134
2135    /// Checked multiplication of `ScalarValue`
2136    ///
2137    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2138    /// should operate on Arrays directly, using vectorized array kernels.
2139    pub fn mul_checked<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2140        let r = mul(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2141        Self::try_from_array(r.as_ref(), 0)
2142    }
2143
2144    /// Performs `lhs / rhs`
2145    ///
2146    /// Overflow or division by zero will result in an error, with exception to
2147    /// floating point numbers, which instead follow the IEEE 754 rules.
2148    ///
2149    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2150    /// should operate on Arrays directly, using vectorized array kernels.
2151    pub fn div<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2152        let r = div(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2153        Self::try_from_array(r.as_ref(), 0)
2154    }
2155
2156    /// Performs `lhs % rhs`
2157    ///
2158    /// Overflow or division by zero will result in an error, with exception to
2159    /// floating point numbers, which instead follow the IEEE 754 rules.
2160    ///
2161    /// NB: operating on `ScalarValue` directly is not efficient, performance sensitive code
2162    /// should operate on Arrays directly, using vectorized array kernels.
2163    pub fn rem<T: Borrow<ScalarValue>>(&self, other: T) -> Result<ScalarValue> {
2164        let r = rem(&self.to_scalar()?, &other.borrow().to_scalar()?)?;
2165        Self::try_from_array(r.as_ref(), 0)
2166    }
2167
2168    pub fn is_unsigned(&self) -> bool {
2169        matches!(
2170            self,
2171            ScalarValue::UInt8(_)
2172                | ScalarValue::UInt16(_)
2173                | ScalarValue::UInt32(_)
2174                | ScalarValue::UInt64(_)
2175        )
2176    }
2177
2178    /// whether this value is null or not.
2179    pub fn is_null(&self) -> bool {
2180        match self {
2181            ScalarValue::Boolean(v) => v.is_none(),
2182            ScalarValue::Null => true,
2183            ScalarValue::Float16(v) => v.is_none(),
2184            ScalarValue::Float32(v) => v.is_none(),
2185            ScalarValue::Float64(v) => v.is_none(),
2186            ScalarValue::Decimal32(v, _, _) => v.is_none(),
2187            ScalarValue::Decimal64(v, _, _) => v.is_none(),
2188            ScalarValue::Decimal128(v, _, _) => v.is_none(),
2189            ScalarValue::Decimal256(v, _, _) => v.is_none(),
2190            ScalarValue::Int8(v) => v.is_none(),
2191            ScalarValue::Int16(v) => v.is_none(),
2192            ScalarValue::Int32(v) => v.is_none(),
2193            ScalarValue::Int64(v) => v.is_none(),
2194            ScalarValue::UInt8(v) => v.is_none(),
2195            ScalarValue::UInt16(v) => v.is_none(),
2196            ScalarValue::UInt32(v) => v.is_none(),
2197            ScalarValue::UInt64(v) => v.is_none(),
2198            ScalarValue::Utf8(v)
2199            | ScalarValue::Utf8View(v)
2200            | ScalarValue::LargeUtf8(v) => v.is_none(),
2201            ScalarValue::Binary(v)
2202            | ScalarValue::BinaryView(v)
2203            | ScalarValue::FixedSizeBinary(_, v)
2204            | ScalarValue::LargeBinary(v) => v.is_none(),
2205            // arr.len() should be 1 for a list scalar, but we don't seem to
2206            // enforce that anywhere, so we still check against array length.
2207            ScalarValue::List(arr) => arr.len() == arr.null_count(),
2208            ScalarValue::LargeList(arr) => arr.len() == arr.null_count(),
2209            ScalarValue::FixedSizeList(arr) => arr.len() == arr.null_count(),
2210            ScalarValue::Struct(arr) => arr.len() == arr.null_count(),
2211            ScalarValue::Map(arr) => arr.len() == arr.null_count(),
2212            ScalarValue::Date32(v) => v.is_none(),
2213            ScalarValue::Date64(v) => v.is_none(),
2214            ScalarValue::Time32Second(v) => v.is_none(),
2215            ScalarValue::Time32Millisecond(v) => v.is_none(),
2216            ScalarValue::Time64Microsecond(v) => v.is_none(),
2217            ScalarValue::Time64Nanosecond(v) => v.is_none(),
2218            ScalarValue::TimestampSecond(v, _) => v.is_none(),
2219            ScalarValue::TimestampMillisecond(v, _) => v.is_none(),
2220            ScalarValue::TimestampMicrosecond(v, _) => v.is_none(),
2221            ScalarValue::TimestampNanosecond(v, _) => v.is_none(),
2222            ScalarValue::IntervalYearMonth(v) => v.is_none(),
2223            ScalarValue::IntervalDayTime(v) => v.is_none(),
2224            ScalarValue::IntervalMonthDayNano(v) => v.is_none(),
2225            ScalarValue::DurationSecond(v) => v.is_none(),
2226            ScalarValue::DurationMillisecond(v) => v.is_none(),
2227            ScalarValue::DurationMicrosecond(v) => v.is_none(),
2228            ScalarValue::DurationNanosecond(v) => v.is_none(),
2229            ScalarValue::Union(v, _, _) => match v {
2230                Some((_, s)) => s.is_null(),
2231                None => true,
2232            },
2233            ScalarValue::Dictionary(_, v) => v.is_null(),
2234        }
2235    }
2236
2237    /// Absolute distance between two numeric values (of the same type). This method will return
2238    /// None if either one of the arguments are null. It might also return None if the resulting
2239    /// distance is greater than [`usize::MAX`]. If the type is a float, then the distance will be
2240    /// rounded to the nearest integer.
2241    ///
2242    ///
2243    /// Note: the datatype itself must support subtraction.
2244    pub fn distance(&self, other: &ScalarValue) -> Option<usize> {
2245        match (self, other) {
2246            (Self::Int8(Some(l)), Self::Int8(Some(r))) => Some(l.abs_diff(*r) as _),
2247            (Self::Int16(Some(l)), Self::Int16(Some(r))) => Some(l.abs_diff(*r) as _),
2248            (Self::Int32(Some(l)), Self::Int32(Some(r))) => Some(l.abs_diff(*r) as _),
2249            (Self::Int64(Some(l)), Self::Int64(Some(r))) => Some(l.abs_diff(*r) as _),
2250            (Self::UInt8(Some(l)), Self::UInt8(Some(r))) => Some(l.abs_diff(*r) as _),
2251            (Self::UInt16(Some(l)), Self::UInt16(Some(r))) => Some(l.abs_diff(*r) as _),
2252            (Self::UInt32(Some(l)), Self::UInt32(Some(r))) => Some(l.abs_diff(*r) as _),
2253            (Self::UInt64(Some(l)), Self::UInt64(Some(r))) => Some(l.abs_diff(*r) as _),
2254            // TODO: we might want to look into supporting ceil/floor here for floats.
2255            (Self::Float16(Some(l)), Self::Float16(Some(r))) => {
2256                Some((f16::to_f32(*l) - f16::to_f32(*r)).abs().round() as _)
2257            }
2258            (Self::Float32(Some(l)), Self::Float32(Some(r))) => {
2259                Some((l - r).abs().round() as _)
2260            }
2261            (Self::Float64(Some(l)), Self::Float64(Some(r))) => {
2262                Some((l - r).abs().round() as _)
2263            }
2264            (
2265                Self::Decimal128(Some(l), lprecision, lscale),
2266                Self::Decimal128(Some(r), rprecision, rscale),
2267            ) => {
2268                if lprecision == rprecision && lscale == rscale {
2269                    l.checked_sub(*r)?.checked_abs()?.to_usize()
2270                } else {
2271                    None
2272                }
2273            }
2274            (
2275                Self::Decimal256(Some(l), lprecision, lscale),
2276                Self::Decimal256(Some(r), rprecision, rscale),
2277            ) => {
2278                if lprecision == rprecision && lscale == rscale {
2279                    l.checked_sub(*r)?.checked_abs()?.to_usize()
2280                } else {
2281                    None
2282                }
2283            }
2284            _ => None,
2285        }
2286    }
2287
2288    /// Converts a scalar value into an 1-row array.
2289    ///
2290    /// # Errors
2291    ///
2292    /// Errors if the ScalarValue cannot be converted into a 1-row array
2293    pub fn to_array(&self) -> Result<ArrayRef> {
2294        self.to_array_of_size(1)
2295    }
2296
2297    /// Converts a scalar into an arrow [`Scalar`] (which implements
2298    /// the [`Datum`] interface).
2299    ///
2300    /// This can be used to call arrow compute kernels such as `lt`
2301    ///
2302    /// # Errors
2303    ///
2304    /// Errors if the ScalarValue cannot be converted into a 1-row array
2305    ///
2306    /// # Example
2307    /// ```
2308    /// use arrow::array::{BooleanArray, Int32Array};
2309    /// use datafusion_common::ScalarValue;
2310    ///
2311    /// let arr = Int32Array::from(vec![Some(1), None, Some(10)]);
2312    /// let five = ScalarValue::Int32(Some(5));
2313    ///
2314    /// let result =
2315    ///     arrow::compute::kernels::cmp::lt(&arr, &five.to_scalar().unwrap()).unwrap();
2316    ///
2317    /// let expected = BooleanArray::from(vec![Some(true), None, Some(false)]);
2318    ///
2319    /// assert_eq!(&result, &expected);
2320    /// ```
2321    /// [`Datum`]: arrow::array::Datum
2322    pub fn to_scalar(&self) -> Result<Scalar<ArrayRef>> {
2323        Ok(Scalar::new(self.to_array_of_size(1)?))
2324    }
2325
2326    /// Converts an iterator of references [`ScalarValue`] into an [`ArrayRef`]
2327    /// corresponding to those values. For example, an iterator of
2328    /// [`ScalarValue::Int32`] would be converted to an [`Int32Array`].
2329    ///
2330    /// Returns an error if the iterator is empty or if the
2331    /// [`ScalarValue`]s are not all the same type
2332    ///
2333    /// # Example
2334    /// ```
2335    /// use arrow::array::{ArrayRef, BooleanArray};
2336    /// use datafusion_common::ScalarValue;
2337    ///
2338    /// let scalars = vec![
2339    ///     ScalarValue::Boolean(Some(true)),
2340    ///     ScalarValue::Boolean(None),
2341    ///     ScalarValue::Boolean(Some(false)),
2342    /// ];
2343    ///
2344    /// // Build an Array from the list of ScalarValues
2345    /// let array = ScalarValue::iter_to_array(scalars.into_iter()).unwrap();
2346    ///
2347    /// let expected: ArrayRef =
2348    ///     std::sync::Arc::new(BooleanArray::from(vec![Some(true), None, Some(false)]));
2349    ///
2350    /// assert_eq!(&array, &expected);
2351    /// ```
2352    pub fn iter_to_array(
2353        scalars: impl IntoIterator<Item = ScalarValue>,
2354    ) -> Result<ArrayRef> {
2355        let mut scalars = scalars.into_iter().peekable();
2356
2357        // figure out the type based on the first element
2358        let data_type = match scalars.peek() {
2359            None => {
2360                return _exec_err!("Empty iterator passed to ScalarValue::iter_to_array");
2361            }
2362            Some(sv) => sv.data_type(),
2363        };
2364
2365        /// Creates an array of $ARRAY_TY by unpacking values of
2366        /// SCALAR_TY for primitive types
2367        macro_rules! build_array_primitive {
2368            ($ARRAY_TY:ident, $SCALAR_TY:ident) => {{
2369                {
2370                    let array = scalars
2371                        .map(|sv| {
2372                            if let ScalarValue::$SCALAR_TY(v) = sv {
2373                                Ok(v)
2374                            } else {
2375                                _exec_err!(
2376                                    "Inconsistent types in ScalarValue::iter_to_array. \
2377                                    Expected {:?}, got {:?}",
2378                                    data_type,
2379                                    sv
2380                                )
2381                            }
2382                        })
2383                        .collect::<Result<$ARRAY_TY>>()?;
2384                    Arc::new(array)
2385                }
2386            }};
2387        }
2388
2389        macro_rules! build_array_primitive_tz {
2390            ($ARRAY_TY:ident, $SCALAR_TY:ident, $TZ:expr) => {{
2391                {
2392                    let array = scalars
2393                        .map(|sv| {
2394                            if let ScalarValue::$SCALAR_TY(v, _) = sv {
2395                                Ok(v)
2396                            } else {
2397                                _exec_err!(
2398                                    "Inconsistent types in ScalarValue::iter_to_array. \
2399                                    Expected {:?}, got {:?}",
2400                                    data_type,
2401                                    sv
2402                                )
2403                            }
2404                        })
2405                        .collect::<Result<$ARRAY_TY>>()?;
2406                    Arc::new(array.with_timezone_opt($TZ.clone()))
2407                }
2408            }};
2409        }
2410
2411        /// Creates an array of $ARRAY_TY by unpacking values of
2412        /// SCALAR_TY for "string-like" types.
2413        macro_rules! build_array_string {
2414            ($ARRAY_TY:ident, $SCALAR_TY:ident) => {{
2415                {
2416                    let array = scalars
2417                        .map(|sv| {
2418                            if let ScalarValue::$SCALAR_TY(v) = sv {
2419                                Ok(v)
2420                            } else {
2421                                _exec_err!(
2422                                    "Inconsistent types in ScalarValue::iter_to_array. \
2423                                    Expected {:?}, got {:?}",
2424                                    data_type,
2425                                    sv
2426                                )
2427                            }
2428                        })
2429                        .collect::<Result<$ARRAY_TY>>()?;
2430                    Arc::new(array)
2431                }
2432            }};
2433        }
2434
2435        let array: ArrayRef = match &data_type {
2436            DataType::Decimal32(precision, scale) => {
2437                let decimal_array =
2438                    ScalarValue::iter_to_decimal32_array(scalars, *precision, *scale)?;
2439                Arc::new(decimal_array)
2440            }
2441            DataType::Decimal64(precision, scale) => {
2442                let decimal_array =
2443                    ScalarValue::iter_to_decimal64_array(scalars, *precision, *scale)?;
2444                Arc::new(decimal_array)
2445            }
2446            DataType::Decimal128(precision, scale) => {
2447                let decimal_array =
2448                    ScalarValue::iter_to_decimal128_array(scalars, *precision, *scale)?;
2449                Arc::new(decimal_array)
2450            }
2451            DataType::Decimal256(precision, scale) => {
2452                let decimal_array =
2453                    ScalarValue::iter_to_decimal256_array(scalars, *precision, *scale)?;
2454                Arc::new(decimal_array)
2455            }
2456            DataType::Null => ScalarValue::iter_to_null_array(scalars)?,
2457            DataType::Boolean => build_array_primitive!(BooleanArray, Boolean),
2458            DataType::Float16 => build_array_primitive!(Float16Array, Float16),
2459            DataType::Float32 => build_array_primitive!(Float32Array, Float32),
2460            DataType::Float64 => build_array_primitive!(Float64Array, Float64),
2461            DataType::Int8 => build_array_primitive!(Int8Array, Int8),
2462            DataType::Int16 => build_array_primitive!(Int16Array, Int16),
2463            DataType::Int32 => build_array_primitive!(Int32Array, Int32),
2464            DataType::Int64 => build_array_primitive!(Int64Array, Int64),
2465            DataType::UInt8 => build_array_primitive!(UInt8Array, UInt8),
2466            DataType::UInt16 => build_array_primitive!(UInt16Array, UInt16),
2467            DataType::UInt32 => build_array_primitive!(UInt32Array, UInt32),
2468            DataType::UInt64 => build_array_primitive!(UInt64Array, UInt64),
2469            DataType::Utf8View => build_array_string!(StringViewArray, Utf8View),
2470            DataType::Utf8 => build_array_string!(StringArray, Utf8),
2471            DataType::LargeUtf8 => build_array_string!(LargeStringArray, LargeUtf8),
2472            DataType::BinaryView => build_array_string!(BinaryViewArray, BinaryView),
2473            DataType::Binary => build_array_string!(BinaryArray, Binary),
2474            DataType::LargeBinary => build_array_string!(LargeBinaryArray, LargeBinary),
2475            DataType::Date32 => build_array_primitive!(Date32Array, Date32),
2476            DataType::Date64 => build_array_primitive!(Date64Array, Date64),
2477            DataType::Time32(TimeUnit::Second) => {
2478                build_array_primitive!(Time32SecondArray, Time32Second)
2479            }
2480            DataType::Time32(TimeUnit::Millisecond) => {
2481                build_array_primitive!(Time32MillisecondArray, Time32Millisecond)
2482            }
2483            DataType::Time64(TimeUnit::Microsecond) => {
2484                build_array_primitive!(Time64MicrosecondArray, Time64Microsecond)
2485            }
2486            DataType::Time64(TimeUnit::Nanosecond) => {
2487                build_array_primitive!(Time64NanosecondArray, Time64Nanosecond)
2488            }
2489            DataType::Timestamp(TimeUnit::Second, tz) => {
2490                build_array_primitive_tz!(TimestampSecondArray, TimestampSecond, tz)
2491            }
2492            DataType::Timestamp(TimeUnit::Millisecond, tz) => {
2493                build_array_primitive_tz!(
2494                    TimestampMillisecondArray,
2495                    TimestampMillisecond,
2496                    tz
2497                )
2498            }
2499            DataType::Timestamp(TimeUnit::Microsecond, tz) => {
2500                build_array_primitive_tz!(
2501                    TimestampMicrosecondArray,
2502                    TimestampMicrosecond,
2503                    tz
2504                )
2505            }
2506            DataType::Timestamp(TimeUnit::Nanosecond, tz) => {
2507                build_array_primitive_tz!(
2508                    TimestampNanosecondArray,
2509                    TimestampNanosecond,
2510                    tz
2511                )
2512            }
2513            DataType::Duration(TimeUnit::Second) => {
2514                build_array_primitive!(DurationSecondArray, DurationSecond)
2515            }
2516            DataType::Duration(TimeUnit::Millisecond) => {
2517                build_array_primitive!(DurationMillisecondArray, DurationMillisecond)
2518            }
2519            DataType::Duration(TimeUnit::Microsecond) => {
2520                build_array_primitive!(DurationMicrosecondArray, DurationMicrosecond)
2521            }
2522            DataType::Duration(TimeUnit::Nanosecond) => {
2523                build_array_primitive!(DurationNanosecondArray, DurationNanosecond)
2524            }
2525            DataType::Interval(IntervalUnit::DayTime) => {
2526                build_array_primitive!(IntervalDayTimeArray, IntervalDayTime)
2527            }
2528            DataType::Interval(IntervalUnit::YearMonth) => {
2529                build_array_primitive!(IntervalYearMonthArray, IntervalYearMonth)
2530            }
2531            DataType::Interval(IntervalUnit::MonthDayNano) => {
2532                build_array_primitive!(IntervalMonthDayNanoArray, IntervalMonthDayNano)
2533            }
2534            DataType::FixedSizeList(_, _) => {
2535                // arrow::compute::concat does not allow inconsistent types including the size of FixedSizeList.
2536                // The length of nulls here we got is 1, so we need to resize the length of nulls to
2537                // the length of non-nulls.
2538                let mut arrays =
2539                    scalars.map(|s| s.to_array()).collect::<Result<Vec<_>>>()?;
2540                let first_non_null_data_type = arrays
2541                    .iter()
2542                    .find(|sv| !sv.is_null(0))
2543                    .map(|sv| sv.data_type().to_owned());
2544                if let Some(DataType::FixedSizeList(f, l)) = first_non_null_data_type {
2545                    for array in arrays.iter_mut() {
2546                        if array.is_null(0) {
2547                            *array = Arc::new(FixedSizeListArray::new_null(
2548                                Arc::clone(&f),
2549                                l,
2550                                1,
2551                            ));
2552                        }
2553                    }
2554                }
2555                let arrays = arrays.iter().map(|a| a.as_ref()).collect::<Vec<_>>();
2556                arrow::compute::concat(arrays.as_slice())?
2557            }
2558            DataType::List(_)
2559            | DataType::LargeList(_)
2560            | DataType::Map(_, _)
2561            | DataType::Struct(_)
2562            | DataType::Union(_, _) => {
2563                let arrays = scalars.map(|s| s.to_array()).collect::<Result<Vec<_>>>()?;
2564                let arrays = arrays.iter().map(|a| a.as_ref()).collect::<Vec<_>>();
2565                arrow::compute::concat(arrays.as_slice())?
2566            }
2567            DataType::Dictionary(key_type, value_type) => {
2568                // create the values array
2569                let value_scalars = scalars
2570                    .map(|scalar| match scalar {
2571                        ScalarValue::Dictionary(inner_key_type, scalar) => {
2572                            if &inner_key_type == key_type {
2573                                Ok(*scalar)
2574                            } else {
2575                                _exec_err!("Expected inner key type of {key_type} but found: {inner_key_type}, value was ({scalar:?})")
2576                            }
2577                        }
2578                        _ => {
2579                            _exec_err!(
2580                                "Expected scalar of type {value_type} but found: {scalar} {scalar:?}"
2581                            )
2582                        }
2583                    })
2584                    .collect::<Result<Vec<_>>>()?;
2585
2586                let values = Self::iter_to_array(value_scalars)?;
2587                assert_eq!(values.data_type(), value_type.as_ref());
2588
2589                match key_type.as_ref() {
2590                    DataType::Int8 => dict_from_values::<Int8Type>(values)?,
2591                    DataType::Int16 => dict_from_values::<Int16Type>(values)?,
2592                    DataType::Int32 => dict_from_values::<Int32Type>(values)?,
2593                    DataType::Int64 => dict_from_values::<Int64Type>(values)?,
2594                    DataType::UInt8 => dict_from_values::<UInt8Type>(values)?,
2595                    DataType::UInt16 => dict_from_values::<UInt16Type>(values)?,
2596                    DataType::UInt32 => dict_from_values::<UInt32Type>(values)?,
2597                    DataType::UInt64 => dict_from_values::<UInt64Type>(values)?,
2598                    _ => unreachable!("Invalid dictionary keys type: {}", key_type),
2599                }
2600            }
2601            DataType::FixedSizeBinary(size) => {
2602                let array = scalars
2603                    .map(|sv| {
2604                        if let ScalarValue::FixedSizeBinary(_, v) = sv {
2605                            Ok(v)
2606                        } else {
2607                            _exec_err!(
2608                                "Inconsistent types in ScalarValue::iter_to_array. \
2609                                Expected {data_type}, got {sv:?}"
2610                            )
2611                        }
2612                    })
2613                    .collect::<Result<Vec<_>>>()?;
2614                let array = FixedSizeBinaryArray::try_from_sparse_iter_with_size(
2615                    array.into_iter(),
2616                    *size,
2617                )?;
2618                Arc::new(array)
2619            }
2620            // explicitly enumerate unsupported types so newly added
2621            // types must be acknowledged, Time32 and Time64 types are
2622            // not supported if the TimeUnit is not valid (Time32 can
2623            // only be used with Second and Millisecond, Time64 only
2624            // with Microsecond and Nanosecond)
2625            DataType::Time32(TimeUnit::Microsecond)
2626            | DataType::Time32(TimeUnit::Nanosecond)
2627            | DataType::Time64(TimeUnit::Second)
2628            | DataType::Time64(TimeUnit::Millisecond)
2629            | DataType::RunEndEncoded(_, _)
2630            | DataType::ListView(_)
2631            | DataType::LargeListView(_) => {
2632                return _not_impl_err!(
2633                    "Unsupported creation of {:?} array from ScalarValue {:?}",
2634                    data_type,
2635                    scalars.peek()
2636                );
2637            }
2638        };
2639        Ok(array)
2640    }
2641
2642    fn iter_to_null_array(
2643        scalars: impl IntoIterator<Item = ScalarValue>,
2644    ) -> Result<ArrayRef> {
2645        let length = scalars.into_iter().try_fold(
2646            0usize,
2647            |r, element: ScalarValue| match element {
2648                ScalarValue::Null => Ok::<usize, DataFusionError>(r + 1),
2649                s => {
2650                    _internal_err!("Expected ScalarValue::Null element. Received {s:?}")
2651                }
2652            },
2653        )?;
2654        Ok(new_null_array(&DataType::Null, length))
2655    }
2656
2657    fn iter_to_decimal32_array(
2658        scalars: impl IntoIterator<Item = ScalarValue>,
2659        precision: u8,
2660        scale: i8,
2661    ) -> Result<Decimal32Array> {
2662        let array = scalars
2663            .into_iter()
2664            .map(|element: ScalarValue| match element {
2665                ScalarValue::Decimal32(v1, _, _) => Ok(v1),
2666                s => {
2667                    _internal_err!("Expected ScalarValue::Null element. Received {s:?}")
2668                }
2669            })
2670            .collect::<Result<Decimal32Array>>()?
2671            .with_precision_and_scale(precision, scale)?;
2672        Ok(array)
2673    }
2674
2675    fn iter_to_decimal64_array(
2676        scalars: impl IntoIterator<Item = ScalarValue>,
2677        precision: u8,
2678        scale: i8,
2679    ) -> Result<Decimal64Array> {
2680        let array = scalars
2681            .into_iter()
2682            .map(|element: ScalarValue| match element {
2683                ScalarValue::Decimal64(v1, _, _) => Ok(v1),
2684                s => {
2685                    _internal_err!("Expected ScalarValue::Null element. Received {s:?}")
2686                }
2687            })
2688            .collect::<Result<Decimal64Array>>()?
2689            .with_precision_and_scale(precision, scale)?;
2690        Ok(array)
2691    }
2692
2693    fn iter_to_decimal128_array(
2694        scalars: impl IntoIterator<Item = ScalarValue>,
2695        precision: u8,
2696        scale: i8,
2697    ) -> Result<Decimal128Array> {
2698        let array = scalars
2699            .into_iter()
2700            .map(|element: ScalarValue| match element {
2701                ScalarValue::Decimal128(v1, _, _) => Ok(v1),
2702                s => {
2703                    _internal_err!("Expected ScalarValue::Null element. Received {s:?}")
2704                }
2705            })
2706            .collect::<Result<Decimal128Array>>()?
2707            .with_precision_and_scale(precision, scale)?;
2708        Ok(array)
2709    }
2710
2711    fn iter_to_decimal256_array(
2712        scalars: impl IntoIterator<Item = ScalarValue>,
2713        precision: u8,
2714        scale: i8,
2715    ) -> Result<Decimal256Array> {
2716        let array = scalars
2717            .into_iter()
2718            .map(|element: ScalarValue| match element {
2719                ScalarValue::Decimal256(v1, _, _) => Ok(v1),
2720                s => {
2721                    _internal_err!(
2722                        "Expected ScalarValue::Decimal256 element. Received {s:?}"
2723                    )
2724                }
2725            })
2726            .collect::<Result<Decimal256Array>>()?
2727            .with_precision_and_scale(precision, scale)?;
2728        Ok(array)
2729    }
2730
2731    /// Converts `Vec<ScalarValue>` where each element has type corresponding to
2732    /// `data_type`, to a single element [`ListArray`].
2733    ///
2734    /// Example
2735    /// ```
2736    /// use arrow::array::{Int32Array, ListArray};
2737    /// use arrow::datatypes::{DataType, Int32Type};
2738    /// use datafusion_common::cast::as_list_array;
2739    /// use datafusion_common::ScalarValue;
2740    ///
2741    /// let scalars = vec![
2742    ///     ScalarValue::Int32(Some(1)),
2743    ///     ScalarValue::Int32(None),
2744    ///     ScalarValue::Int32(Some(2)),
2745    /// ];
2746    ///
2747    /// let result = ScalarValue::new_list(&scalars, &DataType::Int32, true);
2748    ///
2749    /// let expected = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
2750    ///     Some(1),
2751    ///     None,
2752    ///     Some(2),
2753    /// ])]);
2754    ///
2755    /// assert_eq!(*result, expected);
2756    /// ```
2757    pub fn new_list(
2758        values: &[ScalarValue],
2759        data_type: &DataType,
2760        nullable: bool,
2761    ) -> Arc<ListArray> {
2762        let values = if values.is_empty() {
2763            new_empty_array(data_type)
2764        } else {
2765            Self::iter_to_array(values.iter().cloned()).unwrap()
2766        };
2767        Arc::new(
2768            SingleRowListArrayBuilder::new(values)
2769                .with_nullable(nullable)
2770                .build_list_array(),
2771        )
2772    }
2773
2774    /// Same as [`ScalarValue::new_list`] but with nullable set to true.
2775    pub fn new_list_nullable(
2776        values: &[ScalarValue],
2777        data_type: &DataType,
2778    ) -> Arc<ListArray> {
2779        Self::new_list(values, data_type, true)
2780    }
2781
2782    /// Create ListArray with Null with specific data type
2783    ///
2784    /// - new_null_list(i32, nullable, 1): `ListArray[NULL]`
2785    pub fn new_null_list(data_type: DataType, nullable: bool, null_len: usize) -> Self {
2786        let data_type = DataType::List(Field::new_list_field(data_type, nullable).into());
2787        Self::List(Arc::new(ListArray::from(ArrayData::new_null(
2788            &data_type, null_len,
2789        ))))
2790    }
2791
2792    /// Converts `IntoIterator<Item = ScalarValue>` where each element has type corresponding to
2793    /// `data_type`, to a [`ListArray`].
2794    ///
2795    /// Example
2796    /// ```
2797    /// use arrow::array::{Int32Array, ListArray};
2798    /// use arrow::datatypes::{DataType, Int32Type};
2799    /// use datafusion_common::cast::as_list_array;
2800    /// use datafusion_common::ScalarValue;
2801    ///
2802    /// let scalars = vec![
2803    ///     ScalarValue::Int32(Some(1)),
2804    ///     ScalarValue::Int32(None),
2805    ///     ScalarValue::Int32(Some(2)),
2806    /// ];
2807    ///
2808    /// let result =
2809    ///     ScalarValue::new_list_from_iter(scalars.into_iter(), &DataType::Int32, true);
2810    ///
2811    /// let expected = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
2812    ///     Some(1),
2813    ///     None,
2814    ///     Some(2),
2815    /// ])]);
2816    ///
2817    /// assert_eq!(*result, expected);
2818    /// ```
2819    pub fn new_list_from_iter(
2820        values: impl IntoIterator<Item = ScalarValue> + ExactSizeIterator,
2821        data_type: &DataType,
2822        nullable: bool,
2823    ) -> Arc<ListArray> {
2824        let values = if values.len() == 0 {
2825            new_empty_array(data_type)
2826        } else {
2827            Self::iter_to_array(values).unwrap()
2828        };
2829        Arc::new(
2830            SingleRowListArrayBuilder::new(values)
2831                .with_nullable(nullable)
2832                .build_list_array(),
2833        )
2834    }
2835
2836    /// Converts `Vec<ScalarValue>` where each element has type corresponding to
2837    /// `data_type`, to a [`LargeListArray`].
2838    ///
2839    /// Example
2840    /// ```
2841    /// use arrow::array::{Int32Array, LargeListArray};
2842    /// use arrow::datatypes::{DataType, Int32Type};
2843    /// use datafusion_common::cast::as_large_list_array;
2844    /// use datafusion_common::ScalarValue;
2845    ///
2846    /// let scalars = vec![
2847    ///     ScalarValue::Int32(Some(1)),
2848    ///     ScalarValue::Int32(None),
2849    ///     ScalarValue::Int32(Some(2)),
2850    /// ];
2851    ///
2852    /// let result = ScalarValue::new_large_list(&scalars, &DataType::Int32);
2853    ///
2854    /// let expected =
2855    ///     LargeListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
2856    ///         Some(1),
2857    ///         None,
2858    ///         Some(2),
2859    ///     ])]);
2860    ///
2861    /// assert_eq!(*result, expected);
2862    /// ```
2863    pub fn new_large_list(
2864        values: &[ScalarValue],
2865        data_type: &DataType,
2866    ) -> Arc<LargeListArray> {
2867        let values = if values.is_empty() {
2868            new_empty_array(data_type)
2869        } else {
2870            Self::iter_to_array(values.iter().cloned()).unwrap()
2871        };
2872        Arc::new(SingleRowListArrayBuilder::new(values).build_large_list_array())
2873    }
2874
2875    /// Converts a scalar value into an array of `size` rows.
2876    ///
2877    /// # Errors
2878    ///
2879    /// Errors if `self` is
2880    /// - a decimal that fails be converted to a decimal array of size
2881    /// - a `FixedsizeList` that fails to be concatenated into an array of size
2882    /// - a `List` that fails to be concatenated into an array of size
2883    /// - a `Dictionary` that fails be converted to a dictionary array of size
2884    pub fn to_array_of_size(&self, size: usize) -> Result<ArrayRef> {
2885        Ok(match self {
2886            ScalarValue::Decimal32(Some(e), precision, scale) => Arc::new(
2887                Decimal32Array::from_value(*e, size)
2888                    .with_precision_and_scale(*precision, *scale)?,
2889            ),
2890            ScalarValue::Decimal32(None, precision, scale) => {
2891                new_null_array(&DataType::Decimal32(*precision, *scale), size)
2892            }
2893            ScalarValue::Decimal64(Some(e), precision, scale) => Arc::new(
2894                Decimal64Array::from_value(*e, size)
2895                    .with_precision_and_scale(*precision, *scale)?,
2896            ),
2897            ScalarValue::Decimal64(None, precision, scale) => {
2898                new_null_array(&DataType::Decimal64(*precision, *scale), size)
2899            }
2900            ScalarValue::Decimal128(Some(e), precision, scale) => Arc::new(
2901                Decimal128Array::from_value(*e, size)
2902                    .with_precision_and_scale(*precision, *scale)?,
2903            ),
2904            ScalarValue::Decimal128(None, precision, scale) => {
2905                new_null_array(&DataType::Decimal128(*precision, *scale), size)
2906            }
2907            ScalarValue::Decimal256(Some(e), precision, scale) => Arc::new(
2908                Decimal256Array::from_value(*e, size)
2909                    .with_precision_and_scale(*precision, *scale)?,
2910            ),
2911            ScalarValue::Decimal256(None, precision, scale) => {
2912                new_null_array(&DataType::Decimal256(*precision, *scale), size)
2913            }
2914
2915            ScalarValue::Boolean(e) => match e {
2916                None => new_null_array(&DataType::Boolean, size),
2917                Some(true) => {
2918                    Arc::new(BooleanArray::new(BooleanBuffer::new_set(size), None))
2919                        as ArrayRef
2920                }
2921                Some(false) => {
2922                    Arc::new(BooleanArray::new(BooleanBuffer::new_unset(size), None))
2923                        as ArrayRef
2924                }
2925            },
2926            ScalarValue::Float64(e) => {
2927                build_array_from_option!(Float64, Float64Array, e, size)
2928            }
2929            ScalarValue::Float32(e) => {
2930                build_array_from_option!(Float32, Float32Array, e, size)
2931            }
2932            ScalarValue::Float16(e) => {
2933                build_array_from_option!(Float16, Float16Array, e, size)
2934            }
2935            ScalarValue::Int8(e) => build_array_from_option!(Int8, Int8Array, e, size),
2936            ScalarValue::Int16(e) => build_array_from_option!(Int16, Int16Array, e, size),
2937            ScalarValue::Int32(e) => build_array_from_option!(Int32, Int32Array, e, size),
2938            ScalarValue::Int64(e) => build_array_from_option!(Int64, Int64Array, e, size),
2939            ScalarValue::UInt8(e) => build_array_from_option!(UInt8, UInt8Array, e, size),
2940            ScalarValue::UInt16(e) => {
2941                build_array_from_option!(UInt16, UInt16Array, e, size)
2942            }
2943            ScalarValue::UInt32(e) => {
2944                build_array_from_option!(UInt32, UInt32Array, e, size)
2945            }
2946            ScalarValue::UInt64(e) => {
2947                build_array_from_option!(UInt64, UInt64Array, e, size)
2948            }
2949            ScalarValue::TimestampSecond(e, tz_opt) => {
2950                build_timestamp_array_from_option!(
2951                    TimeUnit::Second,
2952                    tz_opt.clone(),
2953                    TimestampSecondArray,
2954                    e,
2955                    size
2956                )
2957            }
2958            ScalarValue::TimestampMillisecond(e, tz_opt) => {
2959                build_timestamp_array_from_option!(
2960                    TimeUnit::Millisecond,
2961                    tz_opt.clone(),
2962                    TimestampMillisecondArray,
2963                    e,
2964                    size
2965                )
2966            }
2967
2968            ScalarValue::TimestampMicrosecond(e, tz_opt) => {
2969                build_timestamp_array_from_option!(
2970                    TimeUnit::Microsecond,
2971                    tz_opt.clone(),
2972                    TimestampMicrosecondArray,
2973                    e,
2974                    size
2975                )
2976            }
2977            ScalarValue::TimestampNanosecond(e, tz_opt) => {
2978                build_timestamp_array_from_option!(
2979                    TimeUnit::Nanosecond,
2980                    tz_opt.clone(),
2981                    TimestampNanosecondArray,
2982                    e,
2983                    size
2984                )
2985            }
2986            ScalarValue::Utf8(e) => match e {
2987                Some(value) => Arc::new(StringArray::new_repeated(value, size)),
2988                None => new_null_array(&DataType::Utf8, size),
2989            },
2990            ScalarValue::Utf8View(e) => match e {
2991                Some(value) => {
2992                    let mut builder =
2993                        StringViewBuilder::with_capacity(size).with_deduplicate_strings();
2994                    // Replace with upstream arrow-rs code when available:
2995                    // https://github.com/apache/arrow-rs/issues/9034
2996                    for _ in 0..size {
2997                        builder.append_value(value);
2998                    }
2999                    let array = builder.finish();
3000                    Arc::new(array)
3001                }
3002                None => new_null_array(&DataType::Utf8View, size),
3003            },
3004            ScalarValue::LargeUtf8(e) => match e {
3005                Some(value) => Arc::new(LargeStringArray::new_repeated(value, size)),
3006                None => new_null_array(&DataType::LargeUtf8, size),
3007            },
3008            ScalarValue::Binary(e) => match e {
3009                Some(value) => {
3010                    Arc::new(BinaryArray::new_repeated(value.as_slice(), size))
3011                }
3012                None => new_null_array(&DataType::Binary, size),
3013            },
3014            ScalarValue::BinaryView(e) => match e {
3015                Some(value) => {
3016                    let mut builder =
3017                        BinaryViewBuilder::with_capacity(size).with_deduplicate_strings();
3018                    for _ in 0..size {
3019                        builder.append_value(value);
3020                    }
3021                    let array = builder.finish();
3022                    Arc::new(array)
3023                }
3024                None => new_null_array(&DataType::BinaryView, size),
3025            },
3026            ScalarValue::FixedSizeBinary(s, e) => match e {
3027                Some(value) => Arc::new(
3028                    FixedSizeBinaryArray::try_from_sparse_iter_with_size(
3029                        repeat_n(Some(value.as_slice()), size),
3030                        *s,
3031                    )
3032                    .unwrap(),
3033                ),
3034                None => {
3035                    // TODO: Replace with FixedSizeBinaryArray::new_null once a fix for
3036                    // https://github.com/apache/arrow-rs/issues/8900 is in the used arrow-rs
3037                    // version.
3038                    let mut builder = FixedSizeBinaryBuilder::new(*s);
3039                    builder.append_nulls(size);
3040                    Arc::new(builder.finish())
3041                }
3042            },
3043            ScalarValue::LargeBinary(e) => match e {
3044                Some(value) => {
3045                    Arc::new(LargeBinaryArray::new_repeated(value.as_slice(), size))
3046                }
3047                None => new_null_array(&DataType::LargeBinary, size),
3048            },
3049            ScalarValue::List(arr) => {
3050                if size == 1 {
3051                    return Ok(Arc::clone(arr) as Arc<dyn Array>);
3052                }
3053                Self::list_to_array_of_size(arr.as_ref() as &dyn Array, size)?
3054            }
3055            ScalarValue::LargeList(arr) => {
3056                if size == 1 {
3057                    return Ok(Arc::clone(arr) as Arc<dyn Array>);
3058                }
3059                Self::list_to_array_of_size(arr.as_ref() as &dyn Array, size)?
3060            }
3061            ScalarValue::FixedSizeList(arr) => {
3062                if size == 1 {
3063                    return Ok(Arc::clone(arr) as Arc<dyn Array>);
3064                }
3065                Self::list_to_array_of_size(arr.as_ref() as &dyn Array, size)?
3066            }
3067            ScalarValue::Struct(arr) => {
3068                if size == 1 {
3069                    return Ok(Arc::clone(arr) as Arc<dyn Array>);
3070                }
3071                Self::list_to_array_of_size(arr.as_ref() as &dyn Array, size)?
3072            }
3073            ScalarValue::Map(arr) => {
3074                if size == 1 {
3075                    return Ok(Arc::clone(arr) as Arc<dyn Array>);
3076                }
3077                Self::list_to_array_of_size(arr.as_ref() as &dyn Array, size)?
3078            }
3079            ScalarValue::Date32(e) => {
3080                build_array_from_option!(Date32, Date32Array, e, size)
3081            }
3082            ScalarValue::Date64(e) => {
3083                build_array_from_option!(Date64, Date64Array, e, size)
3084            }
3085            ScalarValue::Time32Second(e) => {
3086                build_array_from_option!(
3087                    Time32,
3088                    TimeUnit::Second,
3089                    Time32SecondArray,
3090                    e,
3091                    size
3092                )
3093            }
3094            ScalarValue::Time32Millisecond(e) => {
3095                build_array_from_option!(
3096                    Time32,
3097                    TimeUnit::Millisecond,
3098                    Time32MillisecondArray,
3099                    e,
3100                    size
3101                )
3102            }
3103            ScalarValue::Time64Microsecond(e) => {
3104                build_array_from_option!(
3105                    Time64,
3106                    TimeUnit::Microsecond,
3107                    Time64MicrosecondArray,
3108                    e,
3109                    size
3110                )
3111            }
3112            ScalarValue::Time64Nanosecond(e) => {
3113                build_array_from_option!(
3114                    Time64,
3115                    TimeUnit::Nanosecond,
3116                    Time64NanosecondArray,
3117                    e,
3118                    size
3119                )
3120            }
3121            ScalarValue::IntervalDayTime(e) => build_array_from_option!(
3122                Interval,
3123                IntervalUnit::DayTime,
3124                IntervalDayTimeArray,
3125                e,
3126                size
3127            ),
3128            ScalarValue::IntervalYearMonth(e) => build_array_from_option!(
3129                Interval,
3130                IntervalUnit::YearMonth,
3131                IntervalYearMonthArray,
3132                e,
3133                size
3134            ),
3135            ScalarValue::IntervalMonthDayNano(e) => build_array_from_option!(
3136                Interval,
3137                IntervalUnit::MonthDayNano,
3138                IntervalMonthDayNanoArray,
3139                e,
3140                size
3141            ),
3142            ScalarValue::DurationSecond(e) => build_array_from_option!(
3143                Duration,
3144                TimeUnit::Second,
3145                DurationSecondArray,
3146                e,
3147                size
3148            ),
3149            ScalarValue::DurationMillisecond(e) => build_array_from_option!(
3150                Duration,
3151                TimeUnit::Millisecond,
3152                DurationMillisecondArray,
3153                e,
3154                size
3155            ),
3156            ScalarValue::DurationMicrosecond(e) => build_array_from_option!(
3157                Duration,
3158                TimeUnit::Microsecond,
3159                DurationMicrosecondArray,
3160                e,
3161                size
3162            ),
3163            ScalarValue::DurationNanosecond(e) => build_array_from_option!(
3164                Duration,
3165                TimeUnit::Nanosecond,
3166                DurationNanosecondArray,
3167                e,
3168                size
3169            ),
3170            ScalarValue::Union(value, fields, mode) => match value {
3171                Some((v_id, value)) => {
3172                    let mut new_fields = Vec::with_capacity(fields.len());
3173                    let mut child_arrays = Vec::<ArrayRef>::with_capacity(fields.len());
3174                    for (f_id, field) in fields.iter() {
3175                        let ar = if f_id == *v_id {
3176                            value.to_array_of_size(size)?
3177                        } else {
3178                            let dt = field.data_type();
3179                            match mode {
3180                                UnionMode::Sparse => new_null_array(dt, size),
3181                                // In a dense union, only the child with values needs to be
3182                                // allocated
3183                                UnionMode::Dense => new_null_array(dt, 0),
3184                            }
3185                        };
3186                        let field = (**field).clone();
3187                        child_arrays.push(ar);
3188                        new_fields.push(field.clone());
3189                    }
3190                    let type_ids = repeat_n(*v_id, size);
3191                    let type_ids = ScalarBuffer::<i8>::from_iter(type_ids);
3192                    let value_offsets = match mode {
3193                        UnionMode::Sparse => None,
3194                        UnionMode::Dense => Some(ScalarBuffer::from_iter(0..size as i32)),
3195                    };
3196                    let ar = UnionArray::try_new(
3197                        fields.clone(),
3198                        type_ids,
3199                        value_offsets,
3200                        child_arrays,
3201                    )
3202                    .map_err(|e| DataFusionError::ArrowError(Box::new(e), None))?;
3203                    Arc::new(ar)
3204                }
3205                None => new_null_array(&DataType::Union(fields.clone(), *mode), size),
3206            },
3207            ScalarValue::Dictionary(key_type, v) => {
3208                // values array is one element long (the value)
3209                match key_type.as_ref() {
3210                    DataType::Int8 => dict_from_scalar::<Int8Type>(v, size)?,
3211                    DataType::Int16 => dict_from_scalar::<Int16Type>(v, size)?,
3212                    DataType::Int32 => dict_from_scalar::<Int32Type>(v, size)?,
3213                    DataType::Int64 => dict_from_scalar::<Int64Type>(v, size)?,
3214                    DataType::UInt8 => dict_from_scalar::<UInt8Type>(v, size)?,
3215                    DataType::UInt16 => dict_from_scalar::<UInt16Type>(v, size)?,
3216                    DataType::UInt32 => dict_from_scalar::<UInt32Type>(v, size)?,
3217                    DataType::UInt64 => dict_from_scalar::<UInt64Type>(v, size)?,
3218                    _ => unreachable!("Invalid dictionary keys type: {}", key_type),
3219                }
3220            }
3221            ScalarValue::Null => get_or_create_cached_null_array(size),
3222        })
3223    }
3224
3225    fn get_decimal_value_from_array(
3226        array: &dyn Array,
3227        index: usize,
3228        precision: u8,
3229        scale: i8,
3230    ) -> Result<ScalarValue> {
3231        match array.data_type() {
3232            DataType::Decimal32(_, _) => {
3233                let array = as_decimal32_array(array)?;
3234                if array.is_null(index) {
3235                    Ok(ScalarValue::Decimal32(None, precision, scale))
3236                } else {
3237                    let value = array.value(index);
3238                    Ok(ScalarValue::Decimal32(Some(value), precision, scale))
3239                }
3240            }
3241            DataType::Decimal64(_, _) => {
3242                let array = as_decimal64_array(array)?;
3243                if array.is_null(index) {
3244                    Ok(ScalarValue::Decimal64(None, precision, scale))
3245                } else {
3246                    let value = array.value(index);
3247                    Ok(ScalarValue::Decimal64(Some(value), precision, scale))
3248                }
3249            }
3250            DataType::Decimal128(_, _) => {
3251                let array = as_decimal128_array(array)?;
3252                if array.is_null(index) {
3253                    Ok(ScalarValue::Decimal128(None, precision, scale))
3254                } else {
3255                    let value = array.value(index);
3256                    Ok(ScalarValue::Decimal128(Some(value), precision, scale))
3257                }
3258            }
3259            DataType::Decimal256(_, _) => {
3260                let array = as_decimal256_array(array)?;
3261                if array.is_null(index) {
3262                    Ok(ScalarValue::Decimal256(None, precision, scale))
3263                } else {
3264                    let value = array.value(index);
3265                    Ok(ScalarValue::Decimal256(Some(value), precision, scale))
3266                }
3267            }
3268            other => {
3269                unreachable!("Invalid type isn't decimal: {other:?}")
3270            }
3271        }
3272    }
3273
3274    fn list_to_array_of_size(arr: &dyn Array, size: usize) -> Result<ArrayRef> {
3275        let arrays = repeat_n(arr, size).collect::<Vec<_>>();
3276        let ret = match !arrays.is_empty() {
3277            true => arrow::compute::concat(arrays.as_slice())?,
3278            false => arr.slice(0, 0),
3279        };
3280        Ok(ret)
3281    }
3282
3283    /// Retrieve ScalarValue for each row in `array`
3284    ///
3285    /// Elements in `array` may be NULL, in which case the corresponding element in the returned vector is None.
3286    ///
3287    /// Example 1: Array (ScalarValue::Int32)
3288    /// ```
3289    /// use arrow::array::ListArray;
3290    /// use arrow::datatypes::{DataType, Int32Type};
3291    /// use datafusion_common::ScalarValue;
3292    ///
3293    /// // Equivalent to [[1,2,3], [4,5]]
3294    /// let list_arr = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
3295    ///     Some(vec![Some(1), Some(2), Some(3)]),
3296    ///     Some(vec![Some(4), Some(5)]),
3297    /// ]);
3298    ///
3299    /// // Convert the array into Scalar Values for each row
3300    /// let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&list_arr).unwrap();
3301    ///
3302    /// let expected = vec![
3303    ///     Some(vec![
3304    ///         ScalarValue::Int32(Some(1)),
3305    ///         ScalarValue::Int32(Some(2)),
3306    ///         ScalarValue::Int32(Some(3)),
3307    ///     ]),
3308    ///     Some(vec![
3309    ///         ScalarValue::Int32(Some(4)),
3310    ///         ScalarValue::Int32(Some(5)),
3311    ///     ]),
3312    /// ];
3313    ///
3314    /// assert_eq!(scalar_vec, expected);
3315    /// ```
3316    ///
3317    /// Example 2: Nested array (ScalarValue::List)
3318    /// ```
3319    /// use arrow::array::ListArray;
3320    /// use arrow::datatypes::{DataType, Int32Type};
3321    /// use datafusion_common::utils::SingleRowListArrayBuilder;
3322    /// use datafusion_common::ScalarValue;
3323    /// use std::sync::Arc;
3324    ///
3325    /// let list_arr = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
3326    ///     Some(vec![Some(1), Some(2), Some(3)]),
3327    ///     Some(vec![Some(4), Some(5)]),
3328    /// ]);
3329    ///
3330    /// // Wrap into another layer of list, we got nested array as [ [[1,2,3], [4,5]] ]
3331    /// let list_arr = SingleRowListArrayBuilder::new(Arc::new(list_arr)).build_list_array();
3332    ///
3333    /// // Convert the array into Scalar Values for each row, we got 1D arrays in this example
3334    /// let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&list_arr).unwrap();
3335    ///
3336    /// let l1 = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
3337    ///     Some(1),
3338    ///     Some(2),
3339    ///     Some(3),
3340    /// ])]);
3341    /// let l2 = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
3342    ///     Some(4),
3343    ///     Some(5),
3344    /// ])]);
3345    ///
3346    /// let expected = vec![Some(vec![
3347    ///     ScalarValue::List(Arc::new(l1)),
3348    ///     ScalarValue::List(Arc::new(l2)),
3349    /// ])];
3350    ///
3351    /// assert_eq!(scalar_vec, expected);
3352    /// ```
3353    ///
3354    /// Example 3: Nullable array
3355    /// ```
3356    /// use arrow::array::ListArray;
3357    /// use arrow::datatypes::{DataType, Int32Type};
3358    /// use datafusion_common::ScalarValue;
3359    ///
3360    /// let list_arr = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
3361    ///     Some(vec![Some(1), Some(2), Some(3)]),
3362    ///     None,
3363    ///     Some(vec![Some(4), Some(5)]),
3364    /// ]);
3365    ///
3366    /// // Convert the array into Scalar Values for each row
3367    /// let scalar_vec = ScalarValue::convert_array_to_scalar_vec(&list_arr).unwrap();
3368    ///
3369    /// let expected = vec![
3370    ///     Some(vec![
3371    ///         ScalarValue::Int32(Some(1)),
3372    ///         ScalarValue::Int32(Some(2)),
3373    ///         ScalarValue::Int32(Some(3)),
3374    ///     ]),
3375    ///     None,
3376    ///     Some(vec![
3377    ///         ScalarValue::Int32(Some(4)),
3378    ///         ScalarValue::Int32(Some(5)),
3379    ///     ]),
3380    /// ];
3381    ///
3382    /// assert_eq!(scalar_vec, expected);
3383    /// ```
3384    pub fn convert_array_to_scalar_vec(
3385        array: &dyn Array,
3386    ) -> Result<Vec<Option<Vec<Self>>>> {
3387        fn generic_collect<OffsetSize: OffsetSizeTrait>(
3388            array: &dyn Array,
3389        ) -> Result<Vec<Option<Vec<ScalarValue>>>> {
3390            array
3391                .as_list::<OffsetSize>()
3392                .iter()
3393                .map(|nested_array| {
3394                    nested_array
3395                        .map(|array| {
3396                            (0..array.len())
3397                                .map(|i| ScalarValue::try_from_array(&array, i))
3398                                .collect::<Result<Vec<_>>>()
3399                        })
3400                        .transpose()
3401                })
3402                .collect()
3403        }
3404
3405        match array.data_type() {
3406            DataType::List(_) => generic_collect::<i32>(array),
3407            DataType::LargeList(_) => generic_collect::<i64>(array),
3408            _ => _internal_err!(
3409                "ScalarValue::convert_array_to_scalar_vec input must be a List/LargeList type"
3410            ),
3411        }
3412    }
3413
3414    #[deprecated(
3415        since = "46.0.0",
3416        note = "This function is obsolete. Use `to_array` instead"
3417    )]
3418    pub fn raw_data(&self) -> Result<ArrayRef> {
3419        match self {
3420            ScalarValue::List(arr) => Ok(arr.to_owned()),
3421            _ => _internal_err!("ScalarValue is not a list"),
3422        }
3423    }
3424
3425    /// Converts a value in `array` at `index` into a ScalarValue
3426    pub fn try_from_array(array: &dyn Array, index: usize) -> Result<Self> {
3427        // handle NULL value
3428        if !array.is_valid(index) {
3429            return array.data_type().try_into();
3430        }
3431
3432        Ok(match array.data_type() {
3433            DataType::Null => ScalarValue::Null,
3434            DataType::Decimal32(precision, scale) => {
3435                ScalarValue::get_decimal_value_from_array(
3436                    array, index, *precision, *scale,
3437                )?
3438            }
3439            DataType::Decimal64(precision, scale) => {
3440                ScalarValue::get_decimal_value_from_array(
3441                    array, index, *precision, *scale,
3442                )?
3443            }
3444            DataType::Decimal128(precision, scale) => {
3445                ScalarValue::get_decimal_value_from_array(
3446                    array, index, *precision, *scale,
3447                )?
3448            }
3449            DataType::Decimal256(precision, scale) => {
3450                ScalarValue::get_decimal_value_from_array(
3451                    array, index, *precision, *scale,
3452                )?
3453            }
3454            DataType::Boolean => typed_cast!(array, index, as_boolean_array, Boolean)?,
3455            DataType::Float64 => typed_cast!(array, index, as_float64_array, Float64)?,
3456            DataType::Float32 => typed_cast!(array, index, as_float32_array, Float32)?,
3457            DataType::Float16 => typed_cast!(array, index, as_float16_array, Float16)?,
3458            DataType::UInt64 => typed_cast!(array, index, as_uint64_array, UInt64)?,
3459            DataType::UInt32 => typed_cast!(array, index, as_uint32_array, UInt32)?,
3460            DataType::UInt16 => typed_cast!(array, index, as_uint16_array, UInt16)?,
3461            DataType::UInt8 => typed_cast!(array, index, as_uint8_array, UInt8)?,
3462            DataType::Int64 => typed_cast!(array, index, as_int64_array, Int64)?,
3463            DataType::Int32 => typed_cast!(array, index, as_int32_array, Int32)?,
3464            DataType::Int16 => typed_cast!(array, index, as_int16_array, Int16)?,
3465            DataType::Int8 => typed_cast!(array, index, as_int8_array, Int8)?,
3466            DataType::Binary => typed_cast!(array, index, as_binary_array, Binary)?,
3467            DataType::LargeBinary => {
3468                typed_cast!(array, index, as_large_binary_array, LargeBinary)?
3469            }
3470            DataType::BinaryView => {
3471                typed_cast!(array, index, as_binary_view_array, BinaryView)?
3472            }
3473            DataType::Utf8 => typed_cast!(array, index, as_string_array, Utf8)?,
3474            DataType::LargeUtf8 => {
3475                typed_cast!(array, index, as_large_string_array, LargeUtf8)?
3476            }
3477            DataType::Utf8View => {
3478                typed_cast!(array, index, as_string_view_array, Utf8View)?
3479            }
3480            DataType::List(field) => {
3481                let list_array = array.as_list::<i32>();
3482                let nested_array = list_array.value(index);
3483                // Produces a single element `ListArray` with the value at `index`.
3484                SingleRowListArrayBuilder::new(nested_array)
3485                    .with_field(field)
3486                    .build_list_scalar()
3487            }
3488            DataType::LargeList(field) => {
3489                let list_array = as_large_list_array(array)?;
3490                let nested_array = list_array.value(index);
3491                // Produces a single element `LargeListArray` with the value at `index`.
3492                SingleRowListArrayBuilder::new(nested_array)
3493                    .with_field(field)
3494                    .build_large_list_scalar()
3495            }
3496            // TODO: There is no test for FixedSizeList now, add it later
3497            DataType::FixedSizeList(field, _) => {
3498                let list_array = as_fixed_size_list_array(array)?;
3499                let nested_array = list_array.value(index);
3500                // Produces a single element `FixedSizeListArray` with the value at `index`.
3501                let list_size = nested_array.len();
3502                SingleRowListArrayBuilder::new(nested_array)
3503                    .with_field(field)
3504                    .build_fixed_size_list_scalar(list_size)
3505            }
3506            DataType::Date32 => typed_cast!(array, index, as_date32_array, Date32)?,
3507            DataType::Date64 => typed_cast!(array, index, as_date64_array, Date64)?,
3508            DataType::Time32(TimeUnit::Second) => {
3509                typed_cast!(array, index, as_time32_second_array, Time32Second)?
3510            }
3511            DataType::Time32(TimeUnit::Millisecond) => {
3512                typed_cast!(array, index, as_time32_millisecond_array, Time32Millisecond)?
3513            }
3514            DataType::Time64(TimeUnit::Microsecond) => {
3515                typed_cast!(array, index, as_time64_microsecond_array, Time64Microsecond)?
3516            }
3517            DataType::Time64(TimeUnit::Nanosecond) => {
3518                typed_cast!(array, index, as_time64_nanosecond_array, Time64Nanosecond)?
3519            }
3520            DataType::Timestamp(TimeUnit::Second, tz_opt) => typed_cast_tz!(
3521                array,
3522                index,
3523                as_timestamp_second_array,
3524                TimestampSecond,
3525                tz_opt
3526            )?,
3527            DataType::Timestamp(TimeUnit::Millisecond, tz_opt) => typed_cast_tz!(
3528                array,
3529                index,
3530                as_timestamp_millisecond_array,
3531                TimestampMillisecond,
3532                tz_opt
3533            )?,
3534            DataType::Timestamp(TimeUnit::Microsecond, tz_opt) => typed_cast_tz!(
3535                array,
3536                index,
3537                as_timestamp_microsecond_array,
3538                TimestampMicrosecond,
3539                tz_opt
3540            )?,
3541            DataType::Timestamp(TimeUnit::Nanosecond, tz_opt) => typed_cast_tz!(
3542                array,
3543                index,
3544                as_timestamp_nanosecond_array,
3545                TimestampNanosecond,
3546                tz_opt
3547            )?,
3548            DataType::Dictionary(key_type, _) => {
3549                let (values_array, values_index) = match key_type.as_ref() {
3550                    DataType::Int8 => get_dict_value::<Int8Type>(array, index)?,
3551                    DataType::Int16 => get_dict_value::<Int16Type>(array, index)?,
3552                    DataType::Int32 => get_dict_value::<Int32Type>(array, index)?,
3553                    DataType::Int64 => get_dict_value::<Int64Type>(array, index)?,
3554                    DataType::UInt8 => get_dict_value::<UInt8Type>(array, index)?,
3555                    DataType::UInt16 => get_dict_value::<UInt16Type>(array, index)?,
3556                    DataType::UInt32 => get_dict_value::<UInt32Type>(array, index)?,
3557                    DataType::UInt64 => get_dict_value::<UInt64Type>(array, index)?,
3558                    _ => unreachable!("Invalid dictionary keys type: {}", key_type),
3559                };
3560                // look up the index in the values dictionary
3561                let value = match values_index {
3562                    Some(values_index) => {
3563                        ScalarValue::try_from_array(values_array, values_index)
3564                    }
3565                    // else entry was null, so return null
3566                    None => values_array.data_type().try_into(),
3567                }?;
3568
3569                Self::Dictionary(key_type.clone(), Box::new(value))
3570            }
3571            DataType::Struct(_) => {
3572                let a = array.slice(index, 1);
3573                Self::Struct(Arc::new(a.as_struct().to_owned()))
3574            }
3575            DataType::FixedSizeBinary(_) => {
3576                let array = as_fixed_size_binary_array(array)?;
3577                let size = match array.data_type() {
3578                    DataType::FixedSizeBinary(size) => *size,
3579                    _ => unreachable!(),
3580                };
3581                ScalarValue::FixedSizeBinary(
3582                    size,
3583                    match array.is_null(index) {
3584                        true => None,
3585                        false => Some(array.value(index).into()),
3586                    },
3587                )
3588            }
3589            DataType::Interval(IntervalUnit::DayTime) => {
3590                typed_cast!(array, index, as_interval_dt_array, IntervalDayTime)?
3591            }
3592            DataType::Interval(IntervalUnit::YearMonth) => {
3593                typed_cast!(array, index, as_interval_ym_array, IntervalYearMonth)?
3594            }
3595            DataType::Interval(IntervalUnit::MonthDayNano) => {
3596                typed_cast!(array, index, as_interval_mdn_array, IntervalMonthDayNano)?
3597            }
3598
3599            DataType::Duration(TimeUnit::Second) => {
3600                typed_cast!(array, index, as_duration_second_array, DurationSecond)?
3601            }
3602            DataType::Duration(TimeUnit::Millisecond) => typed_cast!(
3603                array,
3604                index,
3605                as_duration_millisecond_array,
3606                DurationMillisecond
3607            )?,
3608            DataType::Duration(TimeUnit::Microsecond) => typed_cast!(
3609                array,
3610                index,
3611                as_duration_microsecond_array,
3612                DurationMicrosecond
3613            )?,
3614            DataType::Duration(TimeUnit::Nanosecond) => typed_cast!(
3615                array,
3616                index,
3617                as_duration_nanosecond_array,
3618                DurationNanosecond
3619            )?,
3620            DataType::Map(_, _) => {
3621                let a = array.slice(index, 1);
3622                Self::Map(Arc::new(a.as_map().to_owned()))
3623            }
3624            DataType::Union(fields, mode) => {
3625                let array = as_union_array(array)?;
3626                let ti = array.type_id(index);
3627                let index = array.value_offset(index);
3628                let value = ScalarValue::try_from_array(array.child(ti), index)?;
3629                ScalarValue::Union(Some((ti, Box::new(value))), fields.clone(), *mode)
3630            }
3631            other => {
3632                return _not_impl_err!(
3633                    "Can't create a scalar from array of type \"{other:?}\""
3634                );
3635            }
3636        })
3637    }
3638
3639    /// Try to parse `value` into a ScalarValue of type `target_type`
3640    pub fn try_from_string(value: String, target_type: &DataType) -> Result<Self> {
3641        ScalarValue::from(value).cast_to(target_type)
3642    }
3643
3644    /// Returns the Some(`&str`) representation of `ScalarValue` of logical string type
3645    ///
3646    /// Returns `None` if this `ScalarValue` is not a logical string type or the
3647    /// `ScalarValue` represents the `NULL` value.
3648    ///
3649    /// Note you can use [`Option::flatten`] to check for non null logical
3650    /// strings.
3651    ///
3652    /// For example, [`ScalarValue::Utf8`], [`ScalarValue::LargeUtf8`], and
3653    /// [`ScalarValue::Dictionary`] with a logical string value and store
3654    /// strings and can be accessed as `&str` using this method.
3655    ///
3656    /// # Example: logical strings
3657    /// ```
3658    /// # use datafusion_common::ScalarValue;
3659    /// /// non strings return None
3660    /// let scalar = ScalarValue::from(42);
3661    /// assert_eq!(scalar.try_as_str(), None);
3662    /// // Non null logical string returns Some(Some(&str))
3663    /// let scalar = ScalarValue::from("hello");
3664    /// assert_eq!(scalar.try_as_str(), Some(Some("hello")));
3665    /// // Null logical string returns Some(None)
3666    /// let scalar = ScalarValue::Utf8(None);
3667    /// assert_eq!(scalar.try_as_str(), Some(None));
3668    /// ```
3669    ///
3670    /// # Example: use [`Option::flatten`] to check for non-null logical strings
3671    /// ```
3672    /// # use datafusion_common::ScalarValue;
3673    /// // Non null logical string returns Some(Some(&str))
3674    /// let scalar = ScalarValue::from("hello");
3675    /// assert_eq!(scalar.try_as_str().flatten(), Some("hello"));
3676    /// ```
3677    pub fn try_as_str(&self) -> Option<Option<&str>> {
3678        let v = match self {
3679            ScalarValue::Utf8(v) => v,
3680            ScalarValue::LargeUtf8(v) => v,
3681            ScalarValue::Utf8View(v) => v,
3682            ScalarValue::Dictionary(_, v) => return v.try_as_str(),
3683            _ => return None,
3684        };
3685        Some(v.as_ref().map(|v| v.as_str()))
3686    }
3687
3688    /// Try to cast this value to a ScalarValue of type `data_type`
3689    pub fn cast_to(&self, target_type: &DataType) -> Result<Self> {
3690        self.cast_to_with_options(target_type, &DEFAULT_CAST_OPTIONS)
3691    }
3692
3693    /// Try to cast this value to a ScalarValue of type `data_type` with [`CastOptions`]
3694    pub fn cast_to_with_options(
3695        &self,
3696        target_type: &DataType,
3697        cast_options: &CastOptions<'static>,
3698    ) -> Result<Self> {
3699        let source_type = self.data_type();
3700        if let Some(multiplier) = date_to_timestamp_multiplier(&source_type, target_type)
3701            && let Some(value) = self.date_scalar_value_as_i64()
3702        {
3703            ensure_timestamp_in_bounds(value, multiplier, &source_type, target_type)?;
3704        }
3705
3706        let scalar_array = self.to_array()?;
3707        let cast_arr = cast_with_options(&scalar_array, target_type, cast_options)?;
3708        ScalarValue::try_from_array(&cast_arr, 0)
3709    }
3710
3711    fn date_scalar_value_as_i64(&self) -> Option<i64> {
3712        match self {
3713            ScalarValue::Date32(Some(value)) => Some(i64::from(*value)),
3714            ScalarValue::Date64(Some(value)) => Some(*value),
3715            _ => None,
3716        }
3717    }
3718
3719    fn eq_array_decimal32(
3720        array: &ArrayRef,
3721        index: usize,
3722        value: Option<&i32>,
3723        precision: u8,
3724        scale: i8,
3725    ) -> Result<bool> {
3726        let array = as_decimal32_array(array)?;
3727        if array.precision() != precision || array.scale() != scale {
3728            return Ok(false);
3729        }
3730        let is_null = array.is_null(index);
3731        if let Some(v) = value {
3732            Ok(!array.is_null(index) && array.value(index) == *v)
3733        } else {
3734            Ok(is_null)
3735        }
3736    }
3737
3738    fn eq_array_decimal64(
3739        array: &ArrayRef,
3740        index: usize,
3741        value: Option<&i64>,
3742        precision: u8,
3743        scale: i8,
3744    ) -> Result<bool> {
3745        let array = as_decimal64_array(array)?;
3746        if array.precision() != precision || array.scale() != scale {
3747            return Ok(false);
3748        }
3749        let is_null = array.is_null(index);
3750        if let Some(v) = value {
3751            Ok(!array.is_null(index) && array.value(index) == *v)
3752        } else {
3753            Ok(is_null)
3754        }
3755    }
3756
3757    fn eq_array_decimal(
3758        array: &ArrayRef,
3759        index: usize,
3760        value: Option<&i128>,
3761        precision: u8,
3762        scale: i8,
3763    ) -> Result<bool> {
3764        let array = as_decimal128_array(array)?;
3765        if array.precision() != precision || array.scale() != scale {
3766            return Ok(false);
3767        }
3768        let is_null = array.is_null(index);
3769        if let Some(v) = value {
3770            Ok(!array.is_null(index) && array.value(index) == *v)
3771        } else {
3772            Ok(is_null)
3773        }
3774    }
3775
3776    fn eq_array_decimal256(
3777        array: &ArrayRef,
3778        index: usize,
3779        value: Option<&i256>,
3780        precision: u8,
3781        scale: i8,
3782    ) -> Result<bool> {
3783        let array = as_decimal256_array(array)?;
3784        if array.precision() != precision || array.scale() != scale {
3785            return Ok(false);
3786        }
3787        let is_null = array.is_null(index);
3788        if let Some(v) = value {
3789            Ok(!array.is_null(index) && array.value(index) == *v)
3790        } else {
3791            Ok(is_null)
3792        }
3793    }
3794
3795    /// Compares a single row of array @ index for equality with self,
3796    /// in an optimized fashion.
3797    ///
3798    /// This method implements an optimized version of:
3799    ///
3800    /// ```text
3801    ///     let arr_scalar = Self::try_from_array(array, index).unwrap();
3802    ///     arr_scalar.eq(self)
3803    /// ```
3804    ///
3805    /// *Performance note*: the arrow compute kernels should be
3806    /// preferred over this function if at all possible as they can be
3807    /// vectorized and are generally much faster.
3808    ///
3809    /// This function has a few narrow use cases such as hash table key
3810    /// comparisons where comparing a single row at a time is necessary.
3811    ///
3812    /// # Errors
3813    ///
3814    /// Errors if
3815    /// - it fails to downcast `array` to the data type of `self`
3816    /// - `self` is a `Struct`
3817    ///
3818    /// # Panics
3819    ///
3820    /// Panics if `self` is a dictionary with invalid key type
3821    #[inline]
3822    pub fn eq_array(&self, array: &ArrayRef, index: usize) -> Result<bool> {
3823        Ok(match self {
3824            ScalarValue::Decimal32(v, precision, scale) => {
3825                ScalarValue::eq_array_decimal32(
3826                    array,
3827                    index,
3828                    v.as_ref(),
3829                    *precision,
3830                    *scale,
3831                )?
3832            }
3833            ScalarValue::Decimal64(v, precision, scale) => {
3834                ScalarValue::eq_array_decimal64(
3835                    array,
3836                    index,
3837                    v.as_ref(),
3838                    *precision,
3839                    *scale,
3840                )?
3841            }
3842            ScalarValue::Decimal128(v, precision, scale) => {
3843                ScalarValue::eq_array_decimal(
3844                    array,
3845                    index,
3846                    v.as_ref(),
3847                    *precision,
3848                    *scale,
3849                )?
3850            }
3851            ScalarValue::Decimal256(v, precision, scale) => {
3852                ScalarValue::eq_array_decimal256(
3853                    array,
3854                    index,
3855                    v.as_ref(),
3856                    *precision,
3857                    *scale,
3858                )?
3859            }
3860            ScalarValue::Boolean(val) => {
3861                eq_array_primitive!(array, index, as_boolean_array, val)?
3862            }
3863            ScalarValue::Float16(val) => {
3864                eq_array_primitive!(array, index, as_float16_array, val)?
3865            }
3866            ScalarValue::Float32(val) => {
3867                eq_array_primitive!(array, index, as_float32_array, val)?
3868            }
3869            ScalarValue::Float64(val) => {
3870                eq_array_primitive!(array, index, as_float64_array, val)?
3871            }
3872            ScalarValue::Int8(val) => {
3873                eq_array_primitive!(array, index, as_int8_array, val)?
3874            }
3875            ScalarValue::Int16(val) => {
3876                eq_array_primitive!(array, index, as_int16_array, val)?
3877            }
3878            ScalarValue::Int32(val) => {
3879                eq_array_primitive!(array, index, as_int32_array, val)?
3880            }
3881            ScalarValue::Int64(val) => {
3882                eq_array_primitive!(array, index, as_int64_array, val)?
3883            }
3884            ScalarValue::UInt8(val) => {
3885                eq_array_primitive!(array, index, as_uint8_array, val)?
3886            }
3887            ScalarValue::UInt16(val) => {
3888                eq_array_primitive!(array, index, as_uint16_array, val)?
3889            }
3890            ScalarValue::UInt32(val) => {
3891                eq_array_primitive!(array, index, as_uint32_array, val)?
3892            }
3893            ScalarValue::UInt64(val) => {
3894                eq_array_primitive!(array, index, as_uint64_array, val)?
3895            }
3896            ScalarValue::Utf8(val) => {
3897                eq_array_primitive!(array, index, as_string_array, val)?
3898            }
3899            ScalarValue::Utf8View(val) => {
3900                eq_array_primitive!(array, index, as_string_view_array, val)?
3901            }
3902            ScalarValue::LargeUtf8(val) => {
3903                eq_array_primitive!(array, index, as_large_string_array, val)?
3904            }
3905            ScalarValue::Binary(val) => {
3906                eq_array_primitive!(array, index, as_binary_array, val)?
3907            }
3908            ScalarValue::BinaryView(val) => {
3909                eq_array_primitive!(array, index, as_binary_view_array, val)?
3910            }
3911            ScalarValue::FixedSizeBinary(_, val) => {
3912                eq_array_primitive!(array, index, as_fixed_size_binary_array, val)?
3913            }
3914            ScalarValue::LargeBinary(val) => {
3915                eq_array_primitive!(array, index, as_large_binary_array, val)?
3916            }
3917            ScalarValue::List(arr) => {
3918                Self::eq_array_list(&(arr.to_owned() as ArrayRef), array, index)
3919            }
3920            ScalarValue::LargeList(arr) => {
3921                Self::eq_array_list(&(arr.to_owned() as ArrayRef), array, index)
3922            }
3923            ScalarValue::FixedSizeList(arr) => {
3924                Self::eq_array_list(&(arr.to_owned() as ArrayRef), array, index)
3925            }
3926            ScalarValue::Struct(arr) => {
3927                Self::eq_array_list(&(arr.to_owned() as ArrayRef), array, index)
3928            }
3929            ScalarValue::Map(arr) => {
3930                Self::eq_array_list(&(arr.to_owned() as ArrayRef), array, index)
3931            }
3932            ScalarValue::Date32(val) => {
3933                eq_array_primitive!(array, index, as_date32_array, val)?
3934            }
3935            ScalarValue::Date64(val) => {
3936                eq_array_primitive!(array, index, as_date64_array, val)?
3937            }
3938            ScalarValue::Time32Second(val) => {
3939                eq_array_primitive!(array, index, as_time32_second_array, val)?
3940            }
3941            ScalarValue::Time32Millisecond(val) => {
3942                eq_array_primitive!(array, index, as_time32_millisecond_array, val)?
3943            }
3944            ScalarValue::Time64Microsecond(val) => {
3945                eq_array_primitive!(array, index, as_time64_microsecond_array, val)?
3946            }
3947            ScalarValue::Time64Nanosecond(val) => {
3948                eq_array_primitive!(array, index, as_time64_nanosecond_array, val)?
3949            }
3950            ScalarValue::TimestampSecond(val, _) => {
3951                eq_array_primitive!(array, index, as_timestamp_second_array, val)?
3952            }
3953            ScalarValue::TimestampMillisecond(val, _) => {
3954                eq_array_primitive!(array, index, as_timestamp_millisecond_array, val)?
3955            }
3956            ScalarValue::TimestampMicrosecond(val, _) => {
3957                eq_array_primitive!(array, index, as_timestamp_microsecond_array, val)?
3958            }
3959            ScalarValue::TimestampNanosecond(val, _) => {
3960                eq_array_primitive!(array, index, as_timestamp_nanosecond_array, val)?
3961            }
3962            ScalarValue::IntervalYearMonth(val) => {
3963                eq_array_primitive!(array, index, as_interval_ym_array, val)?
3964            }
3965            ScalarValue::IntervalDayTime(val) => {
3966                eq_array_primitive!(array, index, as_interval_dt_array, val)?
3967            }
3968            ScalarValue::IntervalMonthDayNano(val) => {
3969                eq_array_primitive!(array, index, as_interval_mdn_array, val)?
3970            }
3971            ScalarValue::DurationSecond(val) => {
3972                eq_array_primitive!(array, index, as_duration_second_array, val)?
3973            }
3974            ScalarValue::DurationMillisecond(val) => {
3975                eq_array_primitive!(array, index, as_duration_millisecond_array, val)?
3976            }
3977            ScalarValue::DurationMicrosecond(val) => {
3978                eq_array_primitive!(array, index, as_duration_microsecond_array, val)?
3979            }
3980            ScalarValue::DurationNanosecond(val) => {
3981                eq_array_primitive!(array, index, as_duration_nanosecond_array, val)?
3982            }
3983            ScalarValue::Union(value, _, _) => {
3984                let array = as_union_array(array)?;
3985                let ti = array.type_id(index);
3986                let index = array.value_offset(index);
3987                if let Some((ti_v, value)) = value {
3988                    ti_v == &ti && value.eq_array(array.child(ti), index)?
3989                } else {
3990                    array.child(ti).is_null(index)
3991                }
3992            }
3993            ScalarValue::Dictionary(key_type, v) => {
3994                let (values_array, values_index) = match key_type.as_ref() {
3995                    DataType::Int8 => get_dict_value::<Int8Type>(array, index)?,
3996                    DataType::Int16 => get_dict_value::<Int16Type>(array, index)?,
3997                    DataType::Int32 => get_dict_value::<Int32Type>(array, index)?,
3998                    DataType::Int64 => get_dict_value::<Int64Type>(array, index)?,
3999                    DataType::UInt8 => get_dict_value::<UInt8Type>(array, index)?,
4000                    DataType::UInt16 => get_dict_value::<UInt16Type>(array, index)?,
4001                    DataType::UInt32 => get_dict_value::<UInt32Type>(array, index)?,
4002                    DataType::UInt64 => get_dict_value::<UInt64Type>(array, index)?,
4003                    _ => unreachable!("Invalid dictionary keys type: {}", key_type),
4004                };
4005                // was the value in the array non null?
4006                match values_index {
4007                    Some(values_index) => v.eq_array(values_array, values_index)?,
4008                    None => v.is_null(),
4009                }
4010            }
4011            ScalarValue::Null => array.is_null(index),
4012        })
4013    }
4014
4015    fn eq_array_list(arr1: &ArrayRef, arr2: &ArrayRef, index: usize) -> bool {
4016        let right = arr2.slice(index, 1);
4017        arr1 == &right
4018    }
4019
4020    /// Compare `self` with `other` and return an `Ordering`.
4021    ///
4022    /// This is the same as [`PartialOrd`] except that it returns
4023    /// `Err` if the values cannot be compared, e.g., they have incompatible data types.
4024    pub fn try_cmp(&self, other: &Self) -> Result<Ordering> {
4025        self.partial_cmp(other).ok_or_else(|| {
4026            _internal_datafusion_err!("Uncomparable values: {self:?}, {other:?}")
4027        })
4028    }
4029
4030    /// Estimate size if bytes including `Self`. For values with internal containers such as `String`
4031    /// includes the allocated size (`capacity`) rather than the current length (`len`)
4032    pub fn size(&self) -> usize {
4033        size_of_val(self)
4034            + match self {
4035                ScalarValue::Null
4036                | ScalarValue::Boolean(_)
4037                | ScalarValue::Float16(_)
4038                | ScalarValue::Float32(_)
4039                | ScalarValue::Float64(_)
4040                | ScalarValue::Decimal32(_, _, _)
4041                | ScalarValue::Decimal64(_, _, _)
4042                | ScalarValue::Decimal128(_, _, _)
4043                | ScalarValue::Decimal256(_, _, _)
4044                | ScalarValue::Int8(_)
4045                | ScalarValue::Int16(_)
4046                | ScalarValue::Int32(_)
4047                | ScalarValue::Int64(_)
4048                | ScalarValue::UInt8(_)
4049                | ScalarValue::UInt16(_)
4050                | ScalarValue::UInt32(_)
4051                | ScalarValue::UInt64(_)
4052                | ScalarValue::Date32(_)
4053                | ScalarValue::Date64(_)
4054                | ScalarValue::Time32Second(_)
4055                | ScalarValue::Time32Millisecond(_)
4056                | ScalarValue::Time64Microsecond(_)
4057                | ScalarValue::Time64Nanosecond(_)
4058                | ScalarValue::IntervalYearMonth(_)
4059                | ScalarValue::IntervalDayTime(_)
4060                | ScalarValue::IntervalMonthDayNano(_)
4061                | ScalarValue::DurationSecond(_)
4062                | ScalarValue::DurationMillisecond(_)
4063                | ScalarValue::DurationMicrosecond(_)
4064                | ScalarValue::DurationNanosecond(_) => 0,
4065                ScalarValue::Utf8(s)
4066                | ScalarValue::LargeUtf8(s)
4067                | ScalarValue::Utf8View(s) => {
4068                    s.as_ref().map(|s| s.capacity()).unwrap_or_default()
4069                }
4070                ScalarValue::TimestampSecond(_, s)
4071                | ScalarValue::TimestampMillisecond(_, s)
4072                | ScalarValue::TimestampMicrosecond(_, s)
4073                | ScalarValue::TimestampNanosecond(_, s) => {
4074                    s.as_ref().map(|s| s.len()).unwrap_or_default()
4075                }
4076                ScalarValue::Binary(b)
4077                | ScalarValue::FixedSizeBinary(_, b)
4078                | ScalarValue::LargeBinary(b)
4079                | ScalarValue::BinaryView(b) => {
4080                    b.as_ref().map(|b| b.capacity()).unwrap_or_default()
4081                }
4082                ScalarValue::List(arr) => arr.get_array_memory_size(),
4083                ScalarValue::LargeList(arr) => arr.get_array_memory_size(),
4084                ScalarValue::FixedSizeList(arr) => arr.get_array_memory_size(),
4085                ScalarValue::Struct(arr) => arr.get_array_memory_size(),
4086                ScalarValue::Map(arr) => arr.get_array_memory_size(),
4087                ScalarValue::Union(vals, fields, _mode) => {
4088                    vals.as_ref()
4089                        .map(|(_id, sv)| sv.size() - size_of_val(sv))
4090                        .unwrap_or_default()
4091                        // `fields` is boxed, so it is NOT already included in `self`
4092                        + size_of_val(fields)
4093                        + (size_of::<Field>() * fields.len())
4094                        + fields.iter().map(|(_idx, field)| field.size() - size_of_val(field)).sum::<usize>()
4095                }
4096                ScalarValue::Dictionary(dt, sv) => {
4097                    // `dt` and `sv` are boxed, so they are NOT already included in `self`
4098                    dt.size() + sv.size()
4099                }
4100            }
4101    }
4102
4103    /// Estimates [size](Self::size) of [`Vec`] in bytes.
4104    ///
4105    /// Includes the size of the [`Vec`] container itself.
4106    pub fn size_of_vec(vec: &Vec<Self>) -> usize {
4107        size_of_val(vec)
4108            + (size_of::<ScalarValue>() * vec.capacity())
4109            + vec
4110                .iter()
4111                .map(|sv| sv.size() - size_of_val(sv))
4112                .sum::<usize>()
4113    }
4114
4115    /// Estimates [size](Self::size) of [`VecDeque`] in bytes.
4116    ///
4117    /// Includes the size of the [`VecDeque`] container itself.
4118    pub fn size_of_vec_deque(vec_deque: &VecDeque<Self>) -> usize {
4119        size_of_val(vec_deque)
4120            + (size_of::<ScalarValue>() * vec_deque.capacity())
4121            + vec_deque
4122                .iter()
4123                .map(|sv| sv.size() - size_of_val(sv))
4124                .sum::<usize>()
4125    }
4126
4127    /// Estimates [size](Self::size) of [`HashSet`] in bytes.
4128    ///
4129    /// Includes the size of the [`HashSet`] container itself.
4130    pub fn size_of_hashset<S>(set: &HashSet<Self, S>) -> usize {
4131        size_of_val(set)
4132            + (size_of::<ScalarValue>() * set.capacity())
4133            + set
4134                .iter()
4135                .map(|sv| sv.size() - size_of_val(sv))
4136                .sum::<usize>()
4137    }
4138
4139    /// Compacts the allocation referenced by `self` to the minimum, copying the data if
4140    /// necessary.
4141    ///
4142    /// This can be relevant when `self` is a list or contains a list as a nested value, as
4143    /// a single list holds an Arc to its entire original array buffer.
4144    pub fn compact(&mut self) {
4145        match self {
4146            ScalarValue::Null
4147            | ScalarValue::Boolean(_)
4148            | ScalarValue::Float16(_)
4149            | ScalarValue::Float32(_)
4150            | ScalarValue::Float64(_)
4151            | ScalarValue::Decimal32(_, _, _)
4152            | ScalarValue::Decimal64(_, _, _)
4153            | ScalarValue::Decimal128(_, _, _)
4154            | ScalarValue::Decimal256(_, _, _)
4155            | ScalarValue::Int8(_)
4156            | ScalarValue::Int16(_)
4157            | ScalarValue::Int32(_)
4158            | ScalarValue::Int64(_)
4159            | ScalarValue::UInt8(_)
4160            | ScalarValue::UInt16(_)
4161            | ScalarValue::UInt32(_)
4162            | ScalarValue::UInt64(_)
4163            | ScalarValue::Date32(_)
4164            | ScalarValue::Date64(_)
4165            | ScalarValue::Time32Second(_)
4166            | ScalarValue::Time32Millisecond(_)
4167            | ScalarValue::Time64Microsecond(_)
4168            | ScalarValue::Time64Nanosecond(_)
4169            | ScalarValue::IntervalYearMonth(_)
4170            | ScalarValue::IntervalDayTime(_)
4171            | ScalarValue::IntervalMonthDayNano(_)
4172            | ScalarValue::DurationSecond(_)
4173            | ScalarValue::DurationMillisecond(_)
4174            | ScalarValue::DurationMicrosecond(_)
4175            | ScalarValue::DurationNanosecond(_)
4176            | ScalarValue::Utf8(_)
4177            | ScalarValue::LargeUtf8(_)
4178            | ScalarValue::Utf8View(_)
4179            | ScalarValue::TimestampSecond(_, _)
4180            | ScalarValue::TimestampMillisecond(_, _)
4181            | ScalarValue::TimestampMicrosecond(_, _)
4182            | ScalarValue::TimestampNanosecond(_, _)
4183            | ScalarValue::Binary(_)
4184            | ScalarValue::FixedSizeBinary(_, _)
4185            | ScalarValue::LargeBinary(_)
4186            | ScalarValue::BinaryView(_) => (),
4187            ScalarValue::FixedSizeList(arr) => {
4188                let array = copy_array_data(&arr.to_data());
4189                *Arc::make_mut(arr) = FixedSizeListArray::from(array);
4190            }
4191            ScalarValue::List(arr) => {
4192                let array = copy_array_data(&arr.to_data());
4193                *Arc::make_mut(arr) = ListArray::from(array);
4194            }
4195            ScalarValue::LargeList(arr) => {
4196                let array = copy_array_data(&arr.to_data());
4197                *Arc::make_mut(arr) = LargeListArray::from(array)
4198            }
4199            ScalarValue::Struct(arr) => {
4200                let array = copy_array_data(&arr.to_data());
4201                *Arc::make_mut(arr) = StructArray::from(array);
4202            }
4203            ScalarValue::Map(arr) => {
4204                let array = copy_array_data(&arr.to_data());
4205                *Arc::make_mut(arr) = MapArray::from(array);
4206            }
4207            ScalarValue::Union(val, _, _) => {
4208                if let Some((_, value)) = val.as_mut() {
4209                    value.compact();
4210                }
4211            }
4212            ScalarValue::Dictionary(_, value) => {
4213                value.compact();
4214            }
4215        }
4216    }
4217
4218    /// Compacts ([ScalarValue::compact]) the current [ScalarValue] and returns it.
4219    pub fn compacted(mut self) -> Self {
4220        self.compact();
4221        self
4222    }
4223
4224    /// Returns the minimum value for the given numeric `DataType`.
4225    ///
4226    /// This function returns the smallest representable value for numeric
4227    /// and temporal data types. For non-numeric types, it returns `None`.
4228    ///
4229    /// # Supported Types
4230    ///
4231    /// - **Integer types**: `i8::MIN`, `i16::MIN`, etc.
4232    /// - **Unsigned types**: Always 0 (`u8::MIN`, `u16::MIN`, etc.)
4233    /// - **Float types**: Negative infinity (IEEE 754)
4234    /// - **Decimal types**: Smallest value based on precision
4235    /// - **Temporal types**: Minimum timestamp/date values
4236    /// - **Time types**: 0 (midnight)
4237    /// - **Duration types**: `i64::MIN`
4238    pub fn min(datatype: &DataType) -> Option<ScalarValue> {
4239        match datatype {
4240            DataType::Int8 => Some(ScalarValue::Int8(Some(i8::MIN))),
4241            DataType::Int16 => Some(ScalarValue::Int16(Some(i16::MIN))),
4242            DataType::Int32 => Some(ScalarValue::Int32(Some(i32::MIN))),
4243            DataType::Int64 => Some(ScalarValue::Int64(Some(i64::MIN))),
4244            DataType::UInt8 => Some(ScalarValue::UInt8(Some(u8::MIN))),
4245            DataType::UInt16 => Some(ScalarValue::UInt16(Some(u16::MIN))),
4246            DataType::UInt32 => Some(ScalarValue::UInt32(Some(u32::MIN))),
4247            DataType::UInt64 => Some(ScalarValue::UInt64(Some(u64::MIN))),
4248            DataType::Float16 => Some(ScalarValue::Float16(Some(f16::NEG_INFINITY))),
4249            DataType::Float32 => Some(ScalarValue::Float32(Some(f32::NEG_INFINITY))),
4250            DataType::Float64 => Some(ScalarValue::Float64(Some(f64::NEG_INFINITY))),
4251            DataType::Decimal128(precision, scale) => {
4252                // For decimal, min is -10^(precision-scale) + 10^(-scale)
4253                // But for simplicity, we use the minimum i128 value that fits the precision
4254                let max_digits = 10_i128.pow(*precision as u32) - 1;
4255                Some(ScalarValue::Decimal128(
4256                    Some(-max_digits),
4257                    *precision,
4258                    *scale,
4259                ))
4260            }
4261            DataType::Decimal256(precision, scale) => {
4262                // Similar to Decimal128 but with i256
4263                // For now, use a large negative value
4264                let max_digits = i256::from_i128(10_i128)
4265                    .checked_pow(*precision as u32)
4266                    .and_then(|v| v.checked_sub(i256::from_i128(1)))
4267                    .unwrap_or(i256::MAX);
4268                Some(ScalarValue::Decimal256(
4269                    Some(max_digits.neg_wrapping()),
4270                    *precision,
4271                    *scale,
4272                ))
4273            }
4274            DataType::Date32 => Some(ScalarValue::Date32(Some(i32::MIN))),
4275            DataType::Date64 => Some(ScalarValue::Date64(Some(i64::MIN))),
4276            DataType::Time32(TimeUnit::Second) => {
4277                Some(ScalarValue::Time32Second(Some(0)))
4278            }
4279            DataType::Time32(TimeUnit::Millisecond) => {
4280                Some(ScalarValue::Time32Millisecond(Some(0)))
4281            }
4282            DataType::Time64(TimeUnit::Microsecond) => {
4283                Some(ScalarValue::Time64Microsecond(Some(0)))
4284            }
4285            DataType::Time64(TimeUnit::Nanosecond) => {
4286                Some(ScalarValue::Time64Nanosecond(Some(0)))
4287            }
4288            DataType::Timestamp(unit, tz) => match unit {
4289                TimeUnit::Second => {
4290                    Some(ScalarValue::TimestampSecond(Some(i64::MIN), tz.clone()))
4291                }
4292                TimeUnit::Millisecond => Some(ScalarValue::TimestampMillisecond(
4293                    Some(i64::MIN),
4294                    tz.clone(),
4295                )),
4296                TimeUnit::Microsecond => Some(ScalarValue::TimestampMicrosecond(
4297                    Some(i64::MIN),
4298                    tz.clone(),
4299                )),
4300                TimeUnit::Nanosecond => {
4301                    Some(ScalarValue::TimestampNanosecond(Some(i64::MIN), tz.clone()))
4302                }
4303            },
4304            DataType::Duration(unit) => match unit {
4305                TimeUnit::Second => Some(ScalarValue::DurationSecond(Some(i64::MIN))),
4306                TimeUnit::Millisecond => {
4307                    Some(ScalarValue::DurationMillisecond(Some(i64::MIN)))
4308                }
4309                TimeUnit::Microsecond => {
4310                    Some(ScalarValue::DurationMicrosecond(Some(i64::MIN)))
4311                }
4312                TimeUnit::Nanosecond => {
4313                    Some(ScalarValue::DurationNanosecond(Some(i64::MIN)))
4314                }
4315            },
4316            _ => None,
4317        }
4318    }
4319
4320    /// Returns the maximum value for the given numeric `DataType`.
4321    ///
4322    /// This function returns the largest representable value for numeric
4323    /// and temporal data types. For non-numeric types, it returns `None`.
4324    ///
4325    /// # Supported Types
4326    ///
4327    /// - **Integer types**: `i8::MAX`, `i16::MAX`, etc.
4328    /// - **Unsigned types**: `u8::MAX`, `u16::MAX`, etc.
4329    /// - **Float types**: Positive infinity (IEEE 754)
4330    /// - **Decimal types**: Largest value based on precision
4331    /// - **Temporal types**: Maximum timestamp/date values
4332    /// - **Time types**: Maximum time in the day (1 day - 1 unit)
4333    /// - **Duration types**: `i64::MAX`
4334    pub fn max(datatype: &DataType) -> Option<ScalarValue> {
4335        match datatype {
4336            DataType::Int8 => Some(ScalarValue::Int8(Some(i8::MAX))),
4337            DataType::Int16 => Some(ScalarValue::Int16(Some(i16::MAX))),
4338            DataType::Int32 => Some(ScalarValue::Int32(Some(i32::MAX))),
4339            DataType::Int64 => Some(ScalarValue::Int64(Some(i64::MAX))),
4340            DataType::UInt8 => Some(ScalarValue::UInt8(Some(u8::MAX))),
4341            DataType::UInt16 => Some(ScalarValue::UInt16(Some(u16::MAX))),
4342            DataType::UInt32 => Some(ScalarValue::UInt32(Some(u32::MAX))),
4343            DataType::UInt64 => Some(ScalarValue::UInt64(Some(u64::MAX))),
4344            DataType::Float16 => Some(ScalarValue::Float16(Some(f16::INFINITY))),
4345            DataType::Float32 => Some(ScalarValue::Float32(Some(f32::INFINITY))),
4346            DataType::Float64 => Some(ScalarValue::Float64(Some(f64::INFINITY))),
4347            DataType::Decimal128(precision, scale) => {
4348                // For decimal, max is 10^(precision-scale) - 10^(-scale)
4349                // But for simplicity, we use the maximum i128 value that fits the precision
4350                let max_digits = 10_i128.pow(*precision as u32) - 1;
4351                Some(ScalarValue::Decimal128(
4352                    Some(max_digits),
4353                    *precision,
4354                    *scale,
4355                ))
4356            }
4357            DataType::Decimal256(precision, scale) => {
4358                // Similar to Decimal128 but with i256
4359                let max_digits = i256::from_i128(10_i128)
4360                    .checked_pow(*precision as u32)
4361                    .and_then(|v| v.checked_sub(i256::from_i128(1)))
4362                    .unwrap_or(i256::MAX);
4363                Some(ScalarValue::Decimal256(
4364                    Some(max_digits),
4365                    *precision,
4366                    *scale,
4367                ))
4368            }
4369            DataType::Date32 => Some(ScalarValue::Date32(Some(i32::MAX))),
4370            DataType::Date64 => Some(ScalarValue::Date64(Some(i64::MAX))),
4371            DataType::Time32(TimeUnit::Second) => {
4372                // 86399 seconds = 23:59:59
4373                Some(ScalarValue::Time32Second(Some(86_399)))
4374            }
4375            DataType::Time32(TimeUnit::Millisecond) => {
4376                // 86_399_999 milliseconds = 23:59:59.999
4377                Some(ScalarValue::Time32Millisecond(Some(86_399_999)))
4378            }
4379            DataType::Time64(TimeUnit::Microsecond) => {
4380                // 86_399_999_999 microseconds = 23:59:59.999999
4381                Some(ScalarValue::Time64Microsecond(Some(86_399_999_999)))
4382            }
4383            DataType::Time64(TimeUnit::Nanosecond) => {
4384                // 86_399_999_999_999 nanoseconds = 23:59:59.999999999
4385                Some(ScalarValue::Time64Nanosecond(Some(86_399_999_999_999)))
4386            }
4387            DataType::Timestamp(unit, tz) => match unit {
4388                TimeUnit::Second => {
4389                    Some(ScalarValue::TimestampSecond(Some(i64::MAX), tz.clone()))
4390                }
4391                TimeUnit::Millisecond => Some(ScalarValue::TimestampMillisecond(
4392                    Some(i64::MAX),
4393                    tz.clone(),
4394                )),
4395                TimeUnit::Microsecond => Some(ScalarValue::TimestampMicrosecond(
4396                    Some(i64::MAX),
4397                    tz.clone(),
4398                )),
4399                TimeUnit::Nanosecond => {
4400                    Some(ScalarValue::TimestampNanosecond(Some(i64::MAX), tz.clone()))
4401                }
4402            },
4403            DataType::Duration(unit) => match unit {
4404                TimeUnit::Second => Some(ScalarValue::DurationSecond(Some(i64::MAX))),
4405                TimeUnit::Millisecond => {
4406                    Some(ScalarValue::DurationMillisecond(Some(i64::MAX)))
4407                }
4408                TimeUnit::Microsecond => {
4409                    Some(ScalarValue::DurationMicrosecond(Some(i64::MAX)))
4410                }
4411                TimeUnit::Nanosecond => {
4412                    Some(ScalarValue::DurationNanosecond(Some(i64::MAX)))
4413                }
4414            },
4415            _ => None,
4416        }
4417    }
4418
4419    /// A thin wrapper on Arrow's validation that throws internal error if validation
4420    /// fails.
4421    fn validate_decimal_or_internal_err<T: DecimalType>(
4422        precision: u8,
4423        scale: i8,
4424    ) -> Result<()> {
4425        validate_decimal_precision_and_scale::<T>(precision, scale).map_err(|err| {
4426            _internal_datafusion_err!(
4427                "Decimal precision/scale invariant violated \
4428                 (precision={precision}, scale={scale}): {err}"
4429            )
4430        })
4431    }
4432}
4433
4434/// Compacts the data of an `ArrayData` into a new `ArrayData`.
4435///
4436/// This is useful when you want to minimize the memory footprint of an
4437/// `ArrayData`. For example, the value returned by [`Array::slice`] still
4438/// points at the same underlying data buffers as the original array, which may
4439/// hold many more values. Calling `copy_array_data` on the sliced array will
4440/// create a new, smaller, `ArrayData` that only contains the data for the
4441/// sliced array.
4442///
4443/// # Example
4444/// ```
4445/// # use arrow::array::{make_array, Array, Int32Array};
4446/// use datafusion_common::scalar::copy_array_data;
4447/// let array = Int32Array::from_iter_values(0..8192);
4448/// // Take only the first 2 elements
4449/// let sliced_array = array.slice(0, 2);
4450/// // The memory footprint of `sliced_array` is close to 8192 * 4 bytes
4451/// assert_eq!(32864, sliced_array.get_array_memory_size());
4452/// // however, we can copy the data to a new `ArrayData`
4453/// let new_array = make_array(copy_array_data(&sliced_array.into_data()));
4454/// // The memory footprint of `new_array` is now only 2 * 4 bytes
4455/// // and overhead:
4456/// assert_eq!(160, new_array.get_array_memory_size());
4457/// ```
4458///
4459/// See also [`ScalarValue::compact`] which applies to `ScalarValue` instances
4460/// as necessary.
4461pub fn copy_array_data(src_data: &ArrayData) -> ArrayData {
4462    let mut copy = MutableArrayData::new(vec![&src_data], true, src_data.len());
4463    copy.extend(0, 0, src_data.len());
4464    copy.freeze()
4465}
4466
4467macro_rules! impl_scalar {
4468    ($ty:ty, $scalar:tt) => {
4469        impl From<$ty> for ScalarValue {
4470            fn from(value: $ty) -> Self {
4471                ScalarValue::$scalar(Some(value))
4472            }
4473        }
4474
4475        impl From<Option<$ty>> for ScalarValue {
4476            fn from(value: Option<$ty>) -> Self {
4477                ScalarValue::$scalar(value)
4478            }
4479        }
4480    };
4481}
4482
4483impl_scalar!(f64, Float64);
4484impl_scalar!(f32, Float32);
4485impl_scalar!(f16, Float16);
4486impl_scalar!(i8, Int8);
4487impl_scalar!(i16, Int16);
4488impl_scalar!(i32, Int32);
4489impl_scalar!(i64, Int64);
4490impl_scalar!(bool, Boolean);
4491impl_scalar!(u8, UInt8);
4492impl_scalar!(u16, UInt16);
4493impl_scalar!(u32, UInt32);
4494impl_scalar!(u64, UInt64);
4495
4496impl From<&str> for ScalarValue {
4497    fn from(value: &str) -> Self {
4498        Some(value).into()
4499    }
4500}
4501
4502impl From<Option<&str>> for ScalarValue {
4503    fn from(value: Option<&str>) -> Self {
4504        let value = value.map(|s| s.to_string());
4505        value.into()
4506    }
4507}
4508
4509/// Wrapper to create ScalarValue::Struct for convenience
4510impl From<Vec<(&str, ScalarValue)>> for ScalarValue {
4511    fn from(value: Vec<(&str, ScalarValue)>) -> Self {
4512        value
4513            .into_iter()
4514            .fold(ScalarStructBuilder::new(), |builder, (name, value)| {
4515                builder.with_name_and_scalar(name, value)
4516            })
4517            .build()
4518            .unwrap()
4519    }
4520}
4521
4522impl FromStr for ScalarValue {
4523    type Err = Infallible;
4524
4525    fn from_str(s: &str) -> Result<Self, Self::Err> {
4526        Ok(s.into())
4527    }
4528}
4529
4530impl From<String> for ScalarValue {
4531    fn from(value: String) -> Self {
4532        Some(value).into()
4533    }
4534}
4535
4536impl From<Option<String>> for ScalarValue {
4537    fn from(value: Option<String>) -> Self {
4538        ScalarValue::Utf8(value)
4539    }
4540}
4541
4542macro_rules! impl_try_from {
4543    ($SCALAR:ident, $NATIVE:ident) => {
4544        impl TryFrom<ScalarValue> for $NATIVE {
4545            type Error = DataFusionError;
4546
4547            fn try_from(value: ScalarValue) -> Result<Self> {
4548                match value {
4549                    ScalarValue::$SCALAR(Some(inner_value)) => Ok(inner_value),
4550                    _ => _internal_err!(
4551                        "Cannot convert {:?} to {}",
4552                        value,
4553                        std::any::type_name::<Self>()
4554                    ),
4555                }
4556            }
4557        }
4558    };
4559}
4560
4561impl_try_from!(Int8, i8);
4562impl_try_from!(Int16, i16);
4563
4564// special implementation for i32 because of Date32 and Time32
4565impl TryFrom<ScalarValue> for i32 {
4566    type Error = DataFusionError;
4567
4568    fn try_from(value: ScalarValue) -> Result<Self> {
4569        match value {
4570            ScalarValue::Int32(Some(inner_value))
4571            | ScalarValue::Date32(Some(inner_value))
4572            | ScalarValue::Time32Second(Some(inner_value))
4573            | ScalarValue::Time32Millisecond(Some(inner_value)) => Ok(inner_value),
4574            _ => _internal_err!(
4575                "Cannot convert {:?} to {}",
4576                value,
4577                std::any::type_name::<Self>()
4578            ),
4579        }
4580    }
4581}
4582
4583// special implementation for i64 because of Date64, Time64 and Timestamp
4584impl TryFrom<ScalarValue> for i64 {
4585    type Error = DataFusionError;
4586
4587    fn try_from(value: ScalarValue) -> Result<Self> {
4588        match value {
4589            ScalarValue::Int64(Some(inner_value))
4590            | ScalarValue::Date64(Some(inner_value))
4591            | ScalarValue::Time64Microsecond(Some(inner_value))
4592            | ScalarValue::Time64Nanosecond(Some(inner_value))
4593            | ScalarValue::TimestampNanosecond(Some(inner_value), _)
4594            | ScalarValue::TimestampMicrosecond(Some(inner_value), _)
4595            | ScalarValue::TimestampMillisecond(Some(inner_value), _)
4596            | ScalarValue::TimestampSecond(Some(inner_value), _) => Ok(inner_value),
4597            _ => _internal_err!(
4598                "Cannot convert {:?} to {}",
4599                value,
4600                std::any::type_name::<Self>()
4601            ),
4602        }
4603    }
4604}
4605
4606// special implementation for i128 because of Decimal128
4607impl TryFrom<ScalarValue> for i128 {
4608    type Error = DataFusionError;
4609
4610    fn try_from(value: ScalarValue) -> Result<Self> {
4611        match value {
4612            ScalarValue::Decimal128(Some(inner_value), _, _) => Ok(inner_value),
4613            _ => _internal_err!(
4614                "Cannot convert {:?} to {}",
4615                value,
4616                std::any::type_name::<Self>()
4617            ),
4618        }
4619    }
4620}
4621
4622// special implementation for i256 because of Decimal128
4623impl TryFrom<ScalarValue> for i256 {
4624    type Error = DataFusionError;
4625
4626    fn try_from(value: ScalarValue) -> Result<Self> {
4627        match value {
4628            ScalarValue::Decimal256(Some(inner_value), _, _) => Ok(inner_value),
4629            _ => _internal_err!(
4630                "Cannot convert {:?} to {}",
4631                value,
4632                std::any::type_name::<Self>()
4633            ),
4634        }
4635    }
4636}
4637
4638impl_try_from!(UInt8, u8);
4639impl_try_from!(UInt16, u16);
4640impl_try_from!(UInt32, u32);
4641impl_try_from!(UInt64, u64);
4642impl_try_from!(Float16, f16);
4643impl_try_from!(Float32, f32);
4644impl_try_from!(Float64, f64);
4645impl_try_from!(Boolean, bool);
4646
4647impl TryFrom<DataType> for ScalarValue {
4648    type Error = DataFusionError;
4649
4650    /// Create a Null instance of ScalarValue for this datatype
4651    fn try_from(datatype: DataType) -> Result<Self> {
4652        (&datatype).try_into()
4653    }
4654}
4655
4656impl TryFrom<&DataType> for ScalarValue {
4657    type Error = DataFusionError;
4658
4659    /// Create a Null instance of ScalarValue for this datatype
4660    fn try_from(data_type: &DataType) -> Result<Self> {
4661        Self::try_new_null(data_type)
4662    }
4663}
4664
4665macro_rules! format_option {
4666    ($F:expr, $EXPR:expr) => {{
4667        match $EXPR {
4668            Some(e) => write!($F, "{e}"),
4669            None => write!($F, "NULL"),
4670        }
4671    }};
4672}
4673
4674// Implement Display trait for ScalarValue
4675//
4676// # Panics
4677//
4678// Panics if there is an error when creating a visual representation of columns via `arrow::util::pretty`
4679impl fmt::Display for ScalarValue {
4680    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
4681        match self {
4682            ScalarValue::Decimal32(v, p, s) => {
4683                write!(f, "{v:?},{p:?},{s:?}")?;
4684            }
4685            ScalarValue::Decimal64(v, p, s) => {
4686                write!(f, "{v:?},{p:?},{s:?}")?;
4687            }
4688            ScalarValue::Decimal128(v, p, s) => {
4689                write!(f, "{v:?},{p:?},{s:?}")?;
4690            }
4691            ScalarValue::Decimal256(v, p, s) => {
4692                write!(f, "{v:?},{p:?},{s:?}")?;
4693            }
4694            ScalarValue::Boolean(e) => format_option!(f, e)?,
4695            ScalarValue::Float16(e) => format_option!(f, e)?,
4696            ScalarValue::Float32(e) => format_option!(f, e)?,
4697            ScalarValue::Float64(e) => format_option!(f, e)?,
4698            ScalarValue::Int8(e) => format_option!(f, e)?,
4699            ScalarValue::Int16(e) => format_option!(f, e)?,
4700            ScalarValue::Int32(e) => format_option!(f, e)?,
4701            ScalarValue::Int64(e) => format_option!(f, e)?,
4702            ScalarValue::UInt8(e) => format_option!(f, e)?,
4703            ScalarValue::UInt16(e) => format_option!(f, e)?,
4704            ScalarValue::UInt32(e) => format_option!(f, e)?,
4705            ScalarValue::UInt64(e) => format_option!(f, e)?,
4706            ScalarValue::TimestampSecond(e, _) => format_option!(f, e)?,
4707            ScalarValue::TimestampMillisecond(e, _) => format_option!(f, e)?,
4708            ScalarValue::TimestampMicrosecond(e, _) => format_option!(f, e)?,
4709            ScalarValue::TimestampNanosecond(e, _) => format_option!(f, e)?,
4710            ScalarValue::Utf8(e)
4711            | ScalarValue::LargeUtf8(e)
4712            | ScalarValue::Utf8View(e) => format_option!(f, e)?,
4713            ScalarValue::Binary(e)
4714            | ScalarValue::FixedSizeBinary(_, e)
4715            | ScalarValue::LargeBinary(e)
4716            | ScalarValue::BinaryView(e) => match e {
4717                Some(bytes) => {
4718                    // print up to first 10 bytes, with trailing ... if needed
4719                    for b in bytes.iter().take(10) {
4720                        write!(f, "{b:02X}")?;
4721                    }
4722                    if bytes.len() > 10 {
4723                        write!(f, "...")?;
4724                    }
4725                }
4726                None => write!(f, "NULL")?,
4727            },
4728            ScalarValue::List(arr) => fmt_list(arr.as_ref(), f)?,
4729            ScalarValue::LargeList(arr) => fmt_list(arr.as_ref(), f)?,
4730            ScalarValue::FixedSizeList(arr) => fmt_list(arr.as_ref(), f)?,
4731            ScalarValue::Date32(e) => format_option!(
4732                f,
4733                e.map(|v| {
4734                    let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap();
4735                    match epoch.checked_add_signed(Duration::try_days(v as i64).unwrap())
4736                    {
4737                        Some(date) => date.to_string(),
4738                        None => "".to_string(),
4739                    }
4740                })
4741            )?,
4742            ScalarValue::Date64(e) => format_option!(
4743                f,
4744                e.map(|v| {
4745                    let epoch = NaiveDate::from_ymd_opt(1970, 1, 1).unwrap();
4746                    match epoch.checked_add_signed(Duration::try_milliseconds(v).unwrap())
4747                    {
4748                        Some(date) => date.to_string(),
4749                        None => "".to_string(),
4750                    }
4751                })
4752            )?,
4753            ScalarValue::Time32Second(e) => format_option!(f, e)?,
4754            ScalarValue::Time32Millisecond(e) => format_option!(f, e)?,
4755            ScalarValue::Time64Microsecond(e) => format_option!(f, e)?,
4756            ScalarValue::Time64Nanosecond(e) => format_option!(f, e)?,
4757            ScalarValue::IntervalYearMonth(e) => format_option!(f, e)?,
4758            ScalarValue::IntervalMonthDayNano(e) => {
4759                format_option!(f, e.map(|v| format!("{v:?}")))?
4760            }
4761            ScalarValue::IntervalDayTime(e) => {
4762                format_option!(f, e.map(|v| format!("{v:?}")))?;
4763            }
4764            ScalarValue::DurationSecond(e) => format_option!(f, e)?,
4765            ScalarValue::DurationMillisecond(e) => format_option!(f, e)?,
4766            ScalarValue::DurationMicrosecond(e) => format_option!(f, e)?,
4767            ScalarValue::DurationNanosecond(e) => format_option!(f, e)?,
4768            ScalarValue::Struct(struct_arr) => {
4769                // ScalarValue Struct should always have a single element
4770                assert_eq!(struct_arr.len(), 1);
4771
4772                if struct_arr.null_count() == struct_arr.len() {
4773                    write!(f, "NULL")?;
4774                    return Ok(());
4775                }
4776
4777                let columns = struct_arr.columns();
4778                let fields = struct_arr.fields();
4779                let nulls = struct_arr.nulls();
4780
4781                write!(
4782                    f,
4783                    "{{{}}}",
4784                    columns
4785                        .iter()
4786                        .zip(fields.iter())
4787                        .map(|(column, field)| {
4788                            if nulls.is_some_and(|b| b.is_null(0)) {
4789                                format!("{}:NULL", field.name())
4790                            } else if let DataType::Struct(_) = field.data_type() {
4791                                let sv = ScalarValue::Struct(Arc::new(
4792                                    column.as_struct().to_owned(),
4793                                ));
4794                                format!("{}:{sv}", field.name())
4795                            } else {
4796                                let sv = array_value_to_string(column, 0).unwrap();
4797                                format!("{}:{sv}", field.name())
4798                            }
4799                        })
4800                        .collect::<Vec<_>>()
4801                        .join(",")
4802                )?
4803            }
4804            ScalarValue::Map(map_arr) => {
4805                if map_arr.null_count() == map_arr.len() {
4806                    write!(f, "NULL")?;
4807                    return Ok(());
4808                }
4809
4810                write!(
4811                    f,
4812                    "[{}]",
4813                    map_arr
4814                        .iter()
4815                        .map(|struct_array| {
4816                            if let Some(arr) = struct_array {
4817                                let mut buffer = VecDeque::new();
4818                                for i in 0..arr.len() {
4819                                    let key =
4820                                        array_value_to_string(arr.column(0), i).unwrap();
4821                                    let value =
4822                                        array_value_to_string(arr.column(1), i).unwrap();
4823                                    buffer.push_back(format!("{key}:{value}"));
4824                                }
4825                                format!(
4826                                    "{{{}}}",
4827                                    buffer
4828                                        .into_iter()
4829                                        .collect::<Vec<_>>()
4830                                        .join(",")
4831                                        .as_str()
4832                                )
4833                            } else {
4834                                "NULL".to_string()
4835                            }
4836                        })
4837                        .collect::<Vec<_>>()
4838                        .join(",")
4839                )?
4840            }
4841            ScalarValue::Union(val, _fields, _mode) => match val {
4842                Some((id, val)) => write!(f, "{id}:{val}")?,
4843                None => write!(f, "NULL")?,
4844            },
4845            ScalarValue::Dictionary(_k, v) => write!(f, "{v}")?,
4846            ScalarValue::Null => write!(f, "NULL")?,
4847        };
4848        Ok(())
4849    }
4850}
4851
4852fn fmt_list(arr: &dyn Array, f: &mut fmt::Formatter) -> fmt::Result {
4853    // ScalarValue List, LargeList, FixedSizeList should always have a single element
4854    assert_eq!(arr.len(), 1);
4855    let options = FormatOptions::default().with_display_error(true);
4856    let formatter = ArrayFormatter::try_new(arr, &options).unwrap();
4857    let value_formatter = formatter.value(0);
4858    write!(f, "{value_formatter}")
4859}
4860
4861/// writes a byte array to formatter. `[1, 2, 3]` ==> `"1,2,3"`
4862fn fmt_binary(data: &[u8], f: &mut fmt::Formatter) -> fmt::Result {
4863    let mut iter = data.iter();
4864    if let Some(b) = iter.next() {
4865        write!(f, "{b}")?;
4866    }
4867    for b in iter {
4868        write!(f, ",{b}")?;
4869    }
4870    Ok(())
4871}
4872
4873impl fmt::Debug for ScalarValue {
4874    fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
4875        match self {
4876            ScalarValue::Decimal32(_, _, _) => write!(f, "Decimal32({self})"),
4877            ScalarValue::Decimal64(_, _, _) => write!(f, "Decimal64({self})"),
4878            ScalarValue::Decimal128(_, _, _) => write!(f, "Decimal128({self})"),
4879            ScalarValue::Decimal256(_, _, _) => write!(f, "Decimal256({self})"),
4880            ScalarValue::Boolean(_) => write!(f, "Boolean({self})"),
4881            ScalarValue::Float16(_) => write!(f, "Float16({self})"),
4882            ScalarValue::Float32(_) => write!(f, "Float32({self})"),
4883            ScalarValue::Float64(_) => write!(f, "Float64({self})"),
4884            ScalarValue::Int8(_) => write!(f, "Int8({self})"),
4885            ScalarValue::Int16(_) => write!(f, "Int16({self})"),
4886            ScalarValue::Int32(_) => write!(f, "Int32({self})"),
4887            ScalarValue::Int64(_) => write!(f, "Int64({self})"),
4888            ScalarValue::UInt8(_) => write!(f, "UInt8({self})"),
4889            ScalarValue::UInt16(_) => write!(f, "UInt16({self})"),
4890            ScalarValue::UInt32(_) => write!(f, "UInt32({self})"),
4891            ScalarValue::UInt64(_) => write!(f, "UInt64({self})"),
4892            ScalarValue::TimestampSecond(_, tz_opt) => {
4893                write!(f, "TimestampSecond({self}, {tz_opt:?})")
4894            }
4895            ScalarValue::TimestampMillisecond(_, tz_opt) => {
4896                write!(f, "TimestampMillisecond({self}, {tz_opt:?})")
4897            }
4898            ScalarValue::TimestampMicrosecond(_, tz_opt) => {
4899                write!(f, "TimestampMicrosecond({self}, {tz_opt:?})")
4900            }
4901            ScalarValue::TimestampNanosecond(_, tz_opt) => {
4902                write!(f, "TimestampNanosecond({self}, {tz_opt:?})")
4903            }
4904            ScalarValue::Utf8(None) => write!(f, "Utf8({self})"),
4905            ScalarValue::Utf8(Some(_)) => write!(f, "Utf8(\"{self}\")"),
4906            ScalarValue::Utf8View(None) => write!(f, "Utf8View({self})"),
4907            ScalarValue::Utf8View(Some(_)) => write!(f, "Utf8View(\"{self}\")"),
4908            ScalarValue::LargeUtf8(None) => write!(f, "LargeUtf8({self})"),
4909            ScalarValue::LargeUtf8(Some(_)) => write!(f, "LargeUtf8(\"{self}\")"),
4910            ScalarValue::Binary(None) => write!(f, "Binary({self})"),
4911            ScalarValue::Binary(Some(b)) => {
4912                write!(f, "Binary(\"")?;
4913                fmt_binary(b.as_slice(), f)?;
4914                write!(f, "\")")
4915            }
4916            ScalarValue::BinaryView(None) => write!(f, "BinaryView({self})"),
4917            ScalarValue::BinaryView(Some(b)) => {
4918                write!(f, "BinaryView(\"")?;
4919                fmt_binary(b.as_slice(), f)?;
4920                write!(f, "\")")
4921            }
4922            ScalarValue::FixedSizeBinary(size, None) => {
4923                write!(f, "FixedSizeBinary({size}, {self})")
4924            }
4925            ScalarValue::FixedSizeBinary(size, Some(b)) => {
4926                write!(f, "FixedSizeBinary({size}, \"")?;
4927                fmt_binary(b.as_slice(), f)?;
4928                write!(f, "\")")
4929            }
4930            ScalarValue::LargeBinary(None) => write!(f, "LargeBinary({self})"),
4931            ScalarValue::LargeBinary(Some(b)) => {
4932                write!(f, "LargeBinary(\"")?;
4933                fmt_binary(b.as_slice(), f)?;
4934                write!(f, "\")")
4935            }
4936            ScalarValue::FixedSizeList(_) => write!(f, "FixedSizeList({self})"),
4937            ScalarValue::List(_) => write!(f, "List({self})"),
4938            ScalarValue::LargeList(_) => write!(f, "LargeList({self})"),
4939            ScalarValue::Struct(struct_arr) => {
4940                // ScalarValue Struct should always have a single element
4941                assert_eq!(struct_arr.len(), 1);
4942
4943                let columns = struct_arr.columns();
4944                let fields = struct_arr.fields();
4945
4946                write!(
4947                    f,
4948                    "Struct({{{}}})",
4949                    columns
4950                        .iter()
4951                        .zip(fields.iter())
4952                        .map(|(column, field)| {
4953                            let sv = array_value_to_string(column, 0).unwrap();
4954                            let name = field.name();
4955                            format!("{name}:{sv}")
4956                        })
4957                        .collect::<Vec<_>>()
4958                        .join(",")
4959                )
4960            }
4961            ScalarValue::Map(map_arr) => {
4962                write!(
4963                    f,
4964                    "Map([{}])",
4965                    map_arr
4966                        .iter()
4967                        .map(|struct_array| {
4968                            if let Some(arr) = struct_array {
4969                                let buffer: Vec<String> = (0..arr.len())
4970                                    .map(|i| {
4971                                        let key = array_value_to_string(arr.column(0), i)
4972                                            .unwrap();
4973                                        let value =
4974                                            array_value_to_string(arr.column(1), i)
4975                                                .unwrap();
4976                                        format!("{key:?}:{value:?}")
4977                                    })
4978                                    .collect();
4979                                format!("{{{}}}", buffer.join(","))
4980                            } else {
4981                                "NULL".to_string()
4982                            }
4983                        })
4984                        .collect::<Vec<_>>()
4985                        .join(",")
4986                )
4987            }
4988            ScalarValue::Date32(_) => write!(f, "Date32(\"{self}\")"),
4989            ScalarValue::Date64(_) => write!(f, "Date64(\"{self}\")"),
4990            ScalarValue::Time32Second(_) => write!(f, "Time32Second(\"{self}\")"),
4991            ScalarValue::Time32Millisecond(_) => {
4992                write!(f, "Time32Millisecond(\"{self}\")")
4993            }
4994            ScalarValue::Time64Microsecond(_) => {
4995                write!(f, "Time64Microsecond(\"{self}\")")
4996            }
4997            ScalarValue::Time64Nanosecond(_) => {
4998                write!(f, "Time64Nanosecond(\"{self}\")")
4999            }
5000            ScalarValue::IntervalDayTime(_) => {
5001                write!(f, "IntervalDayTime(\"{self}\")")
5002            }
5003            ScalarValue::IntervalYearMonth(_) => {
5004                write!(f, "IntervalYearMonth(\"{self}\")")
5005            }
5006            ScalarValue::IntervalMonthDayNano(_) => {
5007                write!(f, "IntervalMonthDayNano(\"{self}\")")
5008            }
5009            ScalarValue::DurationSecond(_) => write!(f, "DurationSecond(\"{self}\")"),
5010            ScalarValue::DurationMillisecond(_) => {
5011                write!(f, "DurationMillisecond(\"{self}\")")
5012            }
5013            ScalarValue::DurationMicrosecond(_) => {
5014                write!(f, "DurationMicrosecond(\"{self}\")")
5015            }
5016            ScalarValue::DurationNanosecond(_) => {
5017                write!(f, "DurationNanosecond(\"{self}\")")
5018            }
5019            ScalarValue::Union(val, _fields, _mode) => match val {
5020                Some((id, val)) => write!(f, "Union {id}:{val}"),
5021                None => write!(f, "Union(NULL)"),
5022            },
5023            ScalarValue::Dictionary(k, v) => write!(f, "Dictionary({k:?}, {v:?})"),
5024            ScalarValue::Null => write!(f, "NULL"),
5025        }
5026    }
5027}
5028
5029/// Trait used to map a NativeType to a ScalarValue
5030pub trait ScalarType<T: ArrowNativeType> {
5031    /// returns a scalar from an optional T
5032    fn scalar(r: Option<T>) -> ScalarValue;
5033}
5034
5035impl ScalarType<f32> for Float32Type {
5036    fn scalar(r: Option<f32>) -> ScalarValue {
5037        ScalarValue::Float32(r)
5038    }
5039}
5040
5041impl ScalarType<i64> for TimestampSecondType {
5042    fn scalar(r: Option<i64>) -> ScalarValue {
5043        ScalarValue::TimestampSecond(r, None)
5044    }
5045}
5046
5047impl ScalarType<i64> for TimestampMillisecondType {
5048    fn scalar(r: Option<i64>) -> ScalarValue {
5049        ScalarValue::TimestampMillisecond(r, None)
5050    }
5051}
5052
5053impl ScalarType<i64> for TimestampMicrosecondType {
5054    fn scalar(r: Option<i64>) -> ScalarValue {
5055        ScalarValue::TimestampMicrosecond(r, None)
5056    }
5057}
5058
5059impl ScalarType<i64> for TimestampNanosecondType {
5060    fn scalar(r: Option<i64>) -> ScalarValue {
5061        ScalarValue::TimestampNanosecond(r, None)
5062    }
5063}
5064
5065impl ScalarType<i32> for Date32Type {
5066    fn scalar(r: Option<i32>) -> ScalarValue {
5067        ScalarValue::Date32(r)
5068    }
5069}
5070
5071#[cfg(test)]
5072mod tests {
5073    use std::sync::Arc;
5074
5075    use super::*;
5076    use crate::cast::{as_list_array, as_map_array, as_struct_array};
5077    use crate::test_util::batches_to_string;
5078    use arrow::array::{
5079        FixedSizeListBuilder, Int32Builder, LargeListBuilder, ListBuilder, MapBuilder,
5080        NullArray, NullBufferBuilder, OffsetSizeTrait, PrimitiveBuilder, RecordBatch,
5081        StringBuilder, StringDictionaryBuilder, StructBuilder, UnionBuilder,
5082    };
5083    use arrow::buffer::{Buffer, NullBuffer, OffsetBuffer};
5084    use arrow::compute::{is_null, kernels};
5085    use arrow::datatypes::{
5086        ArrowNumericType, DECIMAL128_MAX_PRECISION, DECIMAL256_MAX_PRECISION, Fields,
5087        Float64Type, TimeUnit,
5088    };
5089    use arrow::error::ArrowError;
5090    use arrow::util::pretty::pretty_format_columns;
5091    use chrono::NaiveDate;
5092    use insta::assert_snapshot;
5093    use rand::Rng;
5094
5095    #[test]
5096    fn test_scalar_value_from_for_map() {
5097        let string_builder = StringBuilder::new();
5098        let int_builder = Int32Builder::with_capacity(4);
5099        let mut builder = MapBuilder::new(None, string_builder, int_builder);
5100        builder.keys().append_value("joe");
5101        builder.values().append_value(1);
5102        builder.append(true).unwrap();
5103
5104        builder.keys().append_value("blogs");
5105        builder.values().append_value(2);
5106        builder.keys().append_value("foo");
5107        builder.values().append_value(4);
5108        builder.append(true).unwrap();
5109        builder.append(true).unwrap();
5110        builder.append(false).unwrap();
5111
5112        let expected = builder.finish();
5113
5114        let sv = ScalarValue::Map(Arc::new(expected.clone()));
5115        let map_arr = sv.to_array().unwrap();
5116        let actual = as_map_array(&map_arr).unwrap();
5117        assert_eq!(actual, &expected);
5118    }
5119
5120    #[test]
5121    fn test_format_timestamp_type_for_error_and_bounds() {
5122        // format helper
5123        let ts_ns = format_timestamp_type_for_error(&DataType::Timestamp(
5124            TimeUnit::Nanosecond,
5125            None,
5126        ));
5127        assert_eq!(ts_ns, "Timestamp(ns)");
5128
5129        let ts_us = format_timestamp_type_for_error(&DataType::Timestamp(
5130            TimeUnit::Microsecond,
5131            None,
5132        ));
5133        assert_eq!(ts_us, "Timestamp(us)");
5134
5135        // ensure_timestamp_in_bounds: Date32 non-overflow
5136        let ok = ensure_timestamp_in_bounds(
5137            1000,
5138            NANOS_PER_DAY,
5139            &DataType::Date32,
5140            &DataType::Timestamp(TimeUnit::Nanosecond, None),
5141        );
5142        assert!(ok.is_ok());
5143
5144        // Date32 overflow -- known large day value (9999-12-31 -> 2932896)
5145        let err = ensure_timestamp_in_bounds(
5146            2932896,
5147            NANOS_PER_DAY,
5148            &DataType::Date32,
5149            &DataType::Timestamp(TimeUnit::Nanosecond, None),
5150        );
5151        assert!(err.is_err());
5152        let msg = err.unwrap_err().to_string();
5153        assert!(msg.contains("Cannot cast Date32 value 2932896 to Timestamp(ns): converted value exceeds the representable i64 range"));
5154
5155        // Date64 overflow for ns (millis * 1_000_000)
5156        let overflow_millis: i64 = (i64::MAX / NANOS_PER_MILLISECOND) + 1;
5157        let err2 = ensure_timestamp_in_bounds(
5158            overflow_millis,
5159            NANOS_PER_MILLISECOND,
5160            &DataType::Date64,
5161            &DataType::Timestamp(TimeUnit::Nanosecond, None),
5162        );
5163        assert!(err2.is_err());
5164    }
5165
5166    #[test]
5167    fn test_scalar_value_from_for_struct() {
5168        let boolean = Arc::new(BooleanArray::from(vec![false]));
5169        let int = Arc::new(Int32Array::from(vec![42]));
5170
5171        let expected = StructArray::from(vec![
5172            (
5173                Arc::new(Field::new("b", DataType::Boolean, false)),
5174                Arc::clone(&boolean) as ArrayRef,
5175            ),
5176            (
5177                Arc::new(Field::new("c", DataType::Int32, false)),
5178                Arc::clone(&int) as ArrayRef,
5179            ),
5180        ]);
5181
5182        let sv = ScalarStructBuilder::new()
5183            .with_array(Field::new("b", DataType::Boolean, false), boolean)
5184            .with_array(Field::new("c", DataType::Int32, false), int)
5185            .build()
5186            .unwrap();
5187
5188        let struct_arr = sv.to_array().unwrap();
5189        let actual = as_struct_array(&struct_arr).unwrap();
5190        assert_eq!(actual, &expected);
5191    }
5192
5193    #[test]
5194    #[should_panic(
5195        expected = "InvalidArgumentError(\"Incorrect array length for StructArray field \\\"bool\\\", expected 1 got 4\")"
5196    )]
5197    fn test_scalar_value_from_for_struct_should_panic() {
5198        let _ = ScalarStructBuilder::new()
5199            .with_array(
5200                Field::new("bool", DataType::Boolean, false),
5201                Arc::new(BooleanArray::from(vec![false, true, false, false])),
5202            )
5203            .with_array(
5204                Field::new("i32", DataType::Int32, false),
5205                Arc::new(Int32Array::from(vec![42, 28, 19, 31])),
5206            )
5207            .build()
5208            .unwrap();
5209    }
5210
5211    #[test]
5212    fn test_to_array_of_size_for_nested() {
5213        // Struct
5214        let boolean = Arc::new(BooleanArray::from(vec![false, false, true, true]));
5215        let int = Arc::new(Int32Array::from(vec![42, 28, 19, 31]));
5216
5217        let struct_array = StructArray::from(vec![
5218            (
5219                Arc::new(Field::new("b", DataType::Boolean, false)),
5220                Arc::clone(&boolean) as ArrayRef,
5221            ),
5222            (
5223                Arc::new(Field::new("c", DataType::Int32, false)),
5224                Arc::clone(&int) as ArrayRef,
5225            ),
5226        ]);
5227        let sv = ScalarValue::Struct(Arc::new(struct_array));
5228        let actual_arr = sv.to_array_of_size(2).unwrap();
5229
5230        let boolean = Arc::new(BooleanArray::from(vec![
5231            false, false, true, true, false, false, true, true,
5232        ]));
5233        let int = Arc::new(Int32Array::from(vec![42, 28, 19, 31, 42, 28, 19, 31]));
5234
5235        let struct_array = StructArray::from(vec![
5236            (
5237                Arc::new(Field::new("b", DataType::Boolean, false)),
5238                Arc::clone(&boolean) as ArrayRef,
5239            ),
5240            (
5241                Arc::new(Field::new("c", DataType::Int32, false)),
5242                Arc::clone(&int) as ArrayRef,
5243            ),
5244        ]);
5245
5246        let actual = as_struct_array(&actual_arr).unwrap();
5247        assert_eq!(actual, &struct_array);
5248
5249        // List
5250        let arr = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
5251            Some(1),
5252            None,
5253            Some(2),
5254        ])]);
5255
5256        let sv = ScalarValue::List(Arc::new(arr));
5257        let actual_arr = sv
5258            .to_array_of_size(2)
5259            .expect("Failed to convert to array of size");
5260        let actual_list_arr = actual_arr.as_list::<i32>();
5261
5262        let arr = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
5263            Some(vec![Some(1), None, Some(2)]),
5264            Some(vec![Some(1), None, Some(2)]),
5265        ]);
5266
5267        assert_eq!(&arr, actual_list_arr);
5268    }
5269
5270    #[test]
5271    fn test_to_array_of_size_for_fsl() {
5272        let values = Int32Array::from_iter([Some(1), None, Some(2)]);
5273        let field = Arc::new(Field::new_list_field(DataType::Int32, true));
5274        let arr = FixedSizeListArray::new(Arc::clone(&field), 3, Arc::new(values), None);
5275        let sv = ScalarValue::FixedSizeList(Arc::new(arr));
5276        let actual_arr = sv
5277            .to_array_of_size(2)
5278            .expect("Failed to convert to array of size");
5279
5280        let expected_values =
5281            Int32Array::from_iter([Some(1), None, Some(2), Some(1), None, Some(2)]);
5282        let expected_arr =
5283            FixedSizeListArray::new(field, 3, Arc::new(expected_values), None);
5284
5285        assert_eq!(
5286            &expected_arr,
5287            as_fixed_size_list_array(actual_arr.as_ref()).unwrap()
5288        );
5289
5290        let empty_array = sv
5291            .to_array_of_size(0)
5292            .expect("Failed to convert to empty array");
5293
5294        assert_eq!(empty_array.len(), 0);
5295    }
5296
5297    /// See https://github.com/apache/datafusion/issues/18870
5298    #[test]
5299    fn test_to_array_of_size_for_none_fsb() {
5300        let sv = ScalarValue::FixedSizeBinary(5, None);
5301        let result = sv
5302            .to_array_of_size(2)
5303            .expect("Failed to convert to array of size");
5304        assert_eq!(result.len(), 2);
5305        assert_eq!(result.null_count(), 2);
5306        assert_eq!(result.as_fixed_size_binary().values().len(), 10);
5307    }
5308
5309    #[test]
5310    fn test_list_to_array_string() {
5311        let scalars = vec![
5312            ScalarValue::from("rust"),
5313            ScalarValue::from("arrow"),
5314            ScalarValue::from("data-fusion"),
5315        ];
5316
5317        let result = ScalarValue::new_list_nullable(scalars.as_slice(), &DataType::Utf8);
5318
5319        let expected = single_row_list_array(vec!["rust", "arrow", "data-fusion"]);
5320        assert_eq!(*result, expected);
5321    }
5322
5323    fn single_row_list_array(items: Vec<&str>) -> ListArray {
5324        SingleRowListArrayBuilder::new(Arc::new(StringArray::from(items)))
5325            .build_list_array()
5326    }
5327
5328    fn build_list<O: OffsetSizeTrait>(
5329        values: Vec<Option<Vec<Option<i64>>>>,
5330    ) -> Vec<ScalarValue> {
5331        values
5332            .into_iter()
5333            .map(|v| {
5334                let arr = if v.is_some() {
5335                    Arc::new(
5336                        GenericListArray::<O>::from_iter_primitive::<Int64Type, _, _>(
5337                            vec![v],
5338                        ),
5339                    )
5340                } else if O::IS_LARGE {
5341                    new_null_array(
5342                        &DataType::LargeList(Arc::new(Field::new_list_field(
5343                            DataType::Int64,
5344                            true,
5345                        ))),
5346                        1,
5347                    )
5348                } else {
5349                    new_null_array(
5350                        &DataType::List(Arc::new(Field::new_list_field(
5351                            DataType::Int64,
5352                            true,
5353                        ))),
5354                        1,
5355                    )
5356                };
5357
5358                if O::IS_LARGE {
5359                    ScalarValue::LargeList(arr.as_list::<i64>().to_owned().into())
5360                } else {
5361                    ScalarValue::List(arr.as_list::<i32>().to_owned().into())
5362                }
5363            })
5364            .collect()
5365    }
5366
5367    #[test]
5368    fn test_iter_to_array_fixed_size_list() {
5369        let field = Arc::new(Field::new_list_field(DataType::Int32, true));
5370        let f1 = Arc::new(FixedSizeListArray::new(
5371            Arc::clone(&field),
5372            3,
5373            Arc::new(Int32Array::from(vec![1, 2, 3])),
5374            None,
5375        ));
5376        let f2 = Arc::new(FixedSizeListArray::new(
5377            Arc::clone(&field),
5378            3,
5379            Arc::new(Int32Array::from(vec![4, 5, 6])),
5380            None,
5381        ));
5382        let f_nulls = Arc::new(FixedSizeListArray::new_null(field, 1, 1));
5383
5384        let scalars = vec![
5385            ScalarValue::FixedSizeList(Arc::clone(&f_nulls)),
5386            ScalarValue::FixedSizeList(f1),
5387            ScalarValue::FixedSizeList(f2),
5388            ScalarValue::FixedSizeList(f_nulls),
5389        ];
5390
5391        let array = ScalarValue::iter_to_array(scalars).unwrap();
5392
5393        let expected = FixedSizeListArray::from_iter_primitive::<Int32Type, _, _>(
5394            vec![
5395                None,
5396                Some(vec![Some(1), Some(2), Some(3)]),
5397                Some(vec![Some(4), Some(5), Some(6)]),
5398                None,
5399            ],
5400            3,
5401        );
5402        assert_eq!(array.as_ref(), &expected);
5403    }
5404
5405    #[test]
5406    fn test_iter_to_array_struct() {
5407        let s1 = StructArray::from(vec![
5408            (
5409                Arc::new(Field::new("A", DataType::Boolean, false)),
5410                Arc::new(BooleanArray::from(vec![false])) as ArrayRef,
5411            ),
5412            (
5413                Arc::new(Field::new("B", DataType::Int32, false)),
5414                Arc::new(Int32Array::from(vec![42])) as ArrayRef,
5415            ),
5416        ]);
5417
5418        let s2 = StructArray::from(vec![
5419            (
5420                Arc::new(Field::new("A", DataType::Boolean, false)),
5421                Arc::new(BooleanArray::from(vec![false])) as ArrayRef,
5422            ),
5423            (
5424                Arc::new(Field::new("B", DataType::Int32, false)),
5425                Arc::new(Int32Array::from(vec![42])) as ArrayRef,
5426            ),
5427        ]);
5428
5429        let scalars = vec![
5430            ScalarValue::Struct(Arc::new(s1)),
5431            ScalarValue::Struct(Arc::new(s2)),
5432        ];
5433
5434        let array = ScalarValue::iter_to_array(scalars).unwrap();
5435
5436        let expected = StructArray::from(vec![
5437            (
5438                Arc::new(Field::new("A", DataType::Boolean, false)),
5439                Arc::new(BooleanArray::from(vec![false, false])) as ArrayRef,
5440            ),
5441            (
5442                Arc::new(Field::new("B", DataType::Int32, false)),
5443                Arc::new(Int32Array::from(vec![42, 42])) as ArrayRef,
5444            ),
5445        ]);
5446        assert_eq!(array.as_ref(), &expected);
5447    }
5448
5449    #[test]
5450    fn test_iter_to_array_struct_with_nulls() {
5451        // non-null
5452        let s1 = StructArray::from((
5453            vec![
5454                (
5455                    Arc::new(Field::new("A", DataType::Int32, false)),
5456                    Arc::new(Int32Array::from(vec![1])) as ArrayRef,
5457                ),
5458                (
5459                    Arc::new(Field::new("B", DataType::Int64, false)),
5460                    Arc::new(Int64Array::from(vec![2])) as ArrayRef,
5461                ),
5462            ],
5463            // Present the null mask, 1 is non-null, 0 is null
5464            Buffer::from(&[1]),
5465        ));
5466
5467        // null
5468        let s2 = StructArray::from((
5469            vec![
5470                (
5471                    Arc::new(Field::new("A", DataType::Int32, false)),
5472                    Arc::new(Int32Array::from(vec![3])) as ArrayRef,
5473                ),
5474                (
5475                    Arc::new(Field::new("B", DataType::Int64, false)),
5476                    Arc::new(Int64Array::from(vec![4])) as ArrayRef,
5477                ),
5478            ],
5479            Buffer::from(&[0]),
5480        ));
5481
5482        let scalars = vec![
5483            ScalarValue::Struct(Arc::new(s1)),
5484            ScalarValue::Struct(Arc::new(s2)),
5485        ];
5486
5487        let array = ScalarValue::iter_to_array(scalars).unwrap();
5488        let struct_array = array.as_struct();
5489        assert!(struct_array.is_valid(0));
5490        assert!(struct_array.is_null(1));
5491    }
5492
5493    #[test]
5494    fn iter_to_array_primitive_test() {
5495        // List[[1,2,3]], List[null], List[[4,5]]
5496        let scalars = build_list::<i32>(vec![
5497            Some(vec![Some(1), Some(2), Some(3)]),
5498            None,
5499            Some(vec![Some(4), Some(5)]),
5500        ]);
5501
5502        let array = ScalarValue::iter_to_array(scalars).unwrap();
5503        let list_array = as_list_array(&array).unwrap();
5504        // List[[1,2,3], null, [4,5]]
5505        let expected = ListArray::from_iter_primitive::<Int64Type, _, _>(vec![
5506            Some(vec![Some(1), Some(2), Some(3)]),
5507            None,
5508            Some(vec![Some(4), Some(5)]),
5509        ]);
5510        assert_eq!(list_array, &expected);
5511
5512        let scalars = build_list::<i64>(vec![
5513            Some(vec![Some(1), Some(2), Some(3)]),
5514            None,
5515            Some(vec![Some(4), Some(5)]),
5516        ]);
5517
5518        let array = ScalarValue::iter_to_array(scalars).unwrap();
5519        let list_array = as_large_list_array(&array).unwrap();
5520        let expected = LargeListArray::from_iter_primitive::<Int64Type, _, _>(vec![
5521            Some(vec![Some(1), Some(2), Some(3)]),
5522            None,
5523            Some(vec![Some(4), Some(5)]),
5524        ]);
5525        assert_eq!(list_array, &expected);
5526    }
5527
5528    #[test]
5529    fn iter_to_array_string_test() {
5530        let arr1 = single_row_list_array(vec!["foo", "bar", "baz"]);
5531        let arr2 = single_row_list_array(vec!["rust", "world"]);
5532
5533        let scalars = vec![
5534            ScalarValue::List(Arc::new(arr1)),
5535            ScalarValue::List(Arc::new(arr2)),
5536        ];
5537
5538        let array = ScalarValue::iter_to_array(scalars).unwrap();
5539        let result = array.as_list::<i32>();
5540
5541        // build expected array
5542        let string_builder = StringBuilder::with_capacity(5, 25);
5543        let mut list_of_string_builder = ListBuilder::new(string_builder);
5544
5545        list_of_string_builder.values().append_value("foo");
5546        list_of_string_builder.values().append_value("bar");
5547        list_of_string_builder.values().append_value("baz");
5548        list_of_string_builder.append(true);
5549
5550        list_of_string_builder.values().append_value("rust");
5551        list_of_string_builder.values().append_value("world");
5552        list_of_string_builder.append(true);
5553        let expected = list_of_string_builder.finish();
5554
5555        assert_eq!(result, &expected);
5556    }
5557
5558    #[test]
5559    fn test_list_scalar_eq_to_array() {
5560        let list_array: ArrayRef =
5561            Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
5562                Some(vec![Some(0), Some(1), Some(2)]),
5563                None,
5564                Some(vec![None, Some(5)]),
5565            ]));
5566
5567        let fsl_array: ArrayRef =
5568            Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
5569                Some(vec![Some(0), Some(1), Some(2)]),
5570                None,
5571                Some(vec![Some(3), None, Some(5)]),
5572            ]));
5573
5574        for arr in [list_array, fsl_array] {
5575            for i in 0..arr.len() {
5576                let scalar =
5577                    ScalarValue::List(arr.slice(i, 1).as_list::<i32>().to_owned().into());
5578                assert!(scalar.eq_array(&arr, i).unwrap());
5579            }
5580        }
5581    }
5582
5583    #[test]
5584    fn test_eq_array_err_message() {
5585        assert_starts_with(
5586            ScalarValue::Utf8(Some("123".to_string()))
5587                .eq_array(&(Arc::new(Int32Array::from(vec![123])) as ArrayRef), 0)
5588                .unwrap_err()
5589                .message(),
5590            "could not cast array of type Int32 to arrow_array::array::byte_array::GenericByteArray<arrow_array::types::GenericStringType<i32>>",
5591        );
5592    }
5593
5594    #[test]
5595    fn scalar_add_trait_test() -> Result<()> {
5596        let float_value = ScalarValue::Float64(Some(123.));
5597        let float_value_2 = ScalarValue::Float64(Some(123.));
5598        assert_eq!(
5599            (float_value.add(&float_value_2))?,
5600            ScalarValue::Float64(Some(246.))
5601        );
5602        assert_eq!(
5603            (float_value.add(float_value_2))?,
5604            ScalarValue::Float64(Some(246.))
5605        );
5606        Ok(())
5607    }
5608
5609    #[test]
5610    fn scalar_sub_trait_test() -> Result<()> {
5611        let float_value = ScalarValue::Float64(Some(123.));
5612        let float_value_2 = ScalarValue::Float64(Some(123.));
5613        assert_eq!(
5614            float_value.sub(&float_value_2)?,
5615            ScalarValue::Float64(Some(0.))
5616        );
5617        assert_eq!(
5618            float_value.sub(float_value_2)?,
5619            ScalarValue::Float64(Some(0.))
5620        );
5621        Ok(())
5622    }
5623
5624    #[test]
5625    fn scalar_sub_trait_int32_test() -> Result<()> {
5626        let int_value = ScalarValue::Int32(Some(42));
5627        let int_value_2 = ScalarValue::Int32(Some(100));
5628        assert_eq!(int_value.sub(&int_value_2)?, ScalarValue::Int32(Some(-58)));
5629        assert_eq!(int_value_2.sub(int_value)?, ScalarValue::Int32(Some(58)));
5630        Ok(())
5631    }
5632
5633    #[test]
5634    fn scalar_sub_trait_int32_overflow_test() {
5635        let int_value = ScalarValue::Int32(Some(i32::MAX));
5636        let int_value_2 = ScalarValue::Int32(Some(i32::MIN));
5637        let err = int_value
5638            .sub_checked(&int_value_2)
5639            .unwrap_err()
5640            .strip_backtrace();
5641        assert_eq!(
5642            err,
5643            "Arrow error: Arithmetic overflow: Overflow happened on: 2147483647 - -2147483648"
5644        )
5645    }
5646
5647    #[test]
5648    fn scalar_sub_trait_int64_test() -> Result<()> {
5649        let int_value = ScalarValue::Int64(Some(42));
5650        let int_value_2 = ScalarValue::Int64(Some(100));
5651        assert_eq!(int_value.sub(&int_value_2)?, ScalarValue::Int64(Some(-58)));
5652        assert_eq!(int_value_2.sub(int_value)?, ScalarValue::Int64(Some(58)));
5653        Ok(())
5654    }
5655
5656    #[test]
5657    fn scalar_sub_trait_int64_overflow_test() {
5658        let int_value = ScalarValue::Int64(Some(i64::MAX));
5659        let int_value_2 = ScalarValue::Int64(Some(i64::MIN));
5660        let err = int_value
5661            .sub_checked(&int_value_2)
5662            .unwrap_err()
5663            .strip_backtrace();
5664        assert_eq!(
5665            err,
5666            "Arrow error: Arithmetic overflow: Overflow happened on: 9223372036854775807 - -9223372036854775808"
5667        )
5668    }
5669
5670    #[test]
5671    fn scalar_add_overflow_test() -> Result<()> {
5672        check_scalar_add_overflow::<Int8Type>(
5673            ScalarValue::Int8(Some(i8::MAX)),
5674            ScalarValue::Int8(Some(i8::MAX)),
5675        );
5676        check_scalar_add_overflow::<UInt8Type>(
5677            ScalarValue::UInt8(Some(u8::MAX)),
5678            ScalarValue::UInt8(Some(u8::MAX)),
5679        );
5680        check_scalar_add_overflow::<Int16Type>(
5681            ScalarValue::Int16(Some(i16::MAX)),
5682            ScalarValue::Int16(Some(i16::MAX)),
5683        );
5684        check_scalar_add_overflow::<UInt16Type>(
5685            ScalarValue::UInt16(Some(u16::MAX)),
5686            ScalarValue::UInt16(Some(u16::MAX)),
5687        );
5688        check_scalar_add_overflow::<Int32Type>(
5689            ScalarValue::Int32(Some(i32::MAX)),
5690            ScalarValue::Int32(Some(i32::MAX)),
5691        );
5692        check_scalar_add_overflow::<UInt32Type>(
5693            ScalarValue::UInt32(Some(u32::MAX)),
5694            ScalarValue::UInt32(Some(u32::MAX)),
5695        );
5696        check_scalar_add_overflow::<Int64Type>(
5697            ScalarValue::Int64(Some(i64::MAX)),
5698            ScalarValue::Int64(Some(i64::MAX)),
5699        );
5700        check_scalar_add_overflow::<UInt64Type>(
5701            ScalarValue::UInt64(Some(u64::MAX)),
5702            ScalarValue::UInt64(Some(u64::MAX)),
5703        );
5704
5705        Ok(())
5706    }
5707
5708    // Verifies that ScalarValue has the same behavior with compute kernel when it overflows.
5709    fn check_scalar_add_overflow<T>(left: ScalarValue, right: ScalarValue)
5710    where
5711        T: ArrowNumericType,
5712    {
5713        let scalar_result = left.add_checked(&right);
5714
5715        let left_array = left.to_array().expect("Failed to convert to array");
5716        let right_array = right.to_array().expect("Failed to convert to array");
5717        let arrow_left_array = left_array.as_primitive::<T>();
5718        let arrow_right_array = right_array.as_primitive::<T>();
5719        let arrow_result = add(arrow_left_array, arrow_right_array);
5720
5721        assert_eq!(scalar_result.is_ok(), arrow_result.is_ok());
5722    }
5723
5724    #[test]
5725    fn test_interval_add_timestamp() -> Result<()> {
5726        let interval = ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano {
5727            months: 1,
5728            days: 2,
5729            nanoseconds: 3,
5730        }));
5731        let timestamp = ScalarValue::TimestampNanosecond(Some(123), None);
5732        let result = interval.add(&timestamp)?;
5733        let expect = timestamp.add(&interval)?;
5734        assert_eq!(result, expect);
5735
5736        let interval = ScalarValue::IntervalYearMonth(Some(123));
5737        let timestamp = ScalarValue::TimestampNanosecond(Some(123), None);
5738        let result = interval.add(&timestamp)?;
5739        let expect = timestamp.add(&interval)?;
5740        assert_eq!(result, expect);
5741
5742        let interval = ScalarValue::IntervalDayTime(Some(IntervalDayTime {
5743            days: 1,
5744            milliseconds: 23,
5745        }));
5746        let timestamp = ScalarValue::TimestampNanosecond(Some(123), None);
5747        let result = interval.add(&timestamp)?;
5748        let expect = timestamp.add(&interval)?;
5749        assert_eq!(result, expect);
5750        Ok(())
5751    }
5752
5753    #[test]
5754    fn test_try_cmp() {
5755        assert_eq!(
5756            ScalarValue::try_cmp(
5757                &ScalarValue::Int32(Some(1)),
5758                &ScalarValue::Int32(Some(2))
5759            )
5760            .unwrap(),
5761            Ordering::Less
5762        );
5763        assert_eq!(
5764            ScalarValue::try_cmp(&ScalarValue::Int32(None), &ScalarValue::Int32(Some(2)))
5765                .unwrap(),
5766            Ordering::Less
5767        );
5768        assert_starts_with(
5769            ScalarValue::try_cmp(
5770                &ScalarValue::Int32(Some(1)),
5771                &ScalarValue::Int64(Some(2)),
5772            )
5773            .unwrap_err()
5774            .message(),
5775            "Uncomparable values: Int32(1), Int64(2)",
5776        );
5777    }
5778
5779    #[test]
5780    fn scalar_decimal_test() -> Result<()> {
5781        let decimal_value = ScalarValue::Decimal128(Some(123), 10, 1);
5782        assert_eq!(DataType::Decimal128(10, 1), decimal_value.data_type());
5783        let try_into_value: i128 = decimal_value.clone().try_into().unwrap();
5784        assert_eq!(123_i128, try_into_value);
5785        assert!(!decimal_value.is_null());
5786        let neg_decimal_value = decimal_value.arithmetic_negate()?;
5787        match neg_decimal_value {
5788            ScalarValue::Decimal128(v, _, _) => {
5789                assert_eq!(-123, v.unwrap());
5790            }
5791            _ => {
5792                unreachable!();
5793            }
5794        }
5795
5796        // decimal scalar to array
5797        let array = decimal_value
5798            .to_array()
5799            .expect("Failed to convert to array");
5800        let array = as_decimal128_array(&array)?;
5801        assert_eq!(1, array.len());
5802        assert_eq!(DataType::Decimal128(10, 1), array.data_type().clone());
5803        assert_eq!(123i128, array.value(0));
5804
5805        // decimal scalar to array with size
5806        let array = decimal_value
5807            .to_array_of_size(10)
5808            .expect("Failed to convert to array of size");
5809        let array_decimal = as_decimal128_array(&array)?;
5810        assert_eq!(10, array.len());
5811        assert_eq!(DataType::Decimal128(10, 1), array.data_type().clone());
5812        assert_eq!(123i128, array_decimal.value(0));
5813        assert_eq!(123i128, array_decimal.value(9));
5814        // test eq array
5815        assert!(
5816            decimal_value
5817                .eq_array(&array, 1)
5818                .expect("Failed to compare arrays")
5819        );
5820        assert!(
5821            decimal_value
5822                .eq_array(&array, 5)
5823                .expect("Failed to compare arrays")
5824        );
5825        // test try from array
5826        assert_eq!(
5827            decimal_value,
5828            ScalarValue::try_from_array(&array, 5).unwrap()
5829        );
5830
5831        assert_eq!(
5832            decimal_value,
5833            ScalarValue::try_new_decimal128(123, 10, 1).unwrap()
5834        );
5835
5836        // test compare
5837        let left = ScalarValue::Decimal128(Some(123), 10, 2);
5838        let right = ScalarValue::Decimal128(Some(124), 10, 2);
5839        assert!(!left.eq(&right));
5840        let result = left < right;
5841        assert!(result);
5842        let result = left <= right;
5843        assert!(result);
5844        let right = ScalarValue::Decimal128(Some(124), 10, 3);
5845        // make sure that two decimals with diff datatype can't be compared.
5846        let result = left.partial_cmp(&right);
5847        assert_eq!(None, result);
5848
5849        let decimal_vec = vec![
5850            ScalarValue::Decimal128(Some(1), 10, 2),
5851            ScalarValue::Decimal128(Some(2), 10, 2),
5852            ScalarValue::Decimal128(Some(3), 10, 2),
5853        ];
5854        // convert the vec to decimal array and check the result
5855        let array = ScalarValue::iter_to_array(decimal_vec).unwrap();
5856        assert_eq!(3, array.len());
5857        assert_eq!(DataType::Decimal128(10, 2), array.data_type().clone());
5858
5859        let decimal_vec = vec![
5860            ScalarValue::Decimal128(Some(1), 10, 2),
5861            ScalarValue::Decimal128(Some(2), 10, 2),
5862            ScalarValue::Decimal128(Some(3), 10, 2),
5863            ScalarValue::Decimal128(None, 10, 2),
5864        ];
5865        let array = ScalarValue::iter_to_array(decimal_vec).unwrap();
5866        assert_eq!(4, array.len());
5867        assert_eq!(DataType::Decimal128(10, 2), array.data_type().clone());
5868
5869        assert!(
5870            ScalarValue::try_new_decimal128(1, 10, 2)
5871                .unwrap()
5872                .eq_array(&array, 0)
5873                .expect("Failed to compare arrays")
5874        );
5875        assert!(
5876            ScalarValue::try_new_decimal128(2, 10, 2)
5877                .unwrap()
5878                .eq_array(&array, 1)
5879                .expect("Failed to compare arrays")
5880        );
5881        assert!(
5882            ScalarValue::try_new_decimal128(3, 10, 2)
5883                .unwrap()
5884                .eq_array(&array, 2)
5885                .expect("Failed to compare arrays")
5886        );
5887        assert_eq!(
5888            ScalarValue::Decimal128(None, 10, 2),
5889            ScalarValue::try_from_array(&array, 3).unwrap()
5890        );
5891
5892        Ok(())
5893    }
5894
5895    #[test]
5896    fn test_new_one_decimal128() {
5897        assert_eq!(
5898            ScalarValue::new_one(&DataType::Decimal128(5, 0)).unwrap(),
5899            ScalarValue::Decimal128(Some(1), 5, 0)
5900        );
5901        assert_eq!(
5902            ScalarValue::new_one(&DataType::Decimal128(5, 1)).unwrap(),
5903            ScalarValue::Decimal128(Some(10), 5, 1)
5904        );
5905        assert_eq!(
5906            ScalarValue::new_one(&DataType::Decimal128(5, 2)).unwrap(),
5907            ScalarValue::Decimal128(Some(100), 5, 2)
5908        );
5909        // More precision
5910        assert_eq!(
5911            ScalarValue::new_one(&DataType::Decimal128(7, 2)).unwrap(),
5912            ScalarValue::Decimal128(Some(100), 7, 2)
5913        );
5914        // No negative scale
5915        assert!(ScalarValue::new_one(&DataType::Decimal128(5, -1)).is_err());
5916        // Invalid combination
5917        assert!(ScalarValue::new_one(&DataType::Decimal128(0, 2)).is_err());
5918        assert!(ScalarValue::new_one(&DataType::Decimal128(5, 7)).is_err());
5919    }
5920
5921    #[test]
5922    fn test_new_one_decimal256() {
5923        assert_eq!(
5924            ScalarValue::new_one(&DataType::Decimal256(5, 0)).unwrap(),
5925            ScalarValue::Decimal256(Some(1.into()), 5, 0)
5926        );
5927        assert_eq!(
5928            ScalarValue::new_one(&DataType::Decimal256(5, 1)).unwrap(),
5929            ScalarValue::Decimal256(Some(10.into()), 5, 1)
5930        );
5931        assert_eq!(
5932            ScalarValue::new_one(&DataType::Decimal256(5, 2)).unwrap(),
5933            ScalarValue::Decimal256(Some(100.into()), 5, 2)
5934        );
5935        // More precision
5936        assert_eq!(
5937            ScalarValue::new_one(&DataType::Decimal256(7, 2)).unwrap(),
5938            ScalarValue::Decimal256(Some(100.into()), 7, 2)
5939        );
5940        // No negative scale
5941        assert!(ScalarValue::new_one(&DataType::Decimal256(5, -1)).is_err());
5942        // Invalid combination
5943        assert!(ScalarValue::new_one(&DataType::Decimal256(0, 2)).is_err());
5944        assert!(ScalarValue::new_one(&DataType::Decimal256(5, 7)).is_err());
5945    }
5946
5947    #[test]
5948    fn test_new_ten_decimal128() {
5949        assert_eq!(
5950            ScalarValue::new_ten(&DataType::Decimal128(5, 1)).unwrap(),
5951            ScalarValue::Decimal128(Some(100), 5, 1)
5952        );
5953        assert_eq!(
5954            ScalarValue::new_ten(&DataType::Decimal128(5, 2)).unwrap(),
5955            ScalarValue::Decimal128(Some(1000), 5, 2)
5956        );
5957        // More precision
5958        assert_eq!(
5959            ScalarValue::new_ten(&DataType::Decimal128(7, 2)).unwrap(),
5960            ScalarValue::Decimal128(Some(1000), 7, 2)
5961        );
5962        // No negative scale
5963        assert!(ScalarValue::new_ten(&DataType::Decimal128(5, -1)).is_err());
5964        // Invalid combination
5965        assert!(ScalarValue::new_ten(&DataType::Decimal128(0, 2)).is_err());
5966        assert!(ScalarValue::new_ten(&DataType::Decimal128(5, 7)).is_err());
5967    }
5968
5969    #[test]
5970    fn test_new_ten_decimal256() {
5971        assert_eq!(
5972            ScalarValue::new_ten(&DataType::Decimal256(5, 1)).unwrap(),
5973            ScalarValue::Decimal256(Some(100.into()), 5, 1)
5974        );
5975        assert_eq!(
5976            ScalarValue::new_ten(&DataType::Decimal256(5, 2)).unwrap(),
5977            ScalarValue::Decimal256(Some(1000.into()), 5, 2)
5978        );
5979        // More precision
5980        assert_eq!(
5981            ScalarValue::new_ten(&DataType::Decimal256(7, 2)).unwrap(),
5982            ScalarValue::Decimal256(Some(1000.into()), 7, 2)
5983        );
5984        // No negative scale
5985        assert!(ScalarValue::new_ten(&DataType::Decimal256(5, -1)).is_err());
5986        // Invalid combination
5987        assert!(ScalarValue::new_ten(&DataType::Decimal256(0, 2)).is_err());
5988        assert!(ScalarValue::new_ten(&DataType::Decimal256(5, 7)).is_err());
5989    }
5990
5991    #[test]
5992    fn test_new_negative_one_decimal128() {
5993        assert_eq!(
5994            ScalarValue::new_negative_one(&DataType::Decimal128(5, 0)).unwrap(),
5995            ScalarValue::Decimal128(Some(-1), 5, 0)
5996        );
5997        assert_eq!(
5998            ScalarValue::new_negative_one(&DataType::Decimal128(5, 2)).unwrap(),
5999            ScalarValue::Decimal128(Some(-100), 5, 2)
6000        );
6001    }
6002
6003    #[test]
6004    fn test_list_partial_cmp() {
6005        let a =
6006            ScalarValue::List(Arc::new(
6007                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6008                    Some(1),
6009                    Some(2),
6010                    Some(3),
6011                ])]),
6012            ));
6013        let b =
6014            ScalarValue::List(Arc::new(
6015                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6016                    Some(1),
6017                    Some(2),
6018                    Some(3),
6019                ])]),
6020            ));
6021        assert_eq!(a.partial_cmp(&b), Some(Ordering::Equal));
6022
6023        let a =
6024            ScalarValue::List(Arc::new(
6025                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6026                    Some(10),
6027                    Some(2),
6028                    Some(3),
6029                ])]),
6030            ));
6031        let b =
6032            ScalarValue::List(Arc::new(
6033                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6034                    Some(1),
6035                    Some(2),
6036                    Some(30),
6037                ])]),
6038            ));
6039        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6040
6041        let a =
6042            ScalarValue::List(Arc::new(
6043                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6044                    Some(10),
6045                    Some(2),
6046                    Some(3),
6047                ])]),
6048            ));
6049        let b =
6050            ScalarValue::List(Arc::new(
6051                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6052                    Some(10),
6053                    Some(2),
6054                    Some(30),
6055                ])]),
6056            ));
6057        assert_eq!(a.partial_cmp(&b), Some(Ordering::Less));
6058
6059        let a =
6060            ScalarValue::List(Arc::new(
6061                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6062                    Some(1),
6063                    Some(2),
6064                    Some(3),
6065                ])]),
6066            ));
6067        let b =
6068            ScalarValue::List(Arc::new(
6069                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6070                    Some(2),
6071                    Some(3),
6072                ])]),
6073            ));
6074        assert_eq!(a.partial_cmp(&b), Some(Ordering::Less));
6075
6076        let a =
6077            ScalarValue::List(Arc::new(
6078                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6079                    Some(2),
6080                    Some(3),
6081                    Some(4),
6082                ])]),
6083            ));
6084        let b =
6085            ScalarValue::List(Arc::new(
6086                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6087                    Some(1),
6088                    Some(2),
6089                ])]),
6090            ));
6091        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6092
6093        let a =
6094            ScalarValue::List(Arc::new(
6095                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6096                    Some(1),
6097                    Some(2),
6098                    Some(3),
6099                ])]),
6100            ));
6101        let b =
6102            ScalarValue::List(Arc::new(
6103                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6104                    Some(1),
6105                    Some(2),
6106                ])]),
6107            ));
6108        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6109
6110        let a =
6111            ScalarValue::List(Arc::new(
6112                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6113                    None,
6114                    Some(2),
6115                    Some(3),
6116                ])]),
6117            ));
6118        let b =
6119            ScalarValue::List(Arc::new(
6120                ListArray::from_iter_primitive::<Int64Type, _, _>(vec![Some(vec![
6121                    Some(1),
6122                    Some(2),
6123                    Some(3),
6124                ])]),
6125            ));
6126        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6127
6128        let a = ScalarValue::LargeList(Arc::new(LargeListArray::from_iter_primitive::<
6129            Int64Type,
6130            _,
6131            _,
6132        >(vec![Some(vec![
6133            None,
6134            Some(2),
6135            Some(3),
6136        ])])));
6137        let b = ScalarValue::LargeList(Arc::new(LargeListArray::from_iter_primitive::<
6138            Int64Type,
6139            _,
6140            _,
6141        >(vec![Some(vec![
6142            Some(1),
6143            Some(2),
6144            Some(3),
6145        ])])));
6146        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6147
6148        let a = ScalarValue::FixedSizeList(Arc::new(
6149            FixedSizeListArray::from_iter_primitive::<Int64Type, _, _>(
6150                vec![Some(vec![None, Some(2), Some(3)])],
6151                3,
6152            ),
6153        ));
6154        let b = ScalarValue::FixedSizeList(Arc::new(
6155            FixedSizeListArray::from_iter_primitive::<Int64Type, _, _>(
6156                vec![Some(vec![Some(1), Some(2), Some(3)])],
6157                3,
6158            ),
6159        ));
6160        assert_eq!(a.partial_cmp(&b), Some(Ordering::Greater));
6161    }
6162
6163    #[test]
6164    fn scalar_value_to_array_u64() -> Result<()> {
6165        let value = ScalarValue::UInt64(Some(13u64));
6166        let array = value.to_array().expect("Failed to convert to array");
6167        let array = as_uint64_array(&array)?;
6168        assert_eq!(array.len(), 1);
6169        assert!(!array.is_null(0));
6170        assert_eq!(array.value(0), 13);
6171
6172        let value = ScalarValue::UInt64(None);
6173        let array = value.to_array().expect("Failed to convert to array");
6174        let array = as_uint64_array(&array)?;
6175        assert_eq!(array.len(), 1);
6176        assert!(array.is_null(0));
6177        Ok(())
6178    }
6179
6180    #[test]
6181    fn scalar_value_to_array_u32() -> Result<()> {
6182        let value = ScalarValue::UInt32(Some(13u32));
6183        let array = value.to_array().expect("Failed to convert to array");
6184        let array = as_uint32_array(&array)?;
6185        assert_eq!(array.len(), 1);
6186        assert!(!array.is_null(0));
6187        assert_eq!(array.value(0), 13);
6188
6189        let value = ScalarValue::UInt32(None);
6190        let array = value.to_array().expect("Failed to convert to array");
6191        let array = as_uint32_array(&array)?;
6192        assert_eq!(array.len(), 1);
6193        assert!(array.is_null(0));
6194        Ok(())
6195    }
6196
6197    #[test]
6198    fn scalar_list_null_to_array() {
6199        let list_array = ScalarValue::new_list_nullable(&[], &DataType::UInt64);
6200
6201        assert_eq!(list_array.len(), 1);
6202        assert_eq!(list_array.values().len(), 0);
6203    }
6204
6205    #[test]
6206    fn scalar_large_list_null_to_array() {
6207        let list_array = ScalarValue::new_large_list(&[], &DataType::UInt64);
6208
6209        assert_eq!(list_array.len(), 1);
6210        assert_eq!(list_array.values().len(), 0);
6211    }
6212
6213    #[test]
6214    fn scalar_list_to_array() -> Result<()> {
6215        let values = vec![
6216            ScalarValue::UInt64(Some(100)),
6217            ScalarValue::UInt64(None),
6218            ScalarValue::UInt64(Some(101)),
6219        ];
6220        let list_array = ScalarValue::new_list_nullable(&values, &DataType::UInt64);
6221        assert_eq!(list_array.len(), 1);
6222        assert_eq!(list_array.values().len(), 3);
6223
6224        let prim_array_ref = list_array.value(0);
6225        let prim_array = as_uint64_array(&prim_array_ref)?;
6226        assert_eq!(prim_array.len(), 3);
6227        assert_eq!(prim_array.value(0), 100);
6228        assert!(prim_array.is_null(1));
6229        assert_eq!(prim_array.value(2), 101);
6230        Ok(())
6231    }
6232
6233    #[test]
6234    fn scalar_large_list_to_array() -> Result<()> {
6235        let values = vec![
6236            ScalarValue::UInt64(Some(100)),
6237            ScalarValue::UInt64(None),
6238            ScalarValue::UInt64(Some(101)),
6239        ];
6240        let list_array = ScalarValue::new_large_list(&values, &DataType::UInt64);
6241        assert_eq!(list_array.len(), 1);
6242        assert_eq!(list_array.values().len(), 3);
6243
6244        let prim_array_ref = list_array.value(0);
6245        let prim_array = as_uint64_array(&prim_array_ref)?;
6246        assert_eq!(prim_array.len(), 3);
6247        assert_eq!(prim_array.value(0), 100);
6248        assert!(prim_array.is_null(1));
6249        assert_eq!(prim_array.value(2), 101);
6250        Ok(())
6251    }
6252
6253    /// Creates array directly and via ScalarValue and ensures they are the same
6254    macro_rules! check_scalar_iter {
6255        ($SCALAR_T:ident, $ARRAYTYPE:ident, $INPUT:expr) => {{
6256            let scalars: Vec<_> =
6257                $INPUT.iter().map(|v| ScalarValue::$SCALAR_T(*v)).collect();
6258
6259            let array = ScalarValue::iter_to_array(scalars.into_iter()).unwrap();
6260
6261            let expected: ArrayRef = Arc::new($ARRAYTYPE::from($INPUT));
6262
6263            assert_eq!(&array, &expected);
6264        }};
6265    }
6266
6267    /// Creates array directly and via ScalarValue and ensures they are the same
6268    /// but for variants that carry a timezone field.
6269    macro_rules! check_scalar_iter_tz {
6270        ($SCALAR_T:ident, $ARRAYTYPE:ident, $INPUT:expr) => {{
6271            let scalars: Vec<_> = $INPUT
6272                .iter()
6273                .map(|v| ScalarValue::$SCALAR_T(*v, None))
6274                .collect();
6275
6276            let array = ScalarValue::iter_to_array(scalars.into_iter()).unwrap();
6277
6278            let expected: ArrayRef = Arc::new($ARRAYTYPE::from($INPUT));
6279
6280            assert_eq!(&array, &expected);
6281        }};
6282    }
6283
6284    /// Creates array directly and via ScalarValue and ensures they
6285    /// are the same, for string  arrays
6286    macro_rules! check_scalar_iter_string {
6287        ($SCALAR_T:ident, $ARRAYTYPE:ident, $INPUT:expr) => {{
6288            let scalars: Vec<_> = $INPUT
6289                .iter()
6290                .map(|v| ScalarValue::$SCALAR_T(v.map(|v| v.to_string())))
6291                .collect();
6292
6293            let array = ScalarValue::iter_to_array(scalars.into_iter()).unwrap();
6294
6295            let expected: ArrayRef = Arc::new($ARRAYTYPE::from($INPUT));
6296
6297            assert_eq!(&array, &expected);
6298        }};
6299    }
6300
6301    /// Creates array directly and via ScalarValue and ensures they
6302    /// are the same, for binary arrays
6303    macro_rules! check_scalar_iter_binary {
6304        ($SCALAR_T:ident, $ARRAYTYPE:ident, $INPUT:expr) => {{
6305            let scalars: Vec<_> = $INPUT
6306                .iter()
6307                .map(|v| ScalarValue::$SCALAR_T(v.map(|v| v.to_vec())))
6308                .collect();
6309
6310            let array = ScalarValue::iter_to_array(scalars.into_iter()).unwrap();
6311
6312            let expected: $ARRAYTYPE =
6313                $INPUT.iter().map(|v| v.map(|v| v.to_vec())).collect();
6314
6315            let expected: ArrayRef = Arc::new(expected);
6316
6317            assert_eq!(&array, &expected);
6318        }};
6319    }
6320
6321    #[test]
6322    fn scalar_iter_to_array_boolean() {
6323        check_scalar_iter!(Boolean, BooleanArray, vec![Some(true), None, Some(false)]);
6324        check_scalar_iter!(Float32, Float32Array, vec![Some(1.9), None, Some(-2.1)]);
6325        check_scalar_iter!(Float64, Float64Array, vec![Some(1.9), None, Some(-2.1)]);
6326
6327        check_scalar_iter!(Int8, Int8Array, vec![Some(1), None, Some(3)]);
6328        check_scalar_iter!(Int16, Int16Array, vec![Some(1), None, Some(3)]);
6329        check_scalar_iter!(Int32, Int32Array, vec![Some(1), None, Some(3)]);
6330        check_scalar_iter!(Int64, Int64Array, vec![Some(1), None, Some(3)]);
6331
6332        check_scalar_iter!(UInt8, UInt8Array, vec![Some(1), None, Some(3)]);
6333        check_scalar_iter!(UInt16, UInt16Array, vec![Some(1), None, Some(3)]);
6334        check_scalar_iter!(UInt32, UInt32Array, vec![Some(1), None, Some(3)]);
6335        check_scalar_iter!(UInt64, UInt64Array, vec![Some(1), None, Some(3)]);
6336
6337        check_scalar_iter_tz!(
6338            TimestampSecond,
6339            TimestampSecondArray,
6340            vec![Some(1), None, Some(3)]
6341        );
6342        check_scalar_iter_tz!(
6343            TimestampMillisecond,
6344            TimestampMillisecondArray,
6345            vec![Some(1), None, Some(3)]
6346        );
6347        check_scalar_iter_tz!(
6348            TimestampMicrosecond,
6349            TimestampMicrosecondArray,
6350            vec![Some(1), None, Some(3)]
6351        );
6352        check_scalar_iter_tz!(
6353            TimestampNanosecond,
6354            TimestampNanosecondArray,
6355            vec![Some(1), None, Some(3)]
6356        );
6357
6358        check_scalar_iter_string!(
6359            Utf8,
6360            StringArray,
6361            vec![Some("foo"), None, Some("bar")]
6362        );
6363        check_scalar_iter_string!(
6364            LargeUtf8,
6365            LargeStringArray,
6366            vec![Some("foo"), None, Some("bar")]
6367        );
6368        check_scalar_iter_binary!(
6369            Binary,
6370            BinaryArray,
6371            [Some(b"foo"), None, Some(b"bar")]
6372        );
6373        check_scalar_iter_binary!(
6374            LargeBinary,
6375            LargeBinaryArray,
6376            [Some(b"foo"), None, Some(b"bar")]
6377        );
6378    }
6379
6380    #[test]
6381    fn scalar_iter_to_array_empty() {
6382        let scalars = vec![] as Vec<ScalarValue>;
6383
6384        let result = ScalarValue::iter_to_array(scalars).unwrap_err();
6385        assert!(
6386            result
6387                .to_string()
6388                .contains("Empty iterator passed to ScalarValue::iter_to_array"),
6389            "{}",
6390            result
6391        );
6392    }
6393
6394    #[test]
6395    fn scalar_iter_to_dictionary() {
6396        fn make_val(v: Option<String>) -> ScalarValue {
6397            let key_type = DataType::Int32;
6398            let value = ScalarValue::Utf8(v);
6399            ScalarValue::Dictionary(Box::new(key_type), Box::new(value))
6400        }
6401
6402        let scalars = [
6403            make_val(Some("Foo".into())),
6404            make_val(None),
6405            make_val(Some("Bar".into())),
6406        ];
6407
6408        let array = ScalarValue::iter_to_array(scalars).unwrap();
6409        let array = as_dictionary_array::<Int32Type>(&array).unwrap();
6410        let values_array = as_string_array(array.values()).unwrap();
6411
6412        let values = array
6413            .keys_iter()
6414            .map(|k| {
6415                k.map(|k| {
6416                    assert!(values_array.is_valid(k));
6417                    values_array.value(k)
6418                })
6419            })
6420            .collect::<Vec<_>>();
6421
6422        let expected = vec![Some("Foo"), None, Some("Bar")];
6423        assert_eq!(values, expected);
6424    }
6425
6426    #[test]
6427    fn scalar_iter_to_array_mismatched_types() {
6428        use ScalarValue::*;
6429        // If the scalar values are not all the correct type, error here
6430        let scalars = [Boolean(Some(true)), Int32(Some(5))];
6431
6432        let result = ScalarValue::iter_to_array(scalars).unwrap_err();
6433        assert!(result.to_string().contains("Inconsistent types in ScalarValue::iter_to_array. Expected Boolean, got Int32(5)"),
6434                "{}", result);
6435    }
6436
6437    #[test]
6438    fn scalar_try_from_array_null() {
6439        let array = vec![Some(33), None].into_iter().collect::<Int64Array>();
6440        let array: ArrayRef = Arc::new(array);
6441
6442        assert_eq!(
6443            ScalarValue::Int64(Some(33)),
6444            ScalarValue::try_from_array(&array, 0).unwrap()
6445        );
6446        assert_eq!(
6447            ScalarValue::Int64(None),
6448            ScalarValue::try_from_array(&array, 1).unwrap()
6449        );
6450    }
6451
6452    #[test]
6453    fn scalar_try_from_array_list_array_null() {
6454        let list = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
6455            Some(vec![Some(1), Some(2)]),
6456            None,
6457        ]);
6458
6459        let non_null_list_scalar = ScalarValue::try_from_array(&list, 0).unwrap();
6460        let null_list_scalar = ScalarValue::try_from_array(&list, 1).unwrap();
6461
6462        let data_type =
6463            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true)));
6464
6465        assert_eq!(non_null_list_scalar.data_type(), data_type);
6466        assert_eq!(null_list_scalar.data_type(), data_type);
6467    }
6468
6469    #[test]
6470    fn scalar_try_from_list_datatypes() {
6471        let inner_field = Arc::new(Field::new_list_field(DataType::Int32, true));
6472
6473        // Test for List
6474        let data_type = &DataType::List(Arc::clone(&inner_field));
6475        let scalar: ScalarValue = data_type.try_into().unwrap();
6476        let expected = ScalarValue::List(
6477            new_null_array(data_type, 1)
6478                .as_list::<i32>()
6479                .to_owned()
6480                .into(),
6481        );
6482        assert_eq!(expected, scalar);
6483        assert!(expected.is_null());
6484
6485        // Test for LargeList
6486        let data_type = &DataType::LargeList(Arc::clone(&inner_field));
6487        let scalar: ScalarValue = data_type.try_into().unwrap();
6488        let expected = ScalarValue::LargeList(
6489            new_null_array(data_type, 1)
6490                .as_list::<i64>()
6491                .to_owned()
6492                .into(),
6493        );
6494        assert_eq!(expected, scalar);
6495        assert!(expected.is_null());
6496
6497        // Test for FixedSizeList(5)
6498        let data_type = &DataType::FixedSizeList(Arc::clone(&inner_field), 5);
6499        let scalar: ScalarValue = data_type.try_into().unwrap();
6500        let expected = ScalarValue::FixedSizeList(
6501            new_null_array(data_type, 1)
6502                .as_fixed_size_list()
6503                .to_owned()
6504                .into(),
6505        );
6506        assert_eq!(expected, scalar);
6507        assert!(expected.is_null());
6508    }
6509
6510    #[test]
6511    fn scalar_try_from_list_of_list() {
6512        let data_type = DataType::List(Arc::new(Field::new_list_field(
6513            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true))),
6514            true,
6515        )));
6516        let data_type = &data_type;
6517        let scalar: ScalarValue = data_type.try_into().unwrap();
6518
6519        let expected = ScalarValue::List(
6520            new_null_array(
6521                &DataType::List(Arc::new(Field::new_list_field(
6522                    DataType::List(Arc::new(Field::new_list_field(
6523                        DataType::Int32,
6524                        true,
6525                    ))),
6526                    true,
6527                ))),
6528                1,
6529            )
6530            .as_list::<i32>()
6531            .to_owned()
6532            .into(),
6533        );
6534
6535        assert_eq!(expected, scalar)
6536    }
6537
6538    #[test]
6539    fn scalar_try_from_not_equal_list_nested_list() {
6540        let list_data_type =
6541            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true)));
6542        let data_type = &list_data_type;
6543        let list_scalar: ScalarValue = data_type.try_into().unwrap();
6544
6545        let nested_list_data_type = DataType::List(Arc::new(Field::new_list_field(
6546            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true))),
6547            true,
6548        )));
6549        let data_type = &nested_list_data_type;
6550        let nested_list_scalar: ScalarValue = data_type.try_into().unwrap();
6551
6552        assert_ne!(list_scalar, nested_list_scalar);
6553    }
6554
6555    #[test]
6556    fn scalar_try_from_dict_datatype() {
6557        let data_type =
6558            DataType::Dictionary(Box::new(DataType::Int8), Box::new(DataType::Utf8));
6559        let data_type = &data_type;
6560        let expected = ScalarValue::Dictionary(
6561            Box::new(DataType::Int8),
6562            Box::new(ScalarValue::Utf8(None)),
6563        );
6564        assert_eq!(expected, data_type.try_into().unwrap())
6565    }
6566
6567    #[test]
6568    fn size_of_scalar() {
6569        // Since ScalarValues are used in a non trivial number of places,
6570        // making it larger means significant more memory consumption
6571        // per distinct value.
6572        //
6573        // Thus this test ensures that no code change makes ScalarValue larger
6574        //
6575        // The alignment requirements differ across architectures and
6576        // thus the size of the enum appears to as well
6577
6578        // The value may also change depending on rust version
6579        assert_eq!(size_of::<ScalarValue>(), 64);
6580    }
6581
6582    #[test]
6583    fn memory_size() {
6584        let sv = ScalarValue::Binary(Some(Vec::with_capacity(10)));
6585        assert_eq!(sv.size(), size_of::<ScalarValue>() + 10,);
6586        let sv_size = sv.size();
6587
6588        let mut v = Vec::with_capacity(10);
6589        // do NOT clone `sv` here because this may shrink the vector capacity
6590        v.push(sv);
6591        assert_eq!(v.capacity(), 10);
6592        assert_eq!(
6593            ScalarValue::size_of_vec(&v),
6594            size_of::<Vec<ScalarValue>>() + (9 * size_of::<ScalarValue>()) + sv_size,
6595        );
6596
6597        let mut s = HashSet::with_capacity(0);
6598        // do NOT clone `sv` here because this may shrink the vector capacity
6599        s.insert(v.pop().unwrap());
6600        // hashsets may easily grow during insert, so capacity is dynamic
6601        let s_capacity = s.capacity();
6602        assert_eq!(
6603            ScalarValue::size_of_hashset(&s),
6604            size_of::<HashSet<ScalarValue>>()
6605                + ((s_capacity - 1) * size_of::<ScalarValue>())
6606                + sv_size,
6607        );
6608    }
6609
6610    #[test]
6611    fn scalar_eq_array() {
6612        // Validate that eq_array has the same semantics as ScalarValue::eq
6613        macro_rules! make_typed_vec {
6614            ($INPUT:expr, $TYPE:ident) => {{
6615                $INPUT
6616                    .iter()
6617                    .map(|v| v.map(|v| v as $TYPE))
6618                    .collect::<Vec<_>>()
6619            }};
6620        }
6621
6622        let bool_vals = [Some(true), None, Some(false)];
6623        let f32_vals = [Some(-1.0), None, Some(1.0)];
6624        let f64_vals = make_typed_vec!(f32_vals, f64);
6625
6626        let i8_vals = [Some(-1), None, Some(1)];
6627        let i16_vals = make_typed_vec!(i8_vals, i16);
6628        let i32_vals = make_typed_vec!(i8_vals, i32);
6629        let i64_vals = make_typed_vec!(i8_vals, i64);
6630
6631        let u8_vals = [Some(0), None, Some(1)];
6632        let u16_vals = make_typed_vec!(u8_vals, u16);
6633        let u32_vals = make_typed_vec!(u8_vals, u32);
6634        let u64_vals = make_typed_vec!(u8_vals, u64);
6635
6636        let str_vals = [Some("foo"), None, Some("bar")];
6637
6638        let interval_dt_vals = [
6639            Some(IntervalDayTime::MINUS_ONE),
6640            None,
6641            Some(IntervalDayTime::ONE),
6642        ];
6643        let interval_mdn_vals = [
6644            Some(IntervalMonthDayNano::MINUS_ONE),
6645            None,
6646            Some(IntervalMonthDayNano::ONE),
6647        ];
6648
6649        /// Test each value in `scalar` with the corresponding element
6650        /// at `array`. Assumes each element is unique (aka not equal
6651        /// with all other indexes)
6652        #[derive(Debug)]
6653        struct TestCase {
6654            array: ArrayRef,
6655            scalars: Vec<ScalarValue>,
6656        }
6657
6658        /// Create a test case for casing the input to the specified array type
6659        macro_rules! make_test_case {
6660            ($INPUT:expr, $ARRAY_TY:ident, $SCALAR_TY:ident) => {{
6661                TestCase {
6662                    array: Arc::new($INPUT.iter().collect::<$ARRAY_TY>()),
6663                    scalars: $INPUT.iter().map(|v| ScalarValue::$SCALAR_TY(*v)).collect(),
6664                }
6665            }};
6666
6667            ($INPUT:expr, $ARRAY_TY:ident, $SCALAR_TY:ident, $TZ:expr) => {{
6668                let tz = $TZ;
6669                TestCase {
6670                    array: Arc::new($INPUT.iter().collect::<$ARRAY_TY>()),
6671                    scalars: $INPUT
6672                        .iter()
6673                        .map(|v| ScalarValue::$SCALAR_TY(*v, tz.clone()))
6674                        .collect(),
6675                }
6676            }};
6677        }
6678
6679        macro_rules! make_str_test_case {
6680            ($INPUT:expr, $ARRAY_TY:ident, $SCALAR_TY:ident) => {{
6681                TestCase {
6682                    array: Arc::new($INPUT.iter().cloned().collect::<$ARRAY_TY>()),
6683                    scalars: $INPUT
6684                        .iter()
6685                        .map(|v| ScalarValue::$SCALAR_TY(v.map(|v| v.to_string())))
6686                        .collect(),
6687                }
6688            }};
6689        }
6690
6691        macro_rules! make_binary_test_case {
6692            ($INPUT:expr, $ARRAY_TY:ident, $SCALAR_TY:ident) => {{
6693                TestCase {
6694                    array: Arc::new($INPUT.iter().cloned().collect::<$ARRAY_TY>()),
6695                    scalars: $INPUT
6696                        .iter()
6697                        .map(|v| {
6698                            ScalarValue::$SCALAR_TY(v.map(|v| v.as_bytes().to_vec()))
6699                        })
6700                        .collect(),
6701                }
6702            }};
6703        }
6704
6705        /// create a test case for DictionaryArray<$INDEX_TY>
6706        macro_rules! make_str_dict_test_case {
6707            ($INPUT:expr, $INDEX_TY:ident) => {{
6708                TestCase {
6709                    array: Arc::new(
6710                        $INPUT
6711                            .iter()
6712                            .cloned()
6713                            .collect::<DictionaryArray<$INDEX_TY>>(),
6714                    ),
6715                    scalars: $INPUT
6716                        .iter()
6717                        .map(|v| {
6718                            ScalarValue::Dictionary(
6719                                Box::new($INDEX_TY::DATA_TYPE),
6720                                Box::new(ScalarValue::Utf8(v.map(|v| v.to_string()))),
6721                            )
6722                        })
6723                        .collect(),
6724                }
6725            }};
6726        }
6727
6728        let cases = vec![
6729            make_test_case!(bool_vals, BooleanArray, Boolean),
6730            make_test_case!(f32_vals, Float32Array, Float32),
6731            make_test_case!(f64_vals, Float64Array, Float64),
6732            make_test_case!(i8_vals, Int8Array, Int8),
6733            make_test_case!(i16_vals, Int16Array, Int16),
6734            make_test_case!(i32_vals, Int32Array, Int32),
6735            make_test_case!(i64_vals, Int64Array, Int64),
6736            make_test_case!(u8_vals, UInt8Array, UInt8),
6737            make_test_case!(u16_vals, UInt16Array, UInt16),
6738            make_test_case!(u32_vals, UInt32Array, UInt32),
6739            make_test_case!(u64_vals, UInt64Array, UInt64),
6740            make_str_test_case!(str_vals, StringArray, Utf8),
6741            make_str_test_case!(str_vals, LargeStringArray, LargeUtf8),
6742            make_binary_test_case!(str_vals, BinaryArray, Binary),
6743            make_binary_test_case!(str_vals, LargeBinaryArray, LargeBinary),
6744            make_test_case!(i32_vals, Date32Array, Date32),
6745            make_test_case!(i64_vals, Date64Array, Date64),
6746            make_test_case!(i32_vals, Time32SecondArray, Time32Second),
6747            make_test_case!(i32_vals, Time32MillisecondArray, Time32Millisecond),
6748            make_test_case!(i64_vals, Time64MicrosecondArray, Time64Microsecond),
6749            make_test_case!(i64_vals, Time64NanosecondArray, Time64Nanosecond),
6750            make_test_case!(i64_vals, TimestampSecondArray, TimestampSecond, None),
6751            make_test_case!(
6752                i64_vals,
6753                TimestampSecondArray,
6754                TimestampSecond,
6755                Some("UTC".into())
6756            ),
6757            make_test_case!(
6758                i64_vals,
6759                TimestampMillisecondArray,
6760                TimestampMillisecond,
6761                None
6762            ),
6763            make_test_case!(
6764                i64_vals,
6765                TimestampMillisecondArray,
6766                TimestampMillisecond,
6767                Some("UTC".into())
6768            ),
6769            make_test_case!(
6770                i64_vals,
6771                TimestampMicrosecondArray,
6772                TimestampMicrosecond,
6773                None
6774            ),
6775            make_test_case!(
6776                i64_vals,
6777                TimestampMicrosecondArray,
6778                TimestampMicrosecond,
6779                Some("UTC".into())
6780            ),
6781            make_test_case!(
6782                i64_vals,
6783                TimestampNanosecondArray,
6784                TimestampNanosecond,
6785                None
6786            ),
6787            make_test_case!(
6788                i64_vals,
6789                TimestampNanosecondArray,
6790                TimestampNanosecond,
6791                Some("UTC".into())
6792            ),
6793            make_test_case!(i32_vals, IntervalYearMonthArray, IntervalYearMonth),
6794            make_test_case!(interval_dt_vals, IntervalDayTimeArray, IntervalDayTime),
6795            make_test_case!(
6796                interval_mdn_vals,
6797                IntervalMonthDayNanoArray,
6798                IntervalMonthDayNano
6799            ),
6800            make_str_dict_test_case!(str_vals, Int8Type),
6801            make_str_dict_test_case!(str_vals, Int16Type),
6802            make_str_dict_test_case!(str_vals, Int32Type),
6803            make_str_dict_test_case!(str_vals, Int64Type),
6804            make_str_dict_test_case!(str_vals, UInt8Type),
6805            make_str_dict_test_case!(str_vals, UInt16Type),
6806            make_str_dict_test_case!(str_vals, UInt32Type),
6807            make_str_dict_test_case!(str_vals, UInt64Type),
6808        ];
6809
6810        for case in cases {
6811            println!("**** Test Case *****");
6812            let TestCase { array, scalars } = case;
6813            println!("Input array type: {}", array.data_type());
6814            println!("Input scalars: {scalars:#?}");
6815            assert_eq!(array.len(), scalars.len());
6816
6817            for (index, scalar) in scalars.into_iter().enumerate() {
6818                assert!(
6819                    scalar
6820                        .eq_array(&array, index)
6821                        .expect("Failed to compare arrays"),
6822                    "Expected {scalar:?} to be equal to {array:?} at index {index}"
6823                );
6824
6825                // test that all other elements are *not* equal
6826                for other_index in 0..array.len() {
6827                    if index != other_index {
6828                        assert!(
6829                            !scalar
6830                                .eq_array(&array, other_index)
6831                                .expect("Failed to compare arrays"),
6832                            "Expected {scalar:?} to be NOT equal to {array:?} at index {other_index}"
6833                        );
6834                    }
6835                }
6836            }
6837        }
6838    }
6839
6840    #[test]
6841    fn scalar_partial_ordering() {
6842        use ScalarValue::*;
6843
6844        assert_eq!(
6845            Int64(Some(33)).partial_cmp(&Int64(Some(0))),
6846            Some(Ordering::Greater)
6847        );
6848        assert_eq!(
6849            Int64(Some(0)).partial_cmp(&Int64(Some(33))),
6850            Some(Ordering::Less)
6851        );
6852        assert_eq!(
6853            Int64(Some(33)).partial_cmp(&Int64(Some(33))),
6854            Some(Ordering::Equal)
6855        );
6856        // For different data type, `partial_cmp` returns None.
6857        assert_eq!(Int64(Some(33)).partial_cmp(&Int32(Some(33))), None);
6858        assert_eq!(Int32(Some(33)).partial_cmp(&Int64(Some(33))), None);
6859
6860        assert_eq!(
6861            ScalarValue::from(vec![
6862                ("A", ScalarValue::from(1.0)),
6863                ("B", ScalarValue::from("Z")),
6864            ])
6865            .partial_cmp(&ScalarValue::from(vec![
6866                ("A", ScalarValue::from(2.0)),
6867                ("B", ScalarValue::from("A")),
6868            ])),
6869            Some(Ordering::Less)
6870        );
6871
6872        // For different struct fields, `partial_cmp` returns None.
6873        assert_eq!(
6874            ScalarValue::from(vec![
6875                ("A", ScalarValue::from(1.0)),
6876                ("B", ScalarValue::from("Z")),
6877            ])
6878            .partial_cmp(&ScalarValue::from(vec![
6879                ("a", ScalarValue::from(2.0)),
6880                ("b", ScalarValue::from("A")),
6881            ])),
6882            None
6883        );
6884    }
6885
6886    #[test]
6887    fn test_scalar_value_from_string() {
6888        let scalar = ScalarValue::from("foo");
6889        assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string())));
6890        let scalar = ScalarValue::from("foo".to_string());
6891        assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string())));
6892        let scalar = ScalarValue::from_str("foo").unwrap();
6893        assert_eq!(scalar, ScalarValue::Utf8(Some("foo".to_string())));
6894    }
6895
6896    #[test]
6897    fn test_scalar_struct() {
6898        let field_a = Arc::new(Field::new("A", DataType::Int32, false));
6899        let field_b = Arc::new(Field::new("B", DataType::Boolean, false));
6900        let field_c = Arc::new(Field::new("C", DataType::Utf8, false));
6901
6902        let field_e = Arc::new(Field::new("e", DataType::Int16, false));
6903        let field_f = Arc::new(Field::new("f", DataType::Int64, false));
6904        let field_d = Arc::new(Field::new(
6905            "D",
6906            DataType::Struct(vec![Arc::clone(&field_e), Arc::clone(&field_f)].into()),
6907            false,
6908        ));
6909
6910        let struct_array = StructArray::from(vec![
6911            (
6912                Arc::clone(&field_e),
6913                Arc::new(Int16Array::from(vec![2])) as ArrayRef,
6914            ),
6915            (
6916                Arc::clone(&field_f),
6917                Arc::new(Int64Array::from(vec![3])) as ArrayRef,
6918            ),
6919        ]);
6920
6921        let struct_array = StructArray::from(vec![
6922            (
6923                Arc::clone(&field_a),
6924                Arc::new(Int32Array::from(vec![23])) as ArrayRef,
6925            ),
6926            (
6927                Arc::clone(&field_b),
6928                Arc::new(BooleanArray::from(vec![false])) as ArrayRef,
6929            ),
6930            (
6931                Arc::clone(&field_c),
6932                Arc::new(StringArray::from(vec!["Hello"])) as ArrayRef,
6933            ),
6934            (Arc::clone(&field_d), Arc::new(struct_array) as ArrayRef),
6935        ]);
6936        let scalar = ScalarValue::Struct(Arc::new(struct_array));
6937
6938        let array = scalar
6939            .to_array_of_size(2)
6940            .expect("Failed to convert to array of size");
6941
6942        let expected = Arc::new(StructArray::from(vec![
6943            (
6944                Arc::clone(&field_a),
6945                Arc::new(Int32Array::from(vec![23, 23])) as ArrayRef,
6946            ),
6947            (
6948                Arc::clone(&field_b),
6949                Arc::new(BooleanArray::from(vec![false, false])) as ArrayRef,
6950            ),
6951            (
6952                Arc::clone(&field_c),
6953                Arc::new(StringArray::from(vec!["Hello", "Hello"])) as ArrayRef,
6954            ),
6955            (
6956                Arc::clone(&field_d),
6957                Arc::new(StructArray::from(vec![
6958                    (
6959                        Arc::clone(&field_e),
6960                        Arc::new(Int16Array::from(vec![2, 2])) as ArrayRef,
6961                    ),
6962                    (
6963                        Arc::clone(&field_f),
6964                        Arc::new(Int64Array::from(vec![3, 3])) as ArrayRef,
6965                    ),
6966                ])) as ArrayRef,
6967            ),
6968        ])) as ArrayRef;
6969
6970        assert_eq!(&array, &expected);
6971
6972        // Construct from second element of ArrayRef
6973        let constructed = ScalarValue::try_from_array(&expected, 1).unwrap();
6974        assert_eq!(constructed, scalar);
6975
6976        // None version
6977        let none_scalar = ScalarValue::try_from(array.data_type()).unwrap();
6978        assert!(none_scalar.is_null());
6979        assert_eq!(
6980            format!("{none_scalar:?}"),
6981            String::from("Struct({A:,B:,C:,D:})")
6982        );
6983
6984        // Construct with convenience From<Vec<(&str, ScalarValue)>>
6985        let constructed = ScalarValue::from(vec![
6986            ("A", ScalarValue::from(23)),
6987            ("B", ScalarValue::from(false)),
6988            ("C", ScalarValue::from("Hello")),
6989            (
6990                "D",
6991                ScalarValue::from(vec![
6992                    ("e", ScalarValue::from(2i16)),
6993                    ("f", ScalarValue::from(3i64)),
6994                ]),
6995            ),
6996        ]);
6997        assert_eq!(constructed, scalar);
6998
6999        // Build Array from Vec of structs
7000        let scalars = vec![
7001            ScalarValue::from(vec![
7002                ("A", ScalarValue::from(23)),
7003                ("B", ScalarValue::from(false)),
7004                ("C", ScalarValue::from("Hello")),
7005                (
7006                    "D",
7007                    ScalarValue::from(vec![
7008                        ("e", ScalarValue::from(2i16)),
7009                        ("f", ScalarValue::from(3i64)),
7010                    ]),
7011                ),
7012            ]),
7013            ScalarValue::from(vec![
7014                ("A", ScalarValue::from(7)),
7015                ("B", ScalarValue::from(true)),
7016                ("C", ScalarValue::from("World")),
7017                (
7018                    "D",
7019                    ScalarValue::from(vec![
7020                        ("e", ScalarValue::from(4i16)),
7021                        ("f", ScalarValue::from(5i64)),
7022                    ]),
7023                ),
7024            ]),
7025            ScalarValue::from(vec![
7026                ("A", ScalarValue::from(-1000)),
7027                ("B", ScalarValue::from(true)),
7028                ("C", ScalarValue::from("!!!!!")),
7029                (
7030                    "D",
7031                    ScalarValue::from(vec![
7032                        ("e", ScalarValue::from(6i16)),
7033                        ("f", ScalarValue::from(7i64)),
7034                    ]),
7035                ),
7036            ]),
7037        ];
7038        let array = ScalarValue::iter_to_array(scalars).unwrap();
7039
7040        let expected = Arc::new(StructArray::from(vec![
7041            (
7042                Arc::clone(&field_a),
7043                Arc::new(Int32Array::from(vec![23, 7, -1000])) as ArrayRef,
7044            ),
7045            (
7046                Arc::clone(&field_b),
7047                Arc::new(BooleanArray::from(vec![false, true, true])) as ArrayRef,
7048            ),
7049            (
7050                Arc::clone(&field_c),
7051                Arc::new(StringArray::from(vec!["Hello", "World", "!!!!!"])) as ArrayRef,
7052            ),
7053            (
7054                Arc::clone(&field_d),
7055                Arc::new(StructArray::from(vec![
7056                    (
7057                        Arc::clone(&field_e),
7058                        Arc::new(Int16Array::from(vec![2, 4, 6])) as ArrayRef,
7059                    ),
7060                    (
7061                        Arc::clone(&field_f),
7062                        Arc::new(Int64Array::from(vec![3, 5, 7])) as ArrayRef,
7063                    ),
7064                ])) as ArrayRef,
7065            ),
7066        ])) as ArrayRef;
7067
7068        assert_eq!(&array, &expected);
7069    }
7070
7071    #[test]
7072    fn round_trip() {
7073        // Each array type should be able to round tripped through a scalar
7074        let cases: Vec<ArrayRef> = vec![
7075            // int
7076            Arc::new(Int8Array::from(vec![Some(1), None, Some(3)])),
7077            Arc::new(Int16Array::from(vec![Some(1), None, Some(3)])),
7078            Arc::new(Int32Array::from(vec![Some(1), None, Some(3)])),
7079            Arc::new(Int64Array::from(vec![Some(1), None, Some(3)])),
7080            Arc::new(UInt8Array::from(vec![Some(1), None, Some(3)])),
7081            Arc::new(UInt16Array::from(vec![Some(1), None, Some(3)])),
7082            Arc::new(UInt32Array::from(vec![Some(1), None, Some(3)])),
7083            Arc::new(UInt64Array::from(vec![Some(1), None, Some(3)])),
7084            // bool
7085            Arc::new(BooleanArray::from(vec![Some(true), None, Some(false)])),
7086            // float
7087            Arc::new(Float32Array::from(vec![Some(1.0), None, Some(3.0)])),
7088            Arc::new(Float64Array::from(vec![Some(1.0), None, Some(3.0)])),
7089            // string array
7090            Arc::new(StringArray::from(vec![Some("foo"), None, Some("bar")])),
7091            Arc::new(LargeStringArray::from(vec![Some("foo"), None, Some("bar")])),
7092            Arc::new(StringViewArray::from(vec![Some("foo"), None, Some("bar")])),
7093            // string dictionary
7094            {
7095                let mut builder = StringDictionaryBuilder::<Int32Type>::new();
7096                builder.append("foo").unwrap();
7097                builder.append_null();
7098                builder.append("bar").unwrap();
7099                Arc::new(builder.finish())
7100            },
7101            // binary array
7102            Arc::new(BinaryArray::from_iter(vec![
7103                Some(b"foo"),
7104                None,
7105                Some(b"bar"),
7106            ])),
7107            Arc::new(LargeBinaryArray::from_iter(vec![
7108                Some(b"foo"),
7109                None,
7110                Some(b"bar"),
7111            ])),
7112            Arc::new(BinaryViewArray::from_iter(vec![
7113                Some(b"foo"),
7114                None,
7115                Some(b"bar"),
7116            ])),
7117            // timestamp
7118            Arc::new(TimestampSecondArray::from(vec![Some(1), None, Some(3)])),
7119            Arc::new(TimestampMillisecondArray::from(vec![
7120                Some(1),
7121                None,
7122                Some(3),
7123            ])),
7124            Arc::new(TimestampMicrosecondArray::from(vec![
7125                Some(1),
7126                None,
7127                Some(3),
7128            ])),
7129            Arc::new(TimestampNanosecondArray::from(vec![Some(1), None, Some(3)])),
7130            // timestamp with timezone
7131            Arc::new(
7132                TimestampSecondArray::from(vec![Some(1), None, Some(3)])
7133                    .with_timezone_opt(Some("UTC")),
7134            ),
7135            Arc::new(
7136                TimestampMillisecondArray::from(vec![Some(1), None, Some(3)])
7137                    .with_timezone_opt(Some("UTC")),
7138            ),
7139            Arc::new(
7140                TimestampMicrosecondArray::from(vec![Some(1), None, Some(3)])
7141                    .with_timezone_opt(Some("UTC")),
7142            ),
7143            Arc::new(
7144                TimestampNanosecondArray::from(vec![Some(1), None, Some(3)])
7145                    .with_timezone_opt(Some("UTC")),
7146            ),
7147            // date
7148            Arc::new(Date32Array::from(vec![Some(1), None, Some(3)])),
7149            Arc::new(Date64Array::from(vec![Some(1), None, Some(3)])),
7150            // time
7151            Arc::new(Time32SecondArray::from(vec![Some(1), None, Some(3)])),
7152            Arc::new(Time32MillisecondArray::from(vec![Some(1), None, Some(3)])),
7153            Arc::new(Time64MicrosecondArray::from(vec![Some(1), None, Some(3)])),
7154            Arc::new(Time64NanosecondArray::from(vec![Some(1), None, Some(3)])),
7155            // null array
7156            Arc::new(NullArray::new(3)),
7157            // dense union
7158            {
7159                let mut builder = UnionBuilder::new_dense();
7160                builder.append::<Int32Type>("a", 1).unwrap();
7161                builder.append::<Float64Type>("b", 3.4).unwrap();
7162                Arc::new(builder.build().unwrap())
7163            },
7164            // sparse union
7165            {
7166                let mut builder = UnionBuilder::new_sparse();
7167                builder.append::<Int32Type>("a", 1).unwrap();
7168                builder.append::<Float64Type>("b", 3.4).unwrap();
7169                Arc::new(builder.build().unwrap())
7170            },
7171            // list array
7172            {
7173                let values_builder = StringBuilder::new();
7174                let mut builder = ListBuilder::new(values_builder);
7175                // [A, B]
7176                builder.values().append_value("A");
7177                builder.values().append_value("B");
7178                builder.append(true);
7179                // [ ] (empty list)
7180                builder.append(true);
7181                // Null
7182                builder.values().append_value("?"); // irrelevant
7183                builder.append(false);
7184                Arc::new(builder.finish())
7185            },
7186            // large list array
7187            {
7188                let values_builder = StringBuilder::new();
7189                let mut builder = LargeListBuilder::new(values_builder);
7190                // [A, B]
7191                builder.values().append_value("A");
7192                builder.values().append_value("B");
7193                builder.append(true);
7194                // [ ] (empty list)
7195                builder.append(true);
7196                // Null
7197                builder.append(false);
7198                Arc::new(builder.finish())
7199            },
7200            // fixed size list array
7201            {
7202                let values_builder = Int32Builder::new();
7203                let mut builder = FixedSizeListBuilder::new(values_builder, 3);
7204
7205                //  [[0, 1, 2], null, [3, null, 5]
7206                builder.values().append_value(0);
7207                builder.values().append_value(1);
7208                builder.values().append_value(2);
7209                builder.append(true);
7210                builder.values().append_null();
7211                builder.values().append_null();
7212                builder.values().append_null();
7213                builder.append(false);
7214                builder.values().append_value(3);
7215                builder.values().append_null();
7216                builder.values().append_value(5);
7217                builder.append(true);
7218                Arc::new(builder.finish())
7219            },
7220            // map
7221            {
7222                let string_builder = StringBuilder::new();
7223                let int_builder = Int32Builder::with_capacity(4);
7224
7225                let mut builder = MapBuilder::new(None, string_builder, int_builder);
7226                // {"joe": 1}
7227                builder.keys().append_value("joe");
7228                builder.values().append_value(1);
7229                builder.append(true).unwrap();
7230                // {}
7231                builder.append(true).unwrap();
7232                // null
7233                builder.append(false).unwrap();
7234
7235                Arc::new(builder.finish())
7236            },
7237        ];
7238
7239        for arr in cases {
7240            round_trip_through_scalar(arr);
7241        }
7242    }
7243
7244    /// for each row in `arr`:
7245    /// 1. convert to a `ScalarValue`
7246    /// 2. Convert `ScalarValue` back to an `ArrayRef`
7247    /// 3. Compare the original array (sliced) and new array for equality
7248    fn round_trip_through_scalar(arr: ArrayRef) {
7249        for i in 0..arr.len() {
7250            // convert Scalar --> Array
7251            let scalar = ScalarValue::try_from_array(&arr, i).unwrap();
7252            let array = scalar.to_array_of_size(1).unwrap();
7253            assert_eq!(array.len(), 1);
7254            assert_eq!(array.data_type(), arr.data_type());
7255            assert_eq!(array.as_ref(), arr.slice(i, 1).as_ref());
7256        }
7257    }
7258
7259    #[test]
7260    fn test_scalar_union_sparse() {
7261        let field_a = Arc::new(Field::new("A", DataType::Int32, true));
7262        let field_b = Arc::new(Field::new("B", DataType::Boolean, true));
7263        let field_c = Arc::new(Field::new("C", DataType::Utf8, true));
7264        let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]);
7265
7266        let mut values_a = vec![None; 6];
7267        values_a[0] = Some(42);
7268        let mut values_b = vec![None; 6];
7269        values_b[1] = Some(true);
7270        let mut values_c = vec![None; 6];
7271        values_c[2] = Some("foo");
7272        let children: Vec<ArrayRef> = vec![
7273            Arc::new(Int32Array::from(values_a)),
7274            Arc::new(BooleanArray::from(values_b)),
7275            Arc::new(StringArray::from(values_c)),
7276        ];
7277
7278        let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]);
7279        let array: ArrayRef = Arc::new(
7280            UnionArray::try_new(fields.clone(), type_ids, None, children)
7281                .expect("UnionArray"),
7282        );
7283
7284        let expected = [
7285            (0, ScalarValue::from(42)),
7286            (1, ScalarValue::from(true)),
7287            (2, ScalarValue::from("foo")),
7288            (0, ScalarValue::Int32(None)),
7289            (1, ScalarValue::Boolean(None)),
7290            (2, ScalarValue::Utf8(None)),
7291        ];
7292
7293        for (i, (ti, value)) in expected.into_iter().enumerate() {
7294            let is_null = value.is_null();
7295            let value = Some((ti, Box::new(value)));
7296            let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Sparse);
7297            let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array");
7298
7299            assert_eq!(
7300                actual, expected,
7301                "[{i}] {actual} was not equal to {expected}"
7302            );
7303
7304            assert!(
7305                expected.eq_array(&array, i).expect("eq_array"),
7306                "[{i}] {expected}.eq_array was false"
7307            );
7308
7309            if is_null {
7310                assert!(actual.is_null(), "[{i}] {actual} was not null")
7311            }
7312        }
7313    }
7314
7315    #[test]
7316    fn test_scalar_union_dense() {
7317        let field_a = Arc::new(Field::new("A", DataType::Int32, true));
7318        let field_b = Arc::new(Field::new("B", DataType::Boolean, true));
7319        let field_c = Arc::new(Field::new("C", DataType::Utf8, true));
7320        let fields = UnionFields::from_iter([(0, field_a), (1, field_b), (2, field_c)]);
7321        let children: Vec<ArrayRef> = vec![
7322            Arc::new(Int32Array::from(vec![Some(42), None])),
7323            Arc::new(BooleanArray::from(vec![Some(true), None])),
7324            Arc::new(StringArray::from(vec![Some("foo"), None])),
7325        ];
7326
7327        let type_ids = ScalarBuffer::from(vec![0, 1, 2, 0, 1, 2]);
7328        let offsets = ScalarBuffer::from(vec![0, 0, 0, 1, 1, 1]);
7329        let array: ArrayRef = Arc::new(
7330            UnionArray::try_new(fields.clone(), type_ids, Some(offsets), children)
7331                .expect("UnionArray"),
7332        );
7333
7334        let expected = [
7335            (0, ScalarValue::from(42)),
7336            (1, ScalarValue::from(true)),
7337            (2, ScalarValue::from("foo")),
7338            (0, ScalarValue::Int32(None)),
7339            (1, ScalarValue::Boolean(None)),
7340            (2, ScalarValue::Utf8(None)),
7341        ];
7342
7343        for (i, (ti, value)) in expected.into_iter().enumerate() {
7344            let is_null = value.is_null();
7345            let value = Some((ti, Box::new(value)));
7346            let expected = ScalarValue::Union(value, fields.clone(), UnionMode::Dense);
7347            let actual = ScalarValue::try_from_array(&array, i).expect("try_from_array");
7348
7349            assert_eq!(
7350                actual, expected,
7351                "[{i}] {actual} was not equal to {expected}"
7352            );
7353
7354            assert!(
7355                expected.eq_array(&array, i).expect("eq_array"),
7356                "[{i}] {expected}.eq_array was false"
7357            );
7358
7359            if is_null {
7360                assert!(actual.is_null(), "[{i}] {actual} was not null")
7361            }
7362        }
7363    }
7364
7365    #[test]
7366    fn test_lists_in_struct() {
7367        let field_a = Arc::new(Field::new("A", DataType::Utf8, false));
7368        let field_primitive_list = Arc::new(Field::new(
7369            "primitive_list",
7370            DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true))),
7371            false,
7372        ));
7373
7374        // Define primitive list scalars
7375        let l0 =
7376            ScalarValue::List(Arc::new(
7377                ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
7378                    Some(1),
7379                    Some(2),
7380                    Some(3),
7381                ])]),
7382            ));
7383        let l1 =
7384            ScalarValue::List(Arc::new(
7385                ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(vec![
7386                    Some(4),
7387                    Some(5),
7388                ])]),
7389            ));
7390        let l2 = ScalarValue::List(Arc::new(ListArray::from_iter_primitive::<
7391            Int32Type,
7392            _,
7393            _,
7394        >(vec![Some(vec![Some(6)])])));
7395
7396        // Define struct scalars
7397        let s0 = ScalarValue::from(vec![
7398            ("A", ScalarValue::from("First")),
7399            ("primitive_list", l0),
7400        ]);
7401
7402        let s1 = ScalarValue::from(vec![
7403            ("A", ScalarValue::from("Second")),
7404            ("primitive_list", l1),
7405        ]);
7406
7407        let s2 = ScalarValue::from(vec![
7408            ("A", ScalarValue::from("Third")),
7409            ("primitive_list", l2),
7410        ]);
7411
7412        // iter_to_array for struct scalars
7413        let array =
7414            ScalarValue::iter_to_array(vec![s0.clone(), s1.clone(), s2.clone()]).unwrap();
7415
7416        let array = as_struct_array(&array).unwrap();
7417        let expected = StructArray::from(vec![
7418            (
7419                Arc::clone(&field_a),
7420                Arc::new(StringArray::from(vec!["First", "Second", "Third"])) as ArrayRef,
7421            ),
7422            (
7423                Arc::clone(&field_primitive_list),
7424                Arc::new(ListArray::from_iter_primitive::<Int32Type, _, _>(vec![
7425                    Some(vec![Some(1), Some(2), Some(3)]),
7426                    Some(vec![Some(4), Some(5)]),
7427                    Some(vec![Some(6)]),
7428                ])),
7429            ),
7430        ]);
7431
7432        assert_eq!(array, &expected);
7433
7434        // Define list-of-structs scalars
7435
7436        let nl0_array = ScalarValue::iter_to_array(vec![s0, s1.clone()]).unwrap();
7437        let nl0 = SingleRowListArrayBuilder::new(nl0_array).build_list_scalar();
7438
7439        let nl1_array = ScalarValue::iter_to_array(vec![s2]).unwrap();
7440        let nl1 = SingleRowListArrayBuilder::new(nl1_array).build_list_scalar();
7441
7442        let nl2_array = ScalarValue::iter_to_array(vec![s1]).unwrap();
7443        let nl2 = SingleRowListArrayBuilder::new(nl2_array).build_list_scalar();
7444
7445        // iter_to_array for list-of-struct
7446        let array = ScalarValue::iter_to_array(vec![nl0, nl1, nl2]).unwrap();
7447        let array = array.as_list::<i32>();
7448
7449        // Construct expected array with array builders
7450        let field_a_builder = StringBuilder::with_capacity(4, 1024);
7451        let primitive_value_builder = Int32Array::builder(8);
7452        let field_primitive_list_builder = ListBuilder::new(primitive_value_builder);
7453
7454        let element_builder = StructBuilder::new(
7455            vec![field_a, field_primitive_list],
7456            vec![
7457                Box::new(field_a_builder),
7458                Box::new(field_primitive_list_builder),
7459            ],
7460        );
7461
7462        let mut list_builder = ListBuilder::new(element_builder);
7463
7464        list_builder
7465            .values()
7466            .field_builder::<StringBuilder>(0)
7467            .unwrap()
7468            .append_value("First");
7469        list_builder
7470            .values()
7471            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7472            .unwrap()
7473            .values()
7474            .append_value(1);
7475        list_builder
7476            .values()
7477            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7478            .unwrap()
7479            .values()
7480            .append_value(2);
7481        list_builder
7482            .values()
7483            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7484            .unwrap()
7485            .values()
7486            .append_value(3);
7487        list_builder
7488            .values()
7489            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7490            .unwrap()
7491            .append(true);
7492        list_builder.values().append(true);
7493
7494        list_builder
7495            .values()
7496            .field_builder::<StringBuilder>(0)
7497            .unwrap()
7498            .append_value("Second");
7499        list_builder
7500            .values()
7501            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7502            .unwrap()
7503            .values()
7504            .append_value(4);
7505        list_builder
7506            .values()
7507            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7508            .unwrap()
7509            .values()
7510            .append_value(5);
7511        list_builder
7512            .values()
7513            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7514            .unwrap()
7515            .append(true);
7516        list_builder.values().append(true);
7517        list_builder.append(true);
7518
7519        list_builder
7520            .values()
7521            .field_builder::<StringBuilder>(0)
7522            .unwrap()
7523            .append_value("Third");
7524        list_builder
7525            .values()
7526            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7527            .unwrap()
7528            .values()
7529            .append_value(6);
7530        list_builder
7531            .values()
7532            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7533            .unwrap()
7534            .append(true);
7535        list_builder.values().append(true);
7536        list_builder.append(true);
7537
7538        list_builder
7539            .values()
7540            .field_builder::<StringBuilder>(0)
7541            .unwrap()
7542            .append_value("Second");
7543        list_builder
7544            .values()
7545            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7546            .unwrap()
7547            .values()
7548            .append_value(4);
7549        list_builder
7550            .values()
7551            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7552            .unwrap()
7553            .values()
7554            .append_value(5);
7555        list_builder
7556            .values()
7557            .field_builder::<ListBuilder<PrimitiveBuilder<Int32Type>>>(1)
7558            .unwrap()
7559            .append(true);
7560        list_builder.values().append(true);
7561        list_builder.append(true);
7562
7563        let expected = list_builder.finish();
7564
7565        assert_eq!(array, &expected);
7566    }
7567
7568    fn build_2d_list(data: Vec<Option<i32>>) -> ListArray {
7569        let a1 = ListArray::from_iter_primitive::<Int32Type, _, _>(vec![Some(data)]);
7570        ListArray::new(
7571            Arc::new(Field::new_list_field(
7572                DataType::List(Arc::new(Field::new_list_field(DataType::Int32, true))),
7573                true,
7574            )),
7575            OffsetBuffer::<i32>::from_lengths([1]),
7576            Arc::new(a1),
7577            None,
7578        )
7579    }
7580
7581    #[test]
7582    fn test_nested_lists() {
7583        // Define inner list scalars
7584        let arr1 = build_2d_list(vec![Some(1), Some(2), Some(3)]);
7585        let arr2 = build_2d_list(vec![Some(4), Some(5)]);
7586        let arr3 = build_2d_list(vec![Some(6)]);
7587
7588        let array = ScalarValue::iter_to_array(vec![
7589            ScalarValue::List(Arc::new(arr1)),
7590            ScalarValue::List(Arc::new(arr2)),
7591            ScalarValue::List(Arc::new(arr3)),
7592        ])
7593        .unwrap();
7594        let array = array.as_list::<i32>();
7595
7596        // Construct expected array with array builders
7597        let inner_builder = Int32Array::builder(6);
7598        let middle_builder = ListBuilder::new(inner_builder);
7599        let mut outer_builder = ListBuilder::new(middle_builder);
7600
7601        outer_builder.values().values().append_value(1);
7602        outer_builder.values().values().append_value(2);
7603        outer_builder.values().values().append_value(3);
7604        outer_builder.values().append(true);
7605        outer_builder.append(true);
7606
7607        outer_builder.values().values().append_value(4);
7608        outer_builder.values().values().append_value(5);
7609        outer_builder.values().append(true);
7610        outer_builder.append(true);
7611
7612        outer_builder.values().values().append_value(6);
7613        outer_builder.values().append(true);
7614        outer_builder.append(true);
7615
7616        let expected = outer_builder.finish();
7617
7618        assert_eq!(array, &expected);
7619    }
7620
7621    #[test]
7622    fn scalar_timestamp_ns_utc_timezone() {
7623        let scalar = ScalarValue::TimestampNanosecond(
7624            Some(1599566400000000000),
7625            Some("UTC".into()),
7626        );
7627
7628        assert_eq!(
7629            scalar.data_type(),
7630            DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into()))
7631        );
7632
7633        let array = scalar.to_array().expect("Failed to convert to array");
7634        assert_eq!(array.len(), 1);
7635        assert_eq!(
7636            array.data_type(),
7637            &DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into()))
7638        );
7639
7640        let new_scalar = ScalarValue::try_from_array(&array, 0).unwrap();
7641        assert_eq!(
7642            new_scalar.data_type(),
7643            DataType::Timestamp(TimeUnit::Nanosecond, Some("UTC".into()))
7644        );
7645    }
7646
7647    #[test]
7648    fn cast_round_trip() {
7649        check_scalar_cast(ScalarValue::Int8(Some(5)), DataType::Int16);
7650        check_scalar_cast(ScalarValue::Int8(None), DataType::Int16);
7651
7652        check_scalar_cast(ScalarValue::Float64(Some(5.5)), DataType::Int16);
7653
7654        check_scalar_cast(ScalarValue::Float64(None), DataType::Int16);
7655
7656        check_scalar_cast(
7657            ScalarValue::from("foo"),
7658            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
7659        );
7660
7661        check_scalar_cast(
7662            ScalarValue::Utf8(None),
7663            DataType::Dictionary(Box::new(DataType::Int32), Box::new(DataType::Utf8)),
7664        );
7665
7666        check_scalar_cast(ScalarValue::Utf8(None), DataType::Utf8View);
7667        check_scalar_cast(ScalarValue::from("foo"), DataType::Utf8View);
7668        check_scalar_cast(
7669            ScalarValue::from("larger than 12 bytes string"),
7670            DataType::Utf8View,
7671        );
7672        check_scalar_cast(
7673            {
7674                let element_field =
7675                    Arc::new(Field::new("element", DataType::Int32, true));
7676
7677                let mut builder =
7678                    ListBuilder::new(Int32Builder::new()).with_field(element_field);
7679                builder.append_value([Some(1)]);
7680                builder.append(true);
7681
7682                ScalarValue::List(Arc::new(builder.finish()))
7683            },
7684            DataType::List(Arc::new(Field::new("element", DataType::Int64, true))),
7685        );
7686        check_scalar_cast(
7687            {
7688                let element_field =
7689                    Arc::new(Field::new("element", DataType::Int32, true));
7690
7691                let mut builder = FixedSizeListBuilder::new(Int32Builder::new(), 1)
7692                    .with_field(element_field);
7693                builder.values().append_value(1);
7694                builder.append(true);
7695
7696                ScalarValue::FixedSizeList(Arc::new(builder.finish()))
7697            },
7698            DataType::FixedSizeList(
7699                Arc::new(Field::new("element", DataType::Int64, true)),
7700                1,
7701            ),
7702        );
7703        check_scalar_cast(
7704            {
7705                let element_field =
7706                    Arc::new(Field::new("element", DataType::Int32, true));
7707
7708                let mut builder =
7709                    LargeListBuilder::new(Int32Builder::new()).with_field(element_field);
7710                builder.append_value([Some(1)]);
7711                builder.append(true);
7712
7713                ScalarValue::LargeList(Arc::new(builder.finish()))
7714            },
7715            DataType::LargeList(Arc::new(Field::new("element", DataType::Int64, true))),
7716        );
7717    }
7718
7719    // mimics how casting work on scalar values by `casting` `scalar` to `desired_type`
7720    fn check_scalar_cast(scalar: ScalarValue, desired_type: DataType) {
7721        // convert from scalar --> Array to call cast
7722        let scalar_array = scalar.to_array().expect("Failed to convert to array");
7723        // cast the actual value
7724        let cast_array = kernels::cast::cast(&scalar_array, &desired_type).unwrap();
7725
7726        // turn it back to a scalar
7727        let cast_scalar = ScalarValue::try_from_array(&cast_array, 0).unwrap();
7728        assert_eq!(cast_scalar.data_type(), desired_type);
7729
7730        // Some time later the "cast" scalar is turned back into an array:
7731        let array = cast_scalar
7732            .to_array_of_size(10)
7733            .expect("Failed to convert to array of size");
7734
7735        // The datatype should be "Dictionary" but is actually Utf8!!!
7736        assert_eq!(array.data_type(), &desired_type)
7737    }
7738
7739    #[test]
7740    fn test_scalar_negative() -> Result<()> {
7741        // positive test
7742        let value = ScalarValue::Int32(Some(12));
7743        assert_eq!(ScalarValue::Int32(Some(-12)), value.arithmetic_negate()?);
7744        let value = ScalarValue::Int32(None);
7745        assert_eq!(ScalarValue::Int32(None), value.arithmetic_negate()?);
7746
7747        // negative test
7748        let value = ScalarValue::UInt8(Some(12));
7749        assert!(value.arithmetic_negate().is_err());
7750        let value = ScalarValue::Boolean(None);
7751        assert!(value.arithmetic_negate().is_err());
7752        Ok(())
7753    }
7754
7755    #[test]
7756    fn test_scalar_negative_overflows() -> Result<()> {
7757        macro_rules! test_overflow_on_value {
7758            ($($val:expr),* $(,)?) => {$(
7759                {
7760                    let value: ScalarValue = $val;
7761                    let err = value.arithmetic_negate().expect_err("Should receive overflow error on negating {value:?}");
7762                    let root_err = err.find_root();
7763                    match  root_err{
7764                        DataFusionError::ArrowError(err, _) if matches!(err.as_ref(), ArrowError::ArithmeticOverflow(_)) => {}
7765                        _ => return Err(err),
7766                    };
7767                }
7768            )*};
7769        }
7770        test_overflow_on_value!(
7771            // the integers
7772            i8::MIN.into(),
7773            i16::MIN.into(),
7774            i32::MIN.into(),
7775            i64::MIN.into(),
7776            // for decimals, only value needs to be tested
7777            ScalarValue::try_new_decimal128(i128::MIN, 10, 5)?,
7778            ScalarValue::Decimal256(Some(i256::MIN), 20, 5),
7779            // interval, check all possible values
7780            ScalarValue::IntervalYearMonth(Some(i32::MIN)),
7781            ScalarValue::new_interval_dt(i32::MIN, 999),
7782            ScalarValue::new_interval_dt(1, i32::MIN),
7783            ScalarValue::new_interval_mdn(i32::MIN, 15, 123_456),
7784            ScalarValue::new_interval_mdn(12, i32::MIN, 123_456),
7785            ScalarValue::new_interval_mdn(12, 15, i64::MIN),
7786            // tz doesn't matter when negating
7787            ScalarValue::TimestampSecond(Some(i64::MIN), None),
7788            ScalarValue::TimestampMillisecond(Some(i64::MIN), None),
7789            ScalarValue::TimestampMicrosecond(Some(i64::MIN), None),
7790            ScalarValue::TimestampNanosecond(Some(i64::MIN), None),
7791        );
7792
7793        let float_cases = [
7794            (
7795                ScalarValue::Float16(Some(f16::MIN)),
7796                ScalarValue::Float16(Some(f16::MAX)),
7797            ),
7798            (
7799                ScalarValue::Float16(Some(f16::MAX)),
7800                ScalarValue::Float16(Some(f16::MIN)),
7801            ),
7802            (f32::MIN.into(), f32::MAX.into()),
7803            (f32::MAX.into(), f32::MIN.into()),
7804            (f64::MIN.into(), f64::MAX.into()),
7805            (f64::MAX.into(), f64::MIN.into()),
7806        ];
7807        // skip float 16 because they aren't supported
7808        for (test, expected) in float_cases.into_iter().skip(2) {
7809            assert_eq!(test.arithmetic_negate()?, expected);
7810        }
7811        Ok(())
7812    }
7813
7814    #[test]
7815    fn f16_test_overflow() {
7816        // TODO: if negate supports f16, add these cases to `test_scalar_negative_overflows` test case
7817        let cases = [
7818            (
7819                ScalarValue::Float16(Some(f16::MIN)),
7820                ScalarValue::Float16(Some(f16::MAX)),
7821            ),
7822            (
7823                ScalarValue::Float16(Some(f16::MAX)),
7824                ScalarValue::Float16(Some(f16::MIN)),
7825            ),
7826        ];
7827
7828        for (test, expected) in cases {
7829            assert_eq!(test.arithmetic_negate().unwrap(), expected);
7830        }
7831    }
7832
7833    macro_rules! expect_operation_error {
7834        ($TEST_NAME:ident, $FUNCTION:ident, $EXPECTED_ERROR:expr) => {
7835            #[test]
7836            fn $TEST_NAME() {
7837                let lhs = ScalarValue::UInt64(Some(12));
7838                let rhs = ScalarValue::Int32(Some(-3));
7839                match lhs.$FUNCTION(&rhs) {
7840                    Ok(_result) => {
7841                        panic!(
7842                            "Expected binary operation error between lhs: '{:?}', rhs: {:?}",
7843                            lhs, rhs
7844                        );
7845                    }
7846                    Err(e) => {
7847                        let error_message = e.to_string();
7848                        assert!(
7849                            error_message.contains($EXPECTED_ERROR),
7850                            "Expected error '{}' not found in actual error '{}'",
7851                            $EXPECTED_ERROR,
7852                            error_message
7853                        );
7854                    }
7855                }
7856            }
7857        };
7858    }
7859
7860    expect_operation_error!(
7861        expect_add_error,
7862        add,
7863        "Invalid arithmetic operation: UInt64 + Int32"
7864    );
7865    expect_operation_error!(
7866        expect_sub_error,
7867        sub,
7868        "Invalid arithmetic operation: UInt64 - Int32"
7869    );
7870
7871    macro_rules! decimal_op_test_cases {
7872    ($OPERATION:ident, [$([$L_VALUE:expr, $L_PRECISION:expr, $L_SCALE:expr, $R_VALUE:expr, $R_PRECISION:expr, $R_SCALE:expr, $O_VALUE:expr, $O_PRECISION:expr, $O_SCALE:expr]),+]) => {
7873            $(
7874
7875                let left = ScalarValue::Decimal128($L_VALUE, $L_PRECISION, $L_SCALE);
7876                let right = ScalarValue::Decimal128($R_VALUE, $R_PRECISION, $R_SCALE);
7877                let result = left.$OPERATION(&right).unwrap();
7878                assert_eq!(ScalarValue::Decimal128($O_VALUE, $O_PRECISION, $O_SCALE), result);
7879
7880            )+
7881        };
7882    }
7883
7884    #[test]
7885    fn decimal_operations() {
7886        decimal_op_test_cases!(
7887            add,
7888            [
7889                [Some(123), 10, 2, Some(124), 10, 2, Some(123 + 124), 11, 2],
7890                // test sum decimal with diff scale
7891                [
7892                    Some(123),
7893                    10,
7894                    3,
7895                    Some(124),
7896                    10,
7897                    2,
7898                    Some(123 + 124 * 10_i128.pow(1)),
7899                    12,
7900                    3
7901                ],
7902                // diff precision and scale for decimal data type
7903                [
7904                    Some(123),
7905                    10,
7906                    2,
7907                    Some(124),
7908                    11,
7909                    3,
7910                    Some(123 * 10_i128.pow(3 - 2) + 124),
7911                    12,
7912                    3
7913                ]
7914            ]
7915        );
7916    }
7917
7918    #[test]
7919    fn decimal_operations_with_nulls() {
7920        decimal_op_test_cases!(
7921            add,
7922            [
7923                // Case: (None, Some, 0)
7924                [None, 10, 2, Some(123), 10, 2, None, 11, 2],
7925                // Case: (Some, None, 0)
7926                [Some(123), 10, 2, None, 10, 2, None, 11, 2],
7927                // Case: (Some, None, _) + Side=False
7928                [Some(123), 8, 2, None, 10, 3, None, 11, 3],
7929                // Case: (None, Some, _) + Side=False
7930                [None, 8, 2, Some(123), 10, 3, None, 11, 3],
7931                // Case: (Some, None, _) + Side=True
7932                [Some(123), 8, 4, None, 10, 3, None, 12, 4],
7933                // Case: (None, Some, _) + Side=True
7934                [None, 10, 3, Some(123), 8, 4, None, 12, 4]
7935            ]
7936        );
7937    }
7938
7939    #[test]
7940    fn test_scalar_distance() {
7941        let cases = [
7942            // scalar (lhs), scalar (rhs), expected distance
7943            // ---------------------------------------------
7944            (ScalarValue::Int8(Some(1)), ScalarValue::Int8(Some(2)), 1),
7945            (ScalarValue::Int8(Some(2)), ScalarValue::Int8(Some(1)), 1),
7946            (
7947                ScalarValue::Int16(Some(-5)),
7948                ScalarValue::Int16(Some(5)),
7949                10,
7950            ),
7951            (
7952                ScalarValue::Int16(Some(5)),
7953                ScalarValue::Int16(Some(-5)),
7954                10,
7955            ),
7956            (ScalarValue::Int32(Some(0)), ScalarValue::Int32(Some(0)), 0),
7957            (
7958                ScalarValue::Int32(Some(-5)),
7959                ScalarValue::Int32(Some(-10)),
7960                5,
7961            ),
7962            (
7963                ScalarValue::Int64(Some(-10)),
7964                ScalarValue::Int64(Some(-5)),
7965                5,
7966            ),
7967            (ScalarValue::UInt8(Some(1)), ScalarValue::UInt8(Some(2)), 1),
7968            (ScalarValue::UInt8(Some(0)), ScalarValue::UInt8(Some(0)), 0),
7969            (
7970                ScalarValue::UInt16(Some(5)),
7971                ScalarValue::UInt16(Some(10)),
7972                5,
7973            ),
7974            (
7975                ScalarValue::UInt32(Some(10)),
7976                ScalarValue::UInt32(Some(5)),
7977                5,
7978            ),
7979            (
7980                ScalarValue::UInt64(Some(5)),
7981                ScalarValue::UInt64(Some(10)),
7982                5,
7983            ),
7984            (
7985                ScalarValue::Float16(Some(f16::from_f32(1.1))),
7986                ScalarValue::Float16(Some(f16::from_f32(1.9))),
7987                1,
7988            ),
7989            (
7990                ScalarValue::Float16(Some(f16::from_f32(-5.3))),
7991                ScalarValue::Float16(Some(f16::from_f32(-9.2))),
7992                4,
7993            ),
7994            (
7995                ScalarValue::Float16(Some(f16::from_f32(-5.3))),
7996                ScalarValue::Float16(Some(f16::from_f32(-9.7))),
7997                4,
7998            ),
7999            (
8000                ScalarValue::Float32(Some(1.0)),
8001                ScalarValue::Float32(Some(2.0)),
8002                1,
8003            ),
8004            (
8005                ScalarValue::Float32(Some(2.0)),
8006                ScalarValue::Float32(Some(1.0)),
8007                1,
8008            ),
8009            (
8010                ScalarValue::Float64(Some(0.0)),
8011                ScalarValue::Float64(Some(0.0)),
8012                0,
8013            ),
8014            (
8015                ScalarValue::Float64(Some(-5.0)),
8016                ScalarValue::Float64(Some(-10.0)),
8017                5,
8018            ),
8019            (
8020                ScalarValue::Float64(Some(-10.0)),
8021                ScalarValue::Float64(Some(-5.0)),
8022                5,
8023            ),
8024            // Floats are currently special cased to f64/f32 and the result is rounded
8025            // rather than ceiled/floored. In the future we might want to take a mode
8026            // which specified the rounding behavior.
8027            (
8028                ScalarValue::Float32(Some(1.2)),
8029                ScalarValue::Float32(Some(1.3)),
8030                0,
8031            ),
8032            (
8033                ScalarValue::Float32(Some(1.1)),
8034                ScalarValue::Float32(Some(1.9)),
8035                1,
8036            ),
8037            (
8038                ScalarValue::Float64(Some(-5.3)),
8039                ScalarValue::Float64(Some(-9.2)),
8040                4,
8041            ),
8042            (
8043                ScalarValue::Float64(Some(-5.3)),
8044                ScalarValue::Float64(Some(-9.7)),
8045                4,
8046            ),
8047            (
8048                ScalarValue::Float64(Some(-5.3)),
8049                ScalarValue::Float64(Some(-9.9)),
8050                5,
8051            ),
8052            (
8053                ScalarValue::Decimal128(Some(10), 1, 0),
8054                ScalarValue::Decimal128(Some(5), 1, 0),
8055                5,
8056            ),
8057            (
8058                ScalarValue::Decimal128(Some(5), 1, 0),
8059                ScalarValue::Decimal128(Some(10), 1, 0),
8060                5,
8061            ),
8062            (
8063                ScalarValue::Decimal256(Some(10.into()), 1, 0),
8064                ScalarValue::Decimal256(Some(5.into()), 1, 0),
8065                5,
8066            ),
8067            (
8068                ScalarValue::Decimal256(Some(5.into()), 1, 0),
8069                ScalarValue::Decimal256(Some(10.into()), 1, 0),
8070                5,
8071            ),
8072        ];
8073        for (lhs, rhs, expected) in cases.iter() {
8074            let distance = lhs.distance(rhs).unwrap();
8075            assert_eq!(distance, *expected);
8076        }
8077    }
8078
8079    #[test]
8080    fn test_distance_none() {
8081        let cases = [
8082            (
8083                ScalarValue::Decimal128(Some(i128::MAX), DECIMAL128_MAX_PRECISION, 0),
8084                ScalarValue::Decimal128(Some(-i128::MAX), DECIMAL128_MAX_PRECISION, 0),
8085            ),
8086            (
8087                ScalarValue::Decimal256(Some(i256::MAX), DECIMAL256_MAX_PRECISION, 0),
8088                ScalarValue::Decimal256(Some(-i256::MAX), DECIMAL256_MAX_PRECISION, 0),
8089            ),
8090        ];
8091        for (lhs, rhs) in cases.iter() {
8092            let distance = lhs.distance(rhs);
8093            assert!(distance.is_none(), "{lhs} vs {rhs}");
8094        }
8095    }
8096
8097    #[test]
8098    fn test_scalar_distance_invalid() {
8099        let cases = [
8100            // scalar (lhs), scalar (rhs)
8101            // --------------------------
8102            // Same type but with nulls
8103            (ScalarValue::Int8(None), ScalarValue::Int8(None)),
8104            (ScalarValue::Int8(None), ScalarValue::Int8(Some(1))),
8105            (ScalarValue::Int8(Some(1)), ScalarValue::Int8(None)),
8106            // Different type
8107            (ScalarValue::Int8(Some(1)), ScalarValue::Int16(Some(1))),
8108            (ScalarValue::Int8(Some(1)), ScalarValue::Float32(Some(1.0))),
8109            (
8110                ScalarValue::Float16(Some(f16::from_f32(1.0))),
8111                ScalarValue::Float32(Some(1.0)),
8112            ),
8113            (
8114                ScalarValue::Float16(Some(f16::from_f32(1.0))),
8115                ScalarValue::Int32(Some(1)),
8116            ),
8117            (
8118                ScalarValue::Float64(Some(1.1)),
8119                ScalarValue::Float32(Some(2.2)),
8120            ),
8121            (
8122                ScalarValue::UInt64(Some(777)),
8123                ScalarValue::Int32(Some(111)),
8124            ),
8125            // Different types with nulls
8126            (ScalarValue::Int8(None), ScalarValue::Int16(Some(1))),
8127            (ScalarValue::Int8(Some(1)), ScalarValue::Int16(None)),
8128            // Unsupported types
8129            (ScalarValue::from("foo"), ScalarValue::from("bar")),
8130            (
8131                ScalarValue::Boolean(Some(true)),
8132                ScalarValue::Boolean(Some(false)),
8133            ),
8134            (ScalarValue::Date32(Some(0)), ScalarValue::Date32(Some(1))),
8135            (ScalarValue::Date64(Some(0)), ScalarValue::Date64(Some(1))),
8136            (
8137                ScalarValue::Decimal128(Some(123), 5, 5),
8138                ScalarValue::Decimal128(Some(120), 5, 3),
8139            ),
8140            (
8141                ScalarValue::Decimal128(Some(123), 5, 5),
8142                ScalarValue::Decimal128(Some(120), 3, 5),
8143            ),
8144            (
8145                ScalarValue::Decimal256(Some(123.into()), 5, 5),
8146                ScalarValue::Decimal256(Some(120.into()), 3, 5),
8147            ),
8148            // Distance 2 * 2^50 is larger than usize
8149            (
8150                ScalarValue::Decimal256(
8151                    Some(i256::from_parts(0, 2_i64.pow(50).into())),
8152                    1,
8153                    0,
8154                ),
8155                ScalarValue::Decimal256(
8156                    Some(i256::from_parts(0, (-(2_i64).pow(50)).into())),
8157                    1,
8158                    0,
8159                ),
8160            ),
8161            // Distance overflow
8162            (
8163                ScalarValue::Decimal256(Some(i256::from_parts(0, i128::MAX)), 1, 0),
8164                ScalarValue::Decimal256(Some(i256::from_parts(0, -i128::MAX)), 1, 0),
8165            ),
8166        ];
8167        for (lhs, rhs) in cases {
8168            let distance = lhs.distance(&rhs);
8169            assert!(distance.is_none());
8170        }
8171    }
8172
8173    #[test]
8174    fn test_scalar_interval_negate() {
8175        let cases = [
8176            (
8177                ScalarValue::new_interval_ym(1, 12),
8178                ScalarValue::new_interval_ym(-1, -12),
8179            ),
8180            (
8181                ScalarValue::new_interval_dt(1, 999),
8182                ScalarValue::new_interval_dt(-1, -999),
8183            ),
8184            (
8185                ScalarValue::new_interval_mdn(12, 15, 123_456),
8186                ScalarValue::new_interval_mdn(-12, -15, -123_456),
8187            ),
8188        ];
8189        for (expr, expected) in cases.iter() {
8190            let result = expr.arithmetic_negate().unwrap();
8191            assert_eq!(*expected, result, "-expr:{expr:?}");
8192        }
8193    }
8194
8195    #[test]
8196    fn test_scalar_interval_add() {
8197        let cases = [
8198            (
8199                ScalarValue::new_interval_ym(1, 12),
8200                ScalarValue::new_interval_ym(1, 12),
8201                ScalarValue::new_interval_ym(2, 24),
8202            ),
8203            (
8204                ScalarValue::new_interval_dt(1, 999),
8205                ScalarValue::new_interval_dt(1, 999),
8206                ScalarValue::new_interval_dt(2, 1998),
8207            ),
8208            (
8209                ScalarValue::new_interval_mdn(12, 15, 123_456),
8210                ScalarValue::new_interval_mdn(12, 15, 123_456),
8211                ScalarValue::new_interval_mdn(24, 30, 246_912),
8212            ),
8213        ];
8214        for (lhs, rhs, expected) in cases.iter() {
8215            let result = lhs.add(rhs).unwrap();
8216            let result_commute = rhs.add(lhs).unwrap();
8217            assert_eq!(*expected, result, "lhs:{lhs:?} + rhs:{rhs:?}");
8218            assert_eq!(*expected, result_commute, "lhs:{rhs:?} + rhs:{lhs:?}");
8219        }
8220    }
8221
8222    #[test]
8223    fn test_scalar_interval_sub() {
8224        let cases = [
8225            (
8226                ScalarValue::new_interval_ym(1, 12),
8227                ScalarValue::new_interval_ym(1, 12),
8228                ScalarValue::new_interval_ym(0, 0),
8229            ),
8230            (
8231                ScalarValue::new_interval_dt(1, 999),
8232                ScalarValue::new_interval_dt(1, 999),
8233                ScalarValue::new_interval_dt(0, 0),
8234            ),
8235            (
8236                ScalarValue::new_interval_mdn(12, 15, 123_456),
8237                ScalarValue::new_interval_mdn(12, 15, 123_456),
8238                ScalarValue::new_interval_mdn(0, 0, 0),
8239            ),
8240        ];
8241        for (lhs, rhs, expected) in cases.iter() {
8242            let result = lhs.sub(rhs).unwrap();
8243            assert_eq!(*expected, result, "lhs:{lhs:?} - rhs:{rhs:?}");
8244        }
8245    }
8246
8247    #[test]
8248    fn timestamp_op_random_tests() {
8249        // timestamp1 + (or -) interval = timestamp2
8250        // timestamp2 - timestamp1 (or timestamp1 - timestamp2) = interval ?
8251        let sample_size = 1000;
8252        let timestamps1 = get_random_timestamps(sample_size);
8253        let intervals = get_random_intervals(sample_size);
8254        // ts(sec) + interval(ns) = ts(sec); however,
8255        // ts(sec) - ts(sec) cannot be = interval(ns). Therefore,
8256        // timestamps are more precise than intervals in tests.
8257        for (idx, ts1) in timestamps1.iter().enumerate() {
8258            if idx % 2 == 0 {
8259                let timestamp2 = ts1.add(intervals[idx].clone()).unwrap();
8260                let back = timestamp2.sub(intervals[idx].clone()).unwrap();
8261                assert_eq!(ts1, &back);
8262            } else {
8263                let timestamp2 = ts1.sub(intervals[idx].clone()).unwrap();
8264                let back = timestamp2.add(intervals[idx].clone()).unwrap();
8265                assert_eq!(ts1, &back);
8266            };
8267        }
8268    }
8269
8270    #[test]
8271    fn test_struct_nulls() {
8272        let fields_b = Fields::from(vec![
8273            Field::new("ba", DataType::UInt64, true),
8274            Field::new("bb", DataType::UInt64, true),
8275        ]);
8276        let fields = Fields::from(vec![
8277            Field::new("a", DataType::UInt64, true),
8278            Field::new("b", DataType::Struct(fields_b.clone()), true),
8279        ]);
8280
8281        let struct_value = vec![
8282            (
8283                Arc::clone(&fields[0]),
8284                Arc::new(UInt64Array::from(vec![Some(1)])) as ArrayRef,
8285            ),
8286            (
8287                Arc::clone(&fields[1]),
8288                Arc::new(StructArray::from(vec![
8289                    (
8290                        Arc::clone(&fields_b[0]),
8291                        Arc::new(UInt64Array::from(vec![Some(2)])) as ArrayRef,
8292                    ),
8293                    (
8294                        Arc::clone(&fields_b[1]),
8295                        Arc::new(UInt64Array::from(vec![Some(3)])) as ArrayRef,
8296                    ),
8297                ])) as ArrayRef,
8298            ),
8299        ];
8300
8301        let struct_value_with_nulls = vec![
8302            (
8303                Arc::clone(&fields[0]),
8304                Arc::new(UInt64Array::from(vec![Some(1)])) as ArrayRef,
8305            ),
8306            (
8307                Arc::clone(&fields[1]),
8308                Arc::new(StructArray::from((
8309                    vec![
8310                        (
8311                            Arc::clone(&fields_b[0]),
8312                            Arc::new(UInt64Array::from(vec![Some(2)])) as ArrayRef,
8313                        ),
8314                        (
8315                            Arc::clone(&fields_b[1]),
8316                            Arc::new(UInt64Array::from(vec![Some(3)])) as ArrayRef,
8317                        ),
8318                    ],
8319                    Buffer::from(&[0]),
8320                ))) as ArrayRef,
8321            ),
8322        ];
8323
8324        let scalars = vec![
8325            // all null
8326            ScalarValue::Struct(Arc::new(StructArray::from((
8327                struct_value.clone(),
8328                Buffer::from(&[0]),
8329            )))),
8330            // field 1 valid, field 2 null
8331            ScalarValue::Struct(Arc::new(StructArray::from((
8332                struct_value_with_nulls.clone(),
8333                Buffer::from(&[1]),
8334            )))),
8335            // all valid
8336            ScalarValue::Struct(Arc::new(StructArray::from((
8337                struct_value.clone(),
8338                Buffer::from(&[1]),
8339            )))),
8340        ];
8341
8342        let check_array = |array| {
8343            let is_null = is_null(&array).unwrap();
8344            assert_eq!(is_null, BooleanArray::from(vec![true, false, false]));
8345
8346            let formatted = pretty_format_columns("col", &[array]).unwrap().to_string();
8347            let formatted = formatted.split('\n').collect::<Vec<_>>();
8348            let expected = vec![
8349                "+---------------------------+",
8350                "| col                       |",
8351                "+---------------------------+",
8352                "|                           |",
8353                "| {a: 1, b: }               |",
8354                "| {a: 1, b: {ba: 2, bb: 3}} |",
8355                "+---------------------------+",
8356            ];
8357            assert_eq!(
8358                formatted, expected,
8359                "Actual:\n{formatted:#?}\n\nExpected:\n{expected:#?}"
8360            );
8361        };
8362
8363        // test `ScalarValue::iter_to_array`
8364        let array = ScalarValue::iter_to_array(scalars.clone()).unwrap();
8365        check_array(array);
8366
8367        // test `ScalarValue::to_array` / `ScalarValue::to_array_of_size`
8368        let arrays = scalars
8369            .iter()
8370            .map(ScalarValue::to_array)
8371            .collect::<Result<Vec<_>>>()
8372            .expect("Failed to convert to array");
8373        let arrays = arrays.iter().map(|a| a.as_ref()).collect::<Vec<_>>();
8374        let array = arrow::compute::concat(&arrays).unwrap();
8375        check_array(array);
8376    }
8377
8378    #[test]
8379    fn test_struct_display() {
8380        let field_a = Field::new("a", DataType::Int32, true);
8381        let field_b = Field::new("b", DataType::Utf8, true);
8382
8383        let s = ScalarStructBuilder::new()
8384            .with_scalar(field_a, ScalarValue::from(1i32))
8385            .with_scalar(field_b, ScalarValue::Utf8(None))
8386            .build()
8387            .unwrap();
8388
8389        assert_eq!(s.to_string(), "{a:1,b:}");
8390        assert_eq!(format!("{s:?}"), r#"Struct({a:1,b:})"#);
8391
8392        let ScalarValue::Struct(arr) = s else {
8393            panic!("Expected struct");
8394        };
8395
8396        //verify compared to arrow display
8397        let batch = RecordBatch::try_from_iter(vec![("s", arr as _)]).unwrap();
8398        assert_snapshot!(batches_to_string(&[batch]), @r"
8399        +-------------+
8400        | s           |
8401        +-------------+
8402        | {a: 1, b: } |
8403        +-------------+
8404        ");
8405    }
8406
8407    #[test]
8408    fn test_null_bug() {
8409        let field_a = Field::new("a", DataType::Int32, true);
8410        let field_b = Field::new("b", DataType::Int32, true);
8411        let fields = Fields::from(vec![field_a, field_b]);
8412
8413        let array_a = Arc::new(Int32Array::from_iter_values([1]));
8414        let array_b = Arc::new(Int32Array::from_iter_values([2]));
8415        let arrays: Vec<ArrayRef> = vec![array_a, array_b];
8416
8417        let mut not_nulls = NullBufferBuilder::new(1);
8418
8419        not_nulls.append_non_null();
8420
8421        let ar = StructArray::new(fields, arrays, not_nulls.finish());
8422        let s = ScalarValue::Struct(Arc::new(ar));
8423
8424        assert_eq!(s.to_string(), "{a:1,b:2}");
8425        assert_eq!(format!("{s:?}"), r#"Struct({a:1,b:2})"#);
8426
8427        let ScalarValue::Struct(arr) = s else {
8428            panic!("Expected struct");
8429        };
8430
8431        //verify compared to arrow display
8432        let batch = RecordBatch::try_from_iter(vec![("s", arr as _)]).unwrap();
8433        assert_snapshot!(batches_to_string(&[batch]), @r"
8434        +--------------+
8435        | s            |
8436        +--------------+
8437        | {a: 1, b: 2} |
8438        +--------------+
8439        ");
8440    }
8441
8442    #[test]
8443    fn test_display_date64_large_values() {
8444        assert_eq!(
8445            format!("{}", ScalarValue::Date64(Some(790179464505))),
8446            "1995-01-15"
8447        );
8448        // This used to panic, see https://github.com/apache/arrow-rs/issues/7728
8449        assert_eq!(
8450            format!("{}", ScalarValue::Date64(Some(-790179464505600000))),
8451            ""
8452        );
8453    }
8454
8455    #[test]
8456    fn test_struct_display_null() {
8457        let fields = vec![Field::new("a", DataType::Int32, false)];
8458        let s = ScalarStructBuilder::new_null(fields);
8459        assert_eq!(s.to_string(), "NULL");
8460
8461        let ScalarValue::Struct(arr) = s else {
8462            panic!("Expected struct");
8463        };
8464
8465        //verify compared to arrow display
8466        let batch = RecordBatch::try_from_iter(vec![("s", arr as _)]).unwrap();
8467
8468        assert_snapshot!(batches_to_string(&[batch]), @r"
8469        +---+
8470        | s |
8471        +---+
8472        |   |
8473        +---+
8474        ");
8475    }
8476
8477    #[test]
8478    fn test_map_display_and_debug() {
8479        let string_builder = StringBuilder::new();
8480        let int_builder = Int32Builder::with_capacity(4);
8481        let mut builder = MapBuilder::new(None, string_builder, int_builder);
8482        builder.keys().append_value("joe");
8483        builder.values().append_value(1);
8484        builder.append(true).unwrap();
8485
8486        builder.keys().append_value("blogs");
8487        builder.values().append_value(2);
8488        builder.keys().append_value("foo");
8489        builder.values().append_value(4);
8490        builder.append(true).unwrap();
8491        builder.append(true).unwrap();
8492        builder.append(false).unwrap();
8493
8494        let map_value = ScalarValue::Map(Arc::new(builder.finish()));
8495
8496        assert_eq!(map_value.to_string(), "[{joe:1},{blogs:2,foo:4},{},NULL]");
8497        assert_eq!(
8498            format!("{map_value:?}"),
8499            r#"Map([{"joe":"1"},{"blogs":"2","foo":"4"},{},NULL])"#
8500        );
8501
8502        let ScalarValue::Map(arr) = map_value else {
8503            panic!("Expected map");
8504        };
8505
8506        //verify compared to arrow display
8507        let batch = RecordBatch::try_from_iter(vec![("m", arr as _)]).unwrap();
8508        assert_snapshot!(batches_to_string(&[batch]), @r"
8509        +--------------------+
8510        | m                  |
8511        +--------------------+
8512        | {joe: 1}           |
8513        | {blogs: 2, foo: 4} |
8514        | {}                 |
8515        |                    |
8516        +--------------------+
8517        ");
8518    }
8519
8520    #[test]
8521    fn test_binary_display() {
8522        let no_binary_value = ScalarValue::Binary(None);
8523        assert_eq!(format!("{no_binary_value}"), "NULL");
8524        let single_binary_value = ScalarValue::Binary(Some(vec![42u8]));
8525        assert_eq!(format!("{single_binary_value}"), "2A");
8526        let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3]));
8527        assert_eq!(format!("{small_binary_value}"), "010203");
8528        let large_binary_value =
8529            ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8530        assert_eq!(format!("{large_binary_value}"), "0102030405060708090A...");
8531
8532        let no_binary_value = ScalarValue::BinaryView(None);
8533        assert_eq!(format!("{no_binary_value}"), "NULL");
8534        let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3]));
8535        assert_eq!(format!("{small_binary_value}"), "010203");
8536        let large_binary_value =
8537            ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8538        assert_eq!(format!("{large_binary_value}"), "0102030405060708090A...");
8539
8540        let no_binary_value = ScalarValue::LargeBinary(None);
8541        assert_eq!(format!("{no_binary_value}"), "NULL");
8542        let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3]));
8543        assert_eq!(format!("{small_binary_value}"), "010203");
8544        let large_binary_value =
8545            ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8546        assert_eq!(format!("{large_binary_value}"), "0102030405060708090A...");
8547
8548        let no_binary_value = ScalarValue::FixedSizeBinary(3, None);
8549        assert_eq!(format!("{no_binary_value}"), "NULL");
8550        let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3]));
8551        assert_eq!(format!("{small_binary_value}"), "010203");
8552        let large_binary_value = ScalarValue::FixedSizeBinary(
8553            11,
8554            Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]),
8555        );
8556        assert_eq!(format!("{large_binary_value}"), "0102030405060708090A...");
8557    }
8558
8559    #[test]
8560    fn test_binary_debug() {
8561        let no_binary_value = ScalarValue::Binary(None);
8562        assert_eq!(format!("{no_binary_value:?}"), "Binary(NULL)");
8563        let single_binary_value = ScalarValue::Binary(Some(vec![42u8]));
8564        assert_eq!(format!("{single_binary_value:?}"), "Binary(\"42\")");
8565        let small_binary_value = ScalarValue::Binary(Some(vec![1u8, 2, 3]));
8566        assert_eq!(format!("{small_binary_value:?}"), "Binary(\"1,2,3\")");
8567        let large_binary_value =
8568            ScalarValue::Binary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8569        assert_eq!(
8570            format!("{large_binary_value:?}"),
8571            "Binary(\"1,2,3,4,5,6,7,8,9,10,11\")"
8572        );
8573
8574        let no_binary_value = ScalarValue::BinaryView(None);
8575        assert_eq!(format!("{no_binary_value:?}"), "BinaryView(NULL)");
8576        let small_binary_value = ScalarValue::BinaryView(Some(vec![1u8, 2, 3]));
8577        assert_eq!(format!("{small_binary_value:?}"), "BinaryView(\"1,2,3\")");
8578        let large_binary_value =
8579            ScalarValue::BinaryView(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8580        assert_eq!(
8581            format!("{large_binary_value:?}"),
8582            "BinaryView(\"1,2,3,4,5,6,7,8,9,10,11\")"
8583        );
8584
8585        let no_binary_value = ScalarValue::LargeBinary(None);
8586        assert_eq!(format!("{no_binary_value:?}"), "LargeBinary(NULL)");
8587        let small_binary_value = ScalarValue::LargeBinary(Some(vec![1u8, 2, 3]));
8588        assert_eq!(format!("{small_binary_value:?}"), "LargeBinary(\"1,2,3\")");
8589        let large_binary_value =
8590            ScalarValue::LargeBinary(Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]));
8591        assert_eq!(
8592            format!("{large_binary_value:?}"),
8593            "LargeBinary(\"1,2,3,4,5,6,7,8,9,10,11\")"
8594        );
8595
8596        let no_binary_value = ScalarValue::FixedSizeBinary(3, None);
8597        assert_eq!(format!("{no_binary_value:?}"), "FixedSizeBinary(3, NULL)");
8598        let small_binary_value = ScalarValue::FixedSizeBinary(3, Some(vec![1u8, 2, 3]));
8599        assert_eq!(
8600            format!("{small_binary_value:?}"),
8601            "FixedSizeBinary(3, \"1,2,3\")"
8602        );
8603        let large_binary_value = ScalarValue::FixedSizeBinary(
8604            11,
8605            Some(vec![1u8, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11]),
8606        );
8607        assert_eq!(
8608            format!("{large_binary_value:?}"),
8609            "FixedSizeBinary(11, \"1,2,3,4,5,6,7,8,9,10,11\")"
8610        );
8611    }
8612
8613    #[test]
8614    fn test_build_timestamp_millisecond_list() {
8615        let values = vec![ScalarValue::TimestampMillisecond(Some(1), None)];
8616        let arr = ScalarValue::new_list_nullable(
8617            &values,
8618            &DataType::Timestamp(TimeUnit::Millisecond, None),
8619        );
8620        assert_eq!(1, arr.len());
8621    }
8622
8623    #[test]
8624    fn test_newlist_timestamp_zone() {
8625        let s: &'static str = "UTC";
8626        let values = vec![ScalarValue::TimestampMillisecond(Some(1), Some(s.into()))];
8627        let arr = ScalarValue::new_list_nullable(
8628            &values,
8629            &DataType::Timestamp(TimeUnit::Millisecond, Some(s.into())),
8630        );
8631        assert_eq!(1, arr.len());
8632        assert_eq!(
8633            arr.data_type(),
8634            &DataType::List(Arc::new(Field::new_list_field(
8635                DataType::Timestamp(TimeUnit::Millisecond, Some(s.into())),
8636                true,
8637            )))
8638        );
8639    }
8640
8641    fn get_random_timestamps(sample_size: u64) -> Vec<ScalarValue> {
8642        let vector_size = sample_size;
8643        let mut timestamp = vec![];
8644        let mut rng = rand::rng();
8645        for i in 0..vector_size {
8646            let year = rng.random_range(1995..=2050);
8647            let month = rng.random_range(1..=12);
8648            let day = rng.random_range(1..=28); // to exclude invalid dates
8649            let hour = rng.random_range(0..=23);
8650            let minute = rng.random_range(0..=59);
8651            let second = rng.random_range(0..=59);
8652            if i % 4 == 0 {
8653                timestamp.push(ScalarValue::TimestampSecond(
8654                    Some(
8655                        NaiveDate::from_ymd_opt(year, month, day)
8656                            .unwrap()
8657                            .and_hms_opt(hour, minute, second)
8658                            .unwrap()
8659                            .and_utc()
8660                            .timestamp(),
8661                    ),
8662                    None,
8663                ))
8664            } else if i % 4 == 1 {
8665                let millisec = rng.random_range(0..=999);
8666                timestamp.push(ScalarValue::TimestampMillisecond(
8667                    Some(
8668                        NaiveDate::from_ymd_opt(year, month, day)
8669                            .unwrap()
8670                            .and_hms_milli_opt(hour, minute, second, millisec)
8671                            .unwrap()
8672                            .and_utc()
8673                            .timestamp_millis(),
8674                    ),
8675                    None,
8676                ))
8677            } else if i % 4 == 2 {
8678                let microsec = rng.random_range(0..=999_999);
8679                timestamp.push(ScalarValue::TimestampMicrosecond(
8680                    Some(
8681                        NaiveDate::from_ymd_opt(year, month, day)
8682                            .unwrap()
8683                            .and_hms_micro_opt(hour, minute, second, microsec)
8684                            .unwrap()
8685                            .and_utc()
8686                            .timestamp_micros(),
8687                    ),
8688                    None,
8689                ))
8690            } else if i % 4 == 3 {
8691                let nanosec = rng.random_range(0..=999_999_999);
8692                timestamp.push(ScalarValue::TimestampNanosecond(
8693                    Some(
8694                        NaiveDate::from_ymd_opt(year, month, day)
8695                            .unwrap()
8696                            .and_hms_nano_opt(hour, minute, second, nanosec)
8697                            .unwrap()
8698                            .and_utc()
8699                            .timestamp_nanos_opt()
8700                            .unwrap(),
8701                    ),
8702                    None,
8703                ))
8704            }
8705        }
8706        timestamp
8707    }
8708
8709    fn get_random_intervals(sample_size: u64) -> Vec<ScalarValue> {
8710        const MILLISECS_IN_ONE_DAY: i64 = 86_400_000;
8711        const NANOSECS_IN_ONE_DAY: i64 = 86_400_000_000_000;
8712
8713        let vector_size = sample_size;
8714        let mut intervals = vec![];
8715        let mut rng = rand::rng();
8716        const SECS_IN_ONE_DAY: i32 = 86_400;
8717        const MICROSECS_IN_ONE_DAY: i64 = 86_400_000_000;
8718        for i in 0..vector_size {
8719            if i % 4 == 0 {
8720                let days = rng.random_range(0..5000);
8721                // to not break second precision
8722                let millis = rng.random_range(0..SECS_IN_ONE_DAY) * 1000;
8723                intervals.push(ScalarValue::new_interval_dt(days, millis));
8724            } else if i % 4 == 1 {
8725                let days = rng.random_range(0..5000);
8726                let millisec = rng.random_range(0..(MILLISECS_IN_ONE_DAY as i32));
8727                intervals.push(ScalarValue::new_interval_dt(days, millisec));
8728            } else if i % 4 == 2 {
8729                let days = rng.random_range(0..5000);
8730                // to not break microsec precision
8731                let nanosec = rng.random_range(0..MICROSECS_IN_ONE_DAY) * 1000;
8732                intervals.push(ScalarValue::new_interval_mdn(0, days, nanosec));
8733            } else {
8734                let days = rng.random_range(0..5000);
8735                let nanosec = rng.random_range(0..NANOSECS_IN_ONE_DAY);
8736                intervals.push(ScalarValue::new_interval_mdn(0, days, nanosec));
8737            }
8738        }
8739        intervals
8740    }
8741
8742    fn union_fields() -> UnionFields {
8743        [
8744            (0, Arc::new(Field::new("A", DataType::Int32, true))),
8745            (1, Arc::new(Field::new("B", DataType::Float64, true))),
8746        ]
8747        .into_iter()
8748        .collect()
8749    }
8750
8751    #[test]
8752    fn sparse_scalar_union_is_null() {
8753        let sparse_scalar = ScalarValue::Union(
8754            Some((0_i8, Box::new(ScalarValue::Int32(None)))),
8755            union_fields(),
8756            UnionMode::Sparse,
8757        );
8758        assert!(sparse_scalar.is_null());
8759    }
8760
8761    #[test]
8762    fn dense_scalar_union_is_null() {
8763        let dense_scalar = ScalarValue::Union(
8764            Some((0_i8, Box::new(ScalarValue::Int32(None)))),
8765            union_fields(),
8766            UnionMode::Dense,
8767        );
8768        assert!(dense_scalar.is_null());
8769    }
8770
8771    #[test]
8772    fn cast_date_to_timestamp_overflow_returns_error() {
8773        let scalar = ScalarValue::Date32(Some(i32::MAX));
8774        let err = scalar
8775            .cast_to(&DataType::Timestamp(TimeUnit::Nanosecond, None))
8776            .expect_err("expected cast to fail");
8777        assert!(
8778            err.to_string()
8779                .contains("converted value exceeds the representable i64 range"),
8780            "unexpected error: {err}"
8781        );
8782    }
8783
8784    #[test]
8785    fn null_dictionary_scalar_produces_null_dictionary_array() {
8786        let dictionary_scalar = ScalarValue::Dictionary(
8787            Box::new(DataType::Int32),
8788            Box::new(ScalarValue::Null),
8789        );
8790        assert!(dictionary_scalar.is_null());
8791        let dictionary_array = dictionary_scalar.to_array().unwrap();
8792        assert!(dictionary_array.is_null(0));
8793    }
8794
8795    #[test]
8796    fn test_scalar_value_try_new_null() {
8797        let scalars = vec![
8798            ScalarValue::try_new_null(&DataType::Boolean).unwrap(),
8799            ScalarValue::try_new_null(&DataType::Int8).unwrap(),
8800            ScalarValue::try_new_null(&DataType::Int16).unwrap(),
8801            ScalarValue::try_new_null(&DataType::Int32).unwrap(),
8802            ScalarValue::try_new_null(&DataType::Int64).unwrap(),
8803            ScalarValue::try_new_null(&DataType::UInt8).unwrap(),
8804            ScalarValue::try_new_null(&DataType::UInt16).unwrap(),
8805            ScalarValue::try_new_null(&DataType::UInt32).unwrap(),
8806            ScalarValue::try_new_null(&DataType::UInt64).unwrap(),
8807            ScalarValue::try_new_null(&DataType::Float16).unwrap(),
8808            ScalarValue::try_new_null(&DataType::Float32).unwrap(),
8809            ScalarValue::try_new_null(&DataType::Float64).unwrap(),
8810            ScalarValue::try_new_null(&DataType::Decimal128(42, 42)).unwrap(),
8811            ScalarValue::try_new_null(&DataType::Decimal256(42, 42)).unwrap(),
8812            ScalarValue::try_new_null(&DataType::Utf8).unwrap(),
8813            ScalarValue::try_new_null(&DataType::LargeUtf8).unwrap(),
8814            ScalarValue::try_new_null(&DataType::Utf8View).unwrap(),
8815            ScalarValue::try_new_null(&DataType::Binary).unwrap(),
8816            ScalarValue::try_new_null(&DataType::BinaryView).unwrap(),
8817            ScalarValue::try_new_null(&DataType::FixedSizeBinary(42)).unwrap(),
8818            ScalarValue::try_new_null(&DataType::LargeBinary).unwrap(),
8819            ScalarValue::try_new_null(&DataType::Date32).unwrap(),
8820            ScalarValue::try_new_null(&DataType::Date64).unwrap(),
8821            ScalarValue::try_new_null(&DataType::Time32(TimeUnit::Second)).unwrap(),
8822            ScalarValue::try_new_null(&DataType::Time32(TimeUnit::Millisecond)).unwrap(),
8823            ScalarValue::try_new_null(&DataType::Time64(TimeUnit::Microsecond)).unwrap(),
8824            ScalarValue::try_new_null(&DataType::Time64(TimeUnit::Nanosecond)).unwrap(),
8825            ScalarValue::try_new_null(&DataType::Timestamp(TimeUnit::Second, None))
8826                .unwrap(),
8827            ScalarValue::try_new_null(&DataType::Timestamp(TimeUnit::Millisecond, None))
8828                .unwrap(),
8829            ScalarValue::try_new_null(&DataType::Timestamp(TimeUnit::Microsecond, None))
8830                .unwrap(),
8831            ScalarValue::try_new_null(&DataType::Timestamp(TimeUnit::Nanosecond, None))
8832                .unwrap(),
8833            ScalarValue::try_new_null(&DataType::Interval(IntervalUnit::YearMonth))
8834                .unwrap(),
8835            ScalarValue::try_new_null(&DataType::Interval(IntervalUnit::DayTime))
8836                .unwrap(),
8837            ScalarValue::try_new_null(&DataType::Interval(IntervalUnit::MonthDayNano))
8838                .unwrap(),
8839            ScalarValue::try_new_null(&DataType::Duration(TimeUnit::Second)).unwrap(),
8840            ScalarValue::try_new_null(&DataType::Duration(TimeUnit::Microsecond))
8841                .unwrap(),
8842            ScalarValue::try_new_null(&DataType::Duration(TimeUnit::Nanosecond)).unwrap(),
8843            ScalarValue::try_new_null(&DataType::Null).unwrap(),
8844        ];
8845        assert!(scalars.iter().all(|s| s.is_null()));
8846
8847        let field_ref = Arc::new(Field::new("foo", DataType::Int32, true));
8848        let map_field_ref = Arc::new(Field::new(
8849            "foo",
8850            DataType::Struct(Fields::from(vec![
8851                Field::new("bar", DataType::Utf8, true),
8852                Field::new("baz", DataType::Int32, true),
8853            ])),
8854            true,
8855        ));
8856        let scalars = [
8857            ScalarValue::try_new_null(&DataType::List(Arc::clone(&field_ref))).unwrap(),
8858            ScalarValue::try_new_null(&DataType::LargeList(Arc::clone(&field_ref)))
8859                .unwrap(),
8860            ScalarValue::try_new_null(&DataType::FixedSizeList(
8861                Arc::clone(&field_ref),
8862                42,
8863            ))
8864            .unwrap(),
8865            ScalarValue::try_new_null(&DataType::Struct(
8866                vec![Arc::clone(&field_ref)].into(),
8867            ))
8868            .unwrap(),
8869            ScalarValue::try_new_null(&DataType::Map(map_field_ref, false)).unwrap(),
8870            ScalarValue::try_new_null(&DataType::Union(
8871                UnionFields::new(vec![42], vec![field_ref]),
8872                UnionMode::Dense,
8873            ))
8874            .unwrap(),
8875        ];
8876        assert!(scalars.iter().all(|s| s.is_null()));
8877    }
8878
8879    // `err.to_string()` depends on backtrace being present (may have backtrace appended)
8880    // `err.strip_backtrace()` also depends on backtrace being present (may have "This was likely caused by ..." stripped)
8881    fn assert_starts_with(actual: impl AsRef<str>, expected_prefix: impl AsRef<str>) {
8882        let actual = actual.as_ref();
8883        let expected_prefix = expected_prefix.as_ref();
8884        assert!(
8885            actual.starts_with(expected_prefix),
8886            "Expected '{actual}' to start with '{expected_prefix}'"
8887        );
8888    }
8889
8890    #[test]
8891    fn test_new_default() {
8892        // Test numeric types
8893        assert_eq!(
8894            ScalarValue::new_default(&DataType::Int32).unwrap(),
8895            ScalarValue::Int32(Some(0))
8896        );
8897        assert_eq!(
8898            ScalarValue::new_default(&DataType::Float64).unwrap(),
8899            ScalarValue::Float64(Some(0.0))
8900        );
8901        assert_eq!(
8902            ScalarValue::new_default(&DataType::Boolean).unwrap(),
8903            ScalarValue::Boolean(Some(false))
8904        );
8905
8906        // Test string types
8907        assert_eq!(
8908            ScalarValue::new_default(&DataType::Utf8).unwrap(),
8909            ScalarValue::Utf8(Some("".to_string()))
8910        );
8911        assert_eq!(
8912            ScalarValue::new_default(&DataType::LargeUtf8).unwrap(),
8913            ScalarValue::LargeUtf8(Some("".to_string()))
8914        );
8915
8916        // Test binary types
8917        assert_eq!(
8918            ScalarValue::new_default(&DataType::Binary).unwrap(),
8919            ScalarValue::Binary(Some(vec![]))
8920        );
8921
8922        // Test fixed size binary
8923        assert_eq!(
8924            ScalarValue::new_default(&DataType::FixedSizeBinary(5)).unwrap(),
8925            ScalarValue::FixedSizeBinary(5, Some(vec![0, 0, 0, 0, 0]))
8926        );
8927
8928        // Test temporal types
8929        assert_eq!(
8930            ScalarValue::new_default(&DataType::Date32).unwrap(),
8931            ScalarValue::Date32(Some(0))
8932        );
8933        assert_eq!(
8934            ScalarValue::new_default(&DataType::Time32(TimeUnit::Second)).unwrap(),
8935            ScalarValue::Time32Second(Some(0))
8936        );
8937
8938        // Test decimal types
8939        assert_eq!(
8940            ScalarValue::new_default(&DataType::Decimal128(10, 2)).unwrap(),
8941            ScalarValue::Decimal128(Some(0), 10, 2)
8942        );
8943
8944        // Test list type
8945        let list_field = Field::new_list_field(DataType::Int32, true);
8946        let list_result =
8947            ScalarValue::new_default(&DataType::List(Arc::new(list_field.clone())))
8948                .unwrap();
8949        match list_result {
8950            ScalarValue::List(arr) => {
8951                assert_eq!(arr.len(), 1);
8952                assert_eq!(arr.value_length(0), 0); // empty list
8953            }
8954            _ => panic!("Expected List"),
8955        }
8956
8957        // Test struct type
8958        let struct_fields = Fields::from(vec![
8959            Field::new("a", DataType::Int32, false),
8960            Field::new("b", DataType::Utf8, false),
8961        ]);
8962        let struct_result =
8963            ScalarValue::new_default(&DataType::Struct(struct_fields.clone())).unwrap();
8964        match struct_result {
8965            ScalarValue::Struct(arr) => {
8966                assert_eq!(arr.len(), 1);
8967                assert_eq!(arr.column(0).as_primitive::<Int32Type>().value(0), 0);
8968                assert_eq!(arr.column(1).as_string::<i32>().value(0), "");
8969            }
8970            _ => panic!("Expected Struct"),
8971        }
8972
8973        // Test union type
8974        let union_fields = UnionFields::new(
8975            vec![0, 1],
8976            vec![
8977                Field::new("i32", DataType::Int32, false),
8978                Field::new("f64", DataType::Float64, false),
8979            ],
8980        );
8981        let union_result = ScalarValue::new_default(&DataType::Union(
8982            union_fields.clone(),
8983            UnionMode::Sparse,
8984        ))
8985        .unwrap();
8986        match union_result {
8987            ScalarValue::Union(Some((type_id, value)), _, _) => {
8988                assert_eq!(type_id, 0);
8989                assert_eq!(*value, ScalarValue::Int32(Some(0)));
8990            }
8991            _ => panic!("Expected Union"),
8992        }
8993    }
8994
8995    #[test]
8996    fn test_scalar_min() {
8997        // Test integer types
8998        assert_eq!(
8999            ScalarValue::min(&DataType::Int8),
9000            Some(ScalarValue::Int8(Some(i8::MIN)))
9001        );
9002        assert_eq!(
9003            ScalarValue::min(&DataType::Int32),
9004            Some(ScalarValue::Int32(Some(i32::MIN)))
9005        );
9006        assert_eq!(
9007            ScalarValue::min(&DataType::UInt8),
9008            Some(ScalarValue::UInt8(Some(0)))
9009        );
9010        assert_eq!(
9011            ScalarValue::min(&DataType::UInt64),
9012            Some(ScalarValue::UInt64(Some(0)))
9013        );
9014
9015        // Test float types
9016        assert_eq!(
9017            ScalarValue::min(&DataType::Float32),
9018            Some(ScalarValue::Float32(Some(f32::NEG_INFINITY)))
9019        );
9020        assert_eq!(
9021            ScalarValue::min(&DataType::Float64),
9022            Some(ScalarValue::Float64(Some(f64::NEG_INFINITY)))
9023        );
9024
9025        // Test decimal types
9026        let decimal_min = ScalarValue::min(&DataType::Decimal128(5, 2)).unwrap();
9027        match decimal_min {
9028            ScalarValue::Decimal128(Some(val), 5, 2) => {
9029                assert_eq!(val, -99999); // -999.99 with scale 2
9030            }
9031            _ => panic!("Expected Decimal128"),
9032        }
9033
9034        // Test temporal types
9035        assert_eq!(
9036            ScalarValue::min(&DataType::Date32),
9037            Some(ScalarValue::Date32(Some(i32::MIN)))
9038        );
9039        assert_eq!(
9040            ScalarValue::min(&DataType::Time32(TimeUnit::Second)),
9041            Some(ScalarValue::Time32Second(Some(0)))
9042        );
9043        assert_eq!(
9044            ScalarValue::min(&DataType::Timestamp(TimeUnit::Nanosecond, None)),
9045            Some(ScalarValue::TimestampNanosecond(Some(i64::MIN), None))
9046        );
9047
9048        // Test duration types
9049        assert_eq!(
9050            ScalarValue::min(&DataType::Duration(TimeUnit::Second)),
9051            Some(ScalarValue::DurationSecond(Some(i64::MIN)))
9052        );
9053
9054        // Test unsupported types
9055        assert_eq!(ScalarValue::min(&DataType::Utf8), None);
9056        assert_eq!(ScalarValue::min(&DataType::Binary), None);
9057        assert_eq!(
9058            ScalarValue::min(&DataType::List(Arc::new(Field::new(
9059                "item",
9060                DataType::Int32,
9061                true
9062            )))),
9063            None
9064        );
9065    }
9066
9067    #[test]
9068    fn test_scalar_max() {
9069        // Test integer types
9070        assert_eq!(
9071            ScalarValue::max(&DataType::Int8),
9072            Some(ScalarValue::Int8(Some(i8::MAX)))
9073        );
9074        assert_eq!(
9075            ScalarValue::max(&DataType::Int32),
9076            Some(ScalarValue::Int32(Some(i32::MAX)))
9077        );
9078        assert_eq!(
9079            ScalarValue::max(&DataType::UInt8),
9080            Some(ScalarValue::UInt8(Some(u8::MAX)))
9081        );
9082        assert_eq!(
9083            ScalarValue::max(&DataType::UInt64),
9084            Some(ScalarValue::UInt64(Some(u64::MAX)))
9085        );
9086
9087        // Test float types
9088        assert_eq!(
9089            ScalarValue::max(&DataType::Float32),
9090            Some(ScalarValue::Float32(Some(f32::INFINITY)))
9091        );
9092        assert_eq!(
9093            ScalarValue::max(&DataType::Float64),
9094            Some(ScalarValue::Float64(Some(f64::INFINITY)))
9095        );
9096
9097        // Test decimal types
9098        let decimal_max = ScalarValue::max(&DataType::Decimal128(5, 2)).unwrap();
9099        match decimal_max {
9100            ScalarValue::Decimal128(Some(val), 5, 2) => {
9101                assert_eq!(val, 99999); // 999.99 with scale 2
9102            }
9103            _ => panic!("Expected Decimal128"),
9104        }
9105
9106        // Test temporal types
9107        assert_eq!(
9108            ScalarValue::max(&DataType::Date32),
9109            Some(ScalarValue::Date32(Some(i32::MAX)))
9110        );
9111        assert_eq!(
9112            ScalarValue::max(&DataType::Time32(TimeUnit::Second)),
9113            Some(ScalarValue::Time32Second(Some(86_399))) // 23:59:59
9114        );
9115        assert_eq!(
9116            ScalarValue::max(&DataType::Time64(TimeUnit::Microsecond)),
9117            Some(ScalarValue::Time64Microsecond(Some(86_399_999_999))) // 23:59:59.999999
9118        );
9119        assert_eq!(
9120            ScalarValue::max(&DataType::Timestamp(TimeUnit::Nanosecond, None)),
9121            Some(ScalarValue::TimestampNanosecond(Some(i64::MAX), None))
9122        );
9123
9124        // Test duration types
9125        assert_eq!(
9126            ScalarValue::max(&DataType::Duration(TimeUnit::Millisecond)),
9127            Some(ScalarValue::DurationMillisecond(Some(i64::MAX)))
9128        );
9129
9130        // Test unsupported types
9131        assert_eq!(ScalarValue::max(&DataType::Utf8), None);
9132        assert_eq!(ScalarValue::max(&DataType::Binary), None);
9133        assert_eq!(
9134            ScalarValue::max(&DataType::Struct(Fields::from(vec![Field::new(
9135                "field",
9136                DataType::Int32,
9137                true
9138            )]))),
9139            None
9140        );
9141    }
9142
9143    #[test]
9144    fn test_min_max_float16() {
9145        // Test Float16 min and max
9146        let min_f16 = ScalarValue::min(&DataType::Float16).unwrap();
9147        match min_f16 {
9148            ScalarValue::Float16(Some(val)) => {
9149                assert_eq!(val, f16::NEG_INFINITY);
9150            }
9151            _ => panic!("Expected Float16"),
9152        }
9153
9154        let max_f16 = ScalarValue::max(&DataType::Float16).unwrap();
9155        match max_f16 {
9156            ScalarValue::Float16(Some(val)) => {
9157                assert_eq!(val, f16::INFINITY);
9158            }
9159            _ => panic!("Expected Float16"),
9160        }
9161    }
9162
9163    #[test]
9164    fn test_new_default_interval() {
9165        // Test all interval types
9166        assert_eq!(
9167            ScalarValue::new_default(&DataType::Interval(IntervalUnit::YearMonth))
9168                .unwrap(),
9169            ScalarValue::IntervalYearMonth(Some(0))
9170        );
9171        assert_eq!(
9172            ScalarValue::new_default(&DataType::Interval(IntervalUnit::DayTime)).unwrap(),
9173            ScalarValue::IntervalDayTime(Some(IntervalDayTime::ZERO))
9174        );
9175        assert_eq!(
9176            ScalarValue::new_default(&DataType::Interval(IntervalUnit::MonthDayNano))
9177                .unwrap(),
9178            ScalarValue::IntervalMonthDayNano(Some(IntervalMonthDayNano::ZERO))
9179        );
9180    }
9181
9182    #[test]
9183    fn test_min_max_with_timezone() {
9184        let tz = Some(Arc::from("UTC"));
9185
9186        // Test timestamp with timezone
9187        let min_ts =
9188            ScalarValue::min(&DataType::Timestamp(TimeUnit::Second, tz.clone())).unwrap();
9189        match min_ts {
9190            ScalarValue::TimestampSecond(Some(val), Some(tz_str)) => {
9191                assert_eq!(val, i64::MIN);
9192                assert_eq!(tz_str.as_ref(), "UTC");
9193            }
9194            _ => panic!("Expected TimestampSecond with timezone"),
9195        }
9196
9197        let max_ts =
9198            ScalarValue::max(&DataType::Timestamp(TimeUnit::Millisecond, tz.clone()))
9199                .unwrap();
9200        match max_ts {
9201            ScalarValue::TimestampMillisecond(Some(val), Some(tz_str)) => {
9202                assert_eq!(val, i64::MAX);
9203                assert_eq!(tz_str.as_ref(), "UTC");
9204            }
9205            _ => panic!("Expected TimestampMillisecond with timezone"),
9206        }
9207    }
9208
9209    #[test]
9210    fn test_views_minimize_memory() {
9211        let value = "this string is longer than 12 bytes".to_string();
9212
9213        let scalar = ScalarValue::Utf8View(Some(value.clone()));
9214        let array = scalar.to_array_of_size(10).unwrap();
9215        let array = array.as_string_view();
9216        let buffers = array.data_buffers();
9217        assert_eq!(1, buffers.len());
9218        // Ensure we only have a single copy of the value string
9219        assert_eq!(value.len(), buffers[0].len());
9220
9221        // Same but for BinaryView
9222        let scalar = ScalarValue::BinaryView(Some(value.bytes().collect()));
9223        let array = scalar.to_array_of_size(10).unwrap();
9224        let array = array.as_binary_view();
9225        let buffers = array.data_buffers();
9226        assert_eq!(1, buffers.len());
9227        assert_eq!(value.len(), buffers[0].len());
9228    }
9229
9230    #[test]
9231    fn test_convert_array_to_scalar_vec() {
9232        // 1: Regular ListArray
9233        let list = ListArray::from_iter_primitive::<Int64Type, _, _>(vec![
9234            Some(vec![Some(1), Some(2)]),
9235            None,
9236            Some(vec![Some(3), None, Some(4)]),
9237        ]);
9238        let converted = ScalarValue::convert_array_to_scalar_vec(&list).unwrap();
9239        assert_eq!(
9240            converted,
9241            vec![
9242                Some(vec![
9243                    ScalarValue::Int64(Some(1)),
9244                    ScalarValue::Int64(Some(2))
9245                ]),
9246                None,
9247                Some(vec![
9248                    ScalarValue::Int64(Some(3)),
9249                    ScalarValue::Int64(None),
9250                    ScalarValue::Int64(Some(4))
9251                ]),
9252            ]
9253        );
9254
9255        // 2: Regular LargeListArray
9256        let large_list = LargeListArray::from_iter_primitive::<Int64Type, _, _>(vec![
9257            Some(vec![Some(1), Some(2)]),
9258            None,
9259            Some(vec![Some(3), None, Some(4)]),
9260        ]);
9261        let converted = ScalarValue::convert_array_to_scalar_vec(&large_list).unwrap();
9262        assert_eq!(
9263            converted,
9264            vec![
9265                Some(vec![
9266                    ScalarValue::Int64(Some(1)),
9267                    ScalarValue::Int64(Some(2))
9268                ]),
9269                None,
9270                Some(vec![
9271                    ScalarValue::Int64(Some(3)),
9272                    ScalarValue::Int64(None),
9273                    ScalarValue::Int64(Some(4))
9274                ]),
9275            ]
9276        );
9277
9278        // 3: Funky (null slot has non-zero list offsets)
9279        // Offsets + Values looks like this: [[1, 2], [3, 4], [5]]
9280        // But with NullBuffer it's like this: [[1, 2], NULL, [5]]
9281        let funky = ListArray::new(
9282            Field::new_list_field(DataType::Int64, true).into(),
9283            OffsetBuffer::new(vec![0, 2, 4, 5].into()),
9284            Arc::new(Int64Array::from(vec![1, 2, 3, 4, 5, 6])),
9285            Some(NullBuffer::from(vec![true, false, true])),
9286        );
9287        let converted = ScalarValue::convert_array_to_scalar_vec(&funky).unwrap();
9288        assert_eq!(
9289            converted,
9290            vec![
9291                Some(vec![
9292                    ScalarValue::Int64(Some(1)),
9293                    ScalarValue::Int64(Some(2))
9294                ]),
9295                None,
9296                Some(vec![ScalarValue::Int64(Some(5))]),
9297            ]
9298        );
9299
9300        // 4: Offsets + Values looks like this: [[1, 2], [], [5]]
9301        // But with NullBuffer it's like this: [[1, 2], NULL, [5]]
9302        // The converted result is: [[1, 2], None, [5]]
9303        let array4 = ListArray::new(
9304            Field::new_list_field(DataType::Int64, true).into(),
9305            OffsetBuffer::new(vec![0, 2, 2, 5].into()),
9306            Arc::new(Int64Array::from(vec![1, 2, 3, 4, 5, 6])),
9307            Some(NullBuffer::from(vec![true, false, true])),
9308        );
9309        let converted = ScalarValue::convert_array_to_scalar_vec(&array4).unwrap();
9310        assert_eq!(
9311            converted,
9312            vec![
9313                Some(vec![
9314                    ScalarValue::Int64(Some(1)),
9315                    ScalarValue::Int64(Some(2))
9316                ]),
9317                None,
9318                Some(vec![
9319                    ScalarValue::Int64(Some(3)),
9320                    ScalarValue::Int64(Some(4)),
9321                    ScalarValue::Int64(Some(5)),
9322                ]),
9323            ]
9324        );
9325
9326        // 5: Offsets + Values looks like this: [[1, 2], [], [5]]
9327        // Same as 4, but the middle array is not null, so after conversion it's empty.
9328        let array5 = ListArray::new(
9329            Field::new_list_field(DataType::Int64, true).into(),
9330            OffsetBuffer::new(vec![0, 2, 2, 5].into()),
9331            Arc::new(Int64Array::from(vec![1, 2, 3, 4, 5, 6])),
9332            Some(NullBuffer::from(vec![true, true, true])),
9333        );
9334        let converted = ScalarValue::convert_array_to_scalar_vec(&array5).unwrap();
9335        assert_eq!(
9336            converted,
9337            vec![
9338                Some(vec![
9339                    ScalarValue::Int64(Some(1)),
9340                    ScalarValue::Int64(Some(2))
9341                ]),
9342                Some(vec![]),
9343                Some(vec![
9344                    ScalarValue::Int64(Some(3)),
9345                    ScalarValue::Int64(Some(4)),
9346                    ScalarValue::Int64(Some(5)),
9347                ]),
9348            ]
9349        );
9350    }
9351}