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