liquid-cache 0.1.12

10x lower latency for cloud-native DataFusion
Documentation
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
use arrow::array::{
    ArrayRef, BooleanArray, PrimitiveArray,
    cast::AsArray,
    types::{
        TimestampMicrosecondType, TimestampMillisecondType, TimestampNanosecondType,
        TimestampSecondType,
    },
};
use arrow::buffer::{BooleanBuffer, ScalarBuffer};
use arrow::datatypes::{ArrowPrimitiveType, Date32Type, Int32Type, UInt32Type};
use arrow_schema::{DataType, TimeUnit};
use bytes::Bytes;
use num_traits::AsPrimitive;
use std::ops::Range;
use std::sync::Arc;

use super::LiquidArray;
use super::primitive_array::LiquidPrimitiveArray;
use super::{LiquidDataType, LiquidSqueezedArray};
use crate::liquid_array::LiquidPrimitiveType;
use crate::liquid_array::SqueezeIoHandler;
use crate::liquid_array::raw::BitPackedArray;
use crate::utils::get_bit_width;

/// Which component to extract from a `Date32`/Timestamp (days since UNIX epoch).
#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, serde::Serialize)]
pub enum Date32Field {
    /// Year component
    Year,
    /// Month component
    Month,
    /// Day component
    Day,
    /// Day-of-week component (Sunday=0).
    DayOfWeek,
}

/// A bit-packed array that stores a single extracted component (YEAR/MONTH/DAY/DOW)
/// from a `Date32`/Timestamp array.
///
/// Values are stored as unsigned offsets from `reference_value`, using the same
/// bit-packing machinery as primitive arrays.
#[derive(Debug, Clone)]
pub struct SqueezedDate32Array {
    field: Date32Field,
    bit_packed: BitPackedArray<UInt32Type>,
    /// The minimum extracted value used as reference for offsetting.
    reference_value: i32,
    original_data_type: DataType,
    backing: Option<SqueezedBacking>,
}

#[derive(Debug, Clone)]
struct SqueezedBacking {
    io: Arc<dyn SqueezeIoHandler>,
    disk_range: Range<u64>,
}

impl SqueezedDate32Array {
    /// Build a squeezed representation (YEAR/MONTH/DAY/DAYOFWEEK) from a `Date32` array.
    pub fn from_liquid_date32<T: LiquidPrimitiveType>(
        array: &LiquidPrimitiveArray<T>,
        field: Date32Field,
    ) -> Self {
        // Decode the logical Date32 array (i32: days since epoch) from the liquid array.
        let arrow_array: PrimitiveArray<Date32Type> =
            array.to_arrow_array().as_primitive::<Date32Type>().clone();

        let (_dt, values, nulls) = arrow_array.into_parts();

        // Compute min and max for the extracted component, skipping nulls.
        let mut has_value = false;
        let mut min_component: i32 = i32::MAX;
        let mut max_component: i32 = i32::MIN;

        // Fast path: if all nulls, return a null bit-packed array of the same length.
        if let Some(nulls_buf) = &nulls
            && nulls_buf.null_count() == values.len()
        {
            return Self {
                field,
                bit_packed: BitPackedArray::new_null_array(values.len()),
                reference_value: 0,
                original_data_type: DataType::Date32,
                backing: None,
            };
        }

        for (idx, &days) in values.iter().enumerate() {
            if let Some(nulls_buf) = &nulls
                && nulls_buf.is_null(idx)
            {
                continue;
            }
            let comp = component_from_days(field, days);
            has_value = true;
            if comp < min_component {
                min_component = comp;
            }
            if comp > max_component {
                max_component = comp;
            }
        }

        // If no non-null values found, return an all-null structure (defensive)
        if !has_value {
            return Self {
                field,
                bit_packed: BitPackedArray::new_null_array(values.len()),
                reference_value: 0,
                original_data_type: DataType::Date32,
                backing: None,
            };
        }

        // Compute bit width from the value range.
        let max_offset = (max_component as i64 - min_component as i64) as u64;
        let bit_width = get_bit_width(max_offset);

        // Build unsigned offsets for packing; placeholders are fine for nulls.
        let offsets: ScalarBuffer<<UInt32Type as ArrowPrimitiveType>::Native> =
            ScalarBuffer::from_iter((0..values.len()).map(|idx| {
                if nulls.as_ref().is_some_and(|n| n.is_null(idx)) {
                    0u32
                } else {
                    let comp = component_from_days(field, values[idx]);
                    (comp - min_component) as u32
                }
            }));

        let unsigned_array = PrimitiveArray::<UInt32Type>::new(offsets, nulls);
        let bit_packed = BitPackedArray::from_primitive(unsigned_array, bit_width);

        Self {
            field,
            bit_packed,
            reference_value: min_component,
            original_data_type: DataType::Date32,
            backing: None,
        }
    }

    /// Build a squeezed representation (YEAR/MONTH/DAY/DAYOFWEEK) from a timestamp array.
    pub fn from_liquid_timestamp<T: LiquidPrimitiveType>(
        array: &LiquidPrimitiveArray<T>,
        field: Date32Field,
    ) -> Self {
        let unit = timestamp_unit(&T::DATA_TYPE).expect("timestamp data type");
        let arrow_array: PrimitiveArray<T> = array.to_arrow_array().as_primitive::<T>().clone();
        let (_dt, values, nulls) = arrow_array.into_parts();

        let mut has_value = false;
        let mut min_component: i32 = i32::MAX;
        let mut max_component: i32 = i32::MIN;

        if let Some(nulls_buf) = &nulls
            && nulls_buf.null_count() == values.len()
        {
            return Self {
                field,
                bit_packed: BitPackedArray::new_null_array(values.len()),
                reference_value: 0,
                original_data_type: T::DATA_TYPE.clone(),
                backing: None,
            };
        }

        for (idx, &value) in values.iter().enumerate() {
            if let Some(nulls_buf) = &nulls
                && nulls_buf.is_null(idx)
            {
                continue;
            }
            let days = timestamp_to_days_since_epoch(value.as_(), unit);
            let comp = component_from_days(field, days);
            has_value = true;
            if comp < min_component {
                min_component = comp;
            }
            if comp > max_component {
                max_component = comp;
            }
        }

        if !has_value {
            return Self {
                field,
                bit_packed: BitPackedArray::new_null_array(values.len()),
                reference_value: 0,
                original_data_type: T::DATA_TYPE.clone(),
                backing: None,
            };
        }

        let max_offset = (max_component as i64 - min_component as i64) as u64;
        let bit_width = get_bit_width(max_offset);

        let offsets: ScalarBuffer<<UInt32Type as ArrowPrimitiveType>::Native> =
            ScalarBuffer::from_iter((0..values.len()).map(|idx| {
                if nulls.as_ref().is_some_and(|n| n.is_null(idx)) {
                    0u32
                } else {
                    let days = timestamp_to_days_since_epoch(values[idx].as_(), unit);
                    let comp = component_from_days(field, days);
                    (comp - min_component) as u32
                }
            }));

        let unsigned_array = PrimitiveArray::<UInt32Type>::new(offsets, nulls);
        let bit_packed = BitPackedArray::from_primitive(unsigned_array, bit_width);

        Self {
            field,
            bit_packed,
            reference_value: min_component,
            original_data_type: T::DATA_TYPE.clone(),
            backing: None,
        }
    }

    pub(crate) fn with_backing(
        mut self,
        io: Arc<dyn SqueezeIoHandler>,
        disk_range: Range<u64>,
    ) -> Self {
        self.backing = Some(SqueezedBacking { io, disk_range });
        self
    }

    async fn read_backing(&self) -> Bytes {
        let backing = self
            .backing
            .as_ref()
            .expect("SqueezedDate32Array backing not set");
        backing
            .io
            .read(Some(backing.disk_range.clone()))
            .await
            .expect("read squeezed backing")
    }

    /// Length of the array.
    pub fn len(&self) -> usize {
        self.bit_packed.len()
    }

    /// Whether the array has no elements.
    pub fn is_empty(&self) -> bool {
        self.len() == 0
    }

    /// Memory size of the bit-packed representation plus reference value.
    pub fn get_array_memory_size(&self) -> usize {
        self.bit_packed.get_array_memory_size() + std::mem::size_of::<i32>()
    }

    /// The extracted component type.
    pub fn field(&self) -> Date32Field {
        self.field
    }

    /// Convert to an Arrow array holding the extracted component.
    pub fn to_component_array(&self) -> ArrayRef {
        match &self.original_data_type {
            DataType::Date32 => Arc::new(self.to_component_date32()) as ArrayRef,
            DataType::Timestamp(unit, _) => self.to_component_timestamp(*unit),
            _ => Arc::new(self.to_component_date32()) as ArrayRef,
        }
    }

    /// Convert back to an Arrow `Int32` array representing the extracted component values.
    /// Useful for verification or future pushdown logic.
    pub fn to_component_date32(&self) -> PrimitiveArray<Date32Type> {
        let unsigned: PrimitiveArray<UInt32Type> = self.bit_packed.to_primitive();
        let (_dt, values, nulls) = unsigned.into_parts();
        let ref_v = self.reference_value;
        let signed_values: ScalarBuffer<<Int32Type as ArrowPrimitiveType>::Native> =
            ScalarBuffer::from_iter(values.iter().map(|&v| (v as i32).saturating_add(ref_v)));
        PrimitiveArray::<Date32Type>::new(signed_values, nulls)
    }

    fn to_component_timestamp(&self, unit: TimeUnit) -> ArrayRef {
        let unsigned: PrimitiveArray<UInt32Type> = self.bit_packed.to_primitive();
        let (_dt, values, nulls) = unsigned.into_parts();
        let ref_v = self.reference_value;
        let signed_values: ScalarBuffer<i64> =
            ScalarBuffer::from_iter(values.iter().map(|&v| (v as i32 + ref_v) as i64));

        match unit {
            TimeUnit::Second => Arc::new(PrimitiveArray::<TimestampSecondType>::new(
                signed_values,
                nulls,
            )),
            TimeUnit::Millisecond => Arc::new(PrimitiveArray::<TimestampMillisecondType>::new(
                signed_values.clone(),
                nulls,
            )),
            TimeUnit::Microsecond => Arc::new(PrimitiveArray::<TimestampMicrosecondType>::new(
                signed_values.clone(),
                nulls,
            )),
            TimeUnit::Nanosecond => Arc::new(PrimitiveArray::<TimestampNanosecondType>::new(
                signed_values,
                nulls,
            )),
        }
    }

    /// Lossy reconstruction to Arrow `Date32` (days since epoch).
    ///
    /// Mapping used:
    /// - Year: (year, 1, 1)
    /// - Month: (1970, month, 1)
    /// - Day: (1970, 1, day)
    /// - DayOfWeek: (1970, 1, 4 + dow) where 1970-01-04 is Sunday
    pub fn to_arrow_date32_lossy(&self) -> PrimitiveArray<Date32Type> {
        let unsigned: PrimitiveArray<UInt32Type> = self.bit_packed.to_primitive();
        let (_dt, values, nulls) = unsigned.into_parts();

        let ref_v = self.reference_value;
        let days_values: ScalarBuffer<<Date32Type as ArrowPrimitiveType>::Native> =
            ScalarBuffer::from_iter(values.iter().enumerate().map(|(i, &off)| {
                if nulls.as_ref().is_some_and(|n| n.is_null(i)) {
                    0i32
                } else {
                    match self.field {
                        Date32Field::Year => {
                            let y = ref_v + off as i32;
                            ymd_to_epoch_days(y, 1, 1)
                        }
                        Date32Field::Month => {
                            let m = (ref_v + off as i32) as u32;
                            ymd_to_epoch_days(1970, m, 1)
                        }
                        Date32Field::Day => {
                            let d = (ref_v + off as i32) as u32;
                            ymd_to_epoch_days(1970, 1, d)
                        }
                        Date32Field::DayOfWeek => {
                            let dow = ref_v + off as i32;
                            ymd_to_epoch_days(1970, 1, 4).saturating_add(dow)
                        }
                    }
                }
            }));

        PrimitiveArray::<Date32Type>::new(days_values, nulls)
    }
}

/// Convert days since UNIX epoch (1970-01-01) to (year, month, day) in the
/// proleptic Gregorian calendar using a branchless integer algorithm.
fn ymd_from_epoch_days(days_since_epoch: i32) -> (i32, u32, u32) {
    // Based on Howard Hinnant's civil_from_days algorithm.
    let z = days_since_epoch as i64 + 719_468; // shift to civil epoch
    let era = if z >= 0 {
        z / 146_097
    } else {
        (z - 146_096) / 146_097
    };
    let doe = z - era * 146_097; // [0, 146096]
    let yoe = (doe - doe / 1_460 + doe / 36_524 - doe / 146_096) / 365; // [0, 399]
    let mut y = yoe + era * 400;
    let doy = doe - (365 * yoe + yoe / 4 - yoe / 100); // [0, 365]
    let mp = (5 * doy + 2) / 153; // [0, 11]
    let d = (doy - (153 * mp + 2) / 5) + 1; // [1, 31]
    let m = mp + if mp < 10 { 3 } else { -9 }; // [1, 12]
    if m <= 2 {
        y += 1;
    }
    (y as i32, m as u32, d as u32)
}

fn component_from_days(field: Date32Field, days: i32) -> i32 {
    let (year, month, day) = ymd_from_epoch_days(days);
    match field {
        Date32Field::Year => year,
        Date32Field::Month => month as i32,
        Date32Field::Day => day as i32,
        Date32Field::DayOfWeek => day_of_week_sunday0(days),
    }
}

fn day_of_week_sunday0(days_since_epoch: i32) -> i32 {
    (days_since_epoch + 4).rem_euclid(7)
}

fn timestamp_unit(data_type: &DataType) -> Option<TimeUnit> {
    match data_type {
        DataType::Timestamp(unit, _) => Some(*unit),
        _ => None,
    }
}

fn timestamp_to_days_since_epoch(value: i64, unit: TimeUnit) -> i32 {
    let ticks_per_day = match unit {
        TimeUnit::Second => 86_400,
        TimeUnit::Millisecond => 86_400_000,
        TimeUnit::Microsecond => 86_400_000_000,
        TimeUnit::Nanosecond => 86_400_000_000_000,
    };
    (value.div_euclid(ticks_per_day)) as i32
}

/// Convert a date (year, month, day) in proleptic Gregorian calendar to
/// days since UNIX epoch (1970-01-01).
fn ymd_to_epoch_days(year: i32, month: u32, day: u32) -> i32 {
    // Based on Howard Hinnant's civil_to_days algorithm.
    let y = year as i64 - if month <= 2 { 1 } else { 0 };
    let era = if y >= 0 { y / 400 } else { (y - 399) / 400 };
    let yoe = y - era * 400; // [0, 399]
    let m = month as i64;
    let d = day as i64;
    let mp = m + if m > 2 { -3 } else { 9 }; // Mar=0..Jan=10,Feb=11
    let doy = (153 * mp + 2) / 5 + d - 1; // [0, 365]
    let doe = yoe * 365 + yoe / 4 - yoe / 100 + doy; // [0, 146096]
    (era * 146_097 + doe - 719_468) as i32
}

#[async_trait::async_trait]
impl LiquidSqueezedArray for SqueezedDate32Array {
    fn as_any(&self) -> &dyn std::any::Any {
        self
    }

    fn get_array_memory_size(&self) -> usize {
        self.get_array_memory_size()
    }

    fn len(&self) -> usize {
        self.len()
    }

    async fn to_arrow_array(&self) -> ArrayRef {
        let bytes = self.read_backing().await;
        let liquid = crate::liquid_array::ipc::read_from_bytes(
            bytes,
            &crate::liquid_array::ipc::LiquidIPCContext::new(None),
        );
        liquid.to_arrow_array()
    }

    fn data_type(&self) -> LiquidDataType {
        LiquidDataType::Integer
    }

    fn original_arrow_data_type(&self) -> DataType {
        self.original_data_type.clone()
    }

    async fn filter(&self, selection: &BooleanBuffer) -> ArrayRef {
        if selection.count_set_bits() == 0 {
            return arrow::array::new_empty_array(&self.original_arrow_data_type());
        }
        let full = self.to_arrow_array().await;
        let selection_array = BooleanArray::new(selection.clone(), None);
        arrow::compute::filter(&full, &selection_array).unwrap()
    }

    async fn try_eval_predicate(
        &self,
        _predicate: &Arc<dyn datafusion::physical_plan::PhysicalExpr>,
        _filter: &BooleanBuffer,
    ) -> Option<BooleanArray> {
        None
    }
}

#[cfg(test)]
mod tests {
    use super::*;
    use arrow::array::types::TimestampMicrosecondType;
    use arrow::array::{Array, PrimitiveArray};
    use std::sync::Arc;

    fn dates(vals: &[Option<i32>]) -> PrimitiveArray<Date32Type> {
        PrimitiveArray::<Date32Type>::from(vals.to_vec())
    }

    fn assert_prim_eq<T: ArrowPrimitiveType>(a: PrimitiveArray<T>, b: PrimitiveArray<T>) {
        let a_ref: arrow::array::ArrayRef = Arc::new(a);
        let b_ref: arrow::array::ArrayRef = Arc::new(b);
        assert_eq!(a_ref.as_ref(), b_ref.as_ref());
    }

    fn extract(field: Date32Field, input: Vec<Option<i32>>) -> PrimitiveArray<Date32Type> {
        let arr = dates(&input);
        let liquid = LiquidPrimitiveArray::<Date32Type>::from_arrow_array(arr);
        let squeezed = SqueezedDate32Array::from_liquid_date32(&liquid, field);
        squeezed.to_component_date32()
    }

    fn lossy(field: Date32Field, input: Vec<Option<i32>>) -> PrimitiveArray<Date32Type> {
        let arr = dates(&input);
        let liquid = LiquidPrimitiveArray::<Date32Type>::from_arrow_array(arr);
        let squeezed = SqueezedDate32Array::from_liquid_date32(&liquid, field);
        squeezed.to_arrow_date32_lossy()
    }

    #[test]
    fn test_extraction_correctness() {
        // YEAR
        let input = vec![
            Some(-1),
            Some(0),
            Some(ymd_to_epoch_days(1971, 7, 15)),
            None,
        ];
        let expected =
            PrimitiveArray::<Date32Type>::from(vec![Some(1969), Some(1970), Some(1971), None]);
        assert_prim_eq(extract(Date32Field::Year, input), expected);

        // MONTH
        let input = vec![
            Some(ymd_to_epoch_days(1970, 1, 31)),
            Some(ymd_to_epoch_days(1970, 2, 1)),
            Some(ymd_to_epoch_days(1970, 12, 31)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![Some(1), Some(2), Some(12), None]);
        assert_prim_eq(extract(Date32Field::Month, input), expected);

        // DAY
        let input = vec![
            Some(ymd_to_epoch_days(1970, 1, 1)),
            Some(ymd_to_epoch_days(1970, 1, 31)),
            Some(ymd_to_epoch_days(1970, 2, 1)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![Some(1), Some(31), Some(1), None]);
        assert_prim_eq(extract(Date32Field::Day, input), expected);

        // DAYOFWEEK (Sunday=0)
        let input = vec![
            Some(ymd_to_epoch_days(1970, 1, 4)),
            Some(ymd_to_epoch_days(1970, 1, 5)),
            Some(ymd_to_epoch_days(1970, 1, 10)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![Some(0), Some(1), Some(6), None]);
        assert_prim_eq(extract(Date32Field::DayOfWeek, input), expected);
    }

    #[test]
    fn test_lossy_reconstruction_mapping() {
        // YEAR → (y,1,1)
        let input = vec![
            Some(ymd_to_epoch_days(1999, 12, 31)),
            Some(ymd_to_epoch_days(2000, 6, 1)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![
            Some(ymd_to_epoch_days(1999, 1, 1)),
            Some(ymd_to_epoch_days(2000, 1, 1)),
            None,
        ]);
        assert_prim_eq(lossy(Date32Field::Year, input), expected);

        // MONTH → (1970,m,1)
        let input = vec![
            Some(ymd_to_epoch_days(1980, 3, 14)),
            Some(ymd_to_epoch_days(1977, 12, 5)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![
            Some(ymd_to_epoch_days(1970, 3, 1)),
            Some(ymd_to_epoch_days(1970, 12, 1)),
            None,
        ]);
        assert_prim_eq(lossy(Date32Field::Month, input), expected);

        // DAY → (1970,1,d)
        let input = vec![
            Some(ymd_to_epoch_days(1980, 3, 14)),
            Some(ymd_to_epoch_days(1977, 12, 5)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![
            Some(ymd_to_epoch_days(1970, 1, 14)),
            Some(ymd_to_epoch_days(1970, 1, 5)),
            None,
        ]);
        assert_prim_eq(lossy(Date32Field::Day, input), expected);

        // DAYOFWEEK → (1970,1,4 + dow)
        let input = vec![
            Some(ymd_to_epoch_days(2020, 5, 17)),
            Some(ymd_to_epoch_days(2020, 5, 18)),
            None,
        ];
        let expected = PrimitiveArray::<Date32Type>::from(vec![
            Some(ymd_to_epoch_days(1970, 1, 4)),
            Some(ymd_to_epoch_days(1970, 1, 5)),
            None,
        ]);
        assert_prim_eq(lossy(Date32Field::DayOfWeek, input), expected);
    }

    #[test]
    fn test_roundtrip_idempotence() {
        let input = vec![
            Some(ymd_to_epoch_days(1969, 12, 31)),
            Some(ymd_to_epoch_days(1970, 1, 1)),
            Some(ymd_to_epoch_days(1970, 1, 31)),
            Some(ymd_to_epoch_days(1970, 2, 1)),
            Some(ymd_to_epoch_days(1971, 7, 15)),
            None,
        ];

        for &field in &[
            Date32Field::Year,
            Date32Field::Month,
            Date32Field::Day,
            Date32Field::DayOfWeek,
        ] {
            let comp1 = extract(field, input.clone());
            let lossy_dt = lossy(field, input.clone());
            let liquid2 = LiquidPrimitiveArray::<Date32Type>::from_arrow_array(lossy_dt);
            let comp2 =
                SqueezedDate32Array::from_liquid_date32(&liquid2, field).to_component_date32();
            assert_prim_eq(comp1, comp2);
        }
    }

    #[test]
    fn test_all_nulls_behavior() {
        let input = vec![None, None, None];

        for &field in &[
            Date32Field::Year,
            Date32Field::Month,
            Date32Field::Day,
            Date32Field::DayOfWeek,
        ] {
            let comp = extract(field, input.clone());
            let expected_comp = PrimitiveArray::<Date32Type>::from(vec![None, None, None]);
            assert_prim_eq(comp, expected_comp);

            let lossy_dt = lossy(field, input.clone());
            let expected_dt = PrimitiveArray::<Date32Type>::from(vec![None, None, None]);
            assert_prim_eq(lossy_dt, expected_dt);
        }
    }

    #[test]
    fn test_timestamp_extraction() {
        let input = vec![
            Some(1_609_459_200_000_000),
            Some(1_640_995_200_000_000),
            None,
        ];
        let arr = PrimitiveArray::<TimestampMicrosecondType>::from(input);
        let liquid = LiquidPrimitiveArray::<TimestampMicrosecondType>::from_arrow_array(arr);
        let squeezed = SqueezedDate32Array::from_liquid_timestamp(&liquid, Date32Field::Year);
        let component = squeezed.to_component_array();
        let out = component
            .as_any()
            .downcast_ref::<PrimitiveArray<TimestampMicrosecondType>>()
            .expect("timestamp array");

        assert_eq!(out.value(0), 2021);
        assert_eq!(out.value(1), 2022);
        assert!(out.is_null(2));
    }
}