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