Skip to main content

vortex_array/arrays/shared/
vtable.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright the Vortex contributors
3
4use std::hash::Hash;
5
6use vortex_dtype::DType;
7use vortex_error::VortexExpect;
8use vortex_error::VortexResult;
9use vortex_session::VortexSession;
10
11use crate::ArrayBufferVisitor;
12use crate::ArrayChildVisitor;
13use crate::ArrayRef;
14use crate::Canonical;
15use crate::EmptyMetadata;
16use crate::ExecutionCtx;
17use crate::IntoArray;
18use crate::Precision;
19use crate::arrays::shared::SharedArray;
20use crate::buffer::BufferHandle;
21use crate::hash::ArrayEq;
22use crate::hash::ArrayHash;
23use crate::scalar::Scalar;
24use crate::stats::StatsSetRef;
25use crate::validity::Validity;
26use crate::vtable;
27use crate::vtable::ArrayId;
28use crate::vtable::BaseArrayVTable;
29use crate::vtable::OperationsVTable;
30use crate::vtable::VTable;
31use crate::vtable::ValidityVTable;
32use crate::vtable::VisitorVTable;
33
34vtable!(Shared);
35
36// TODO(ngates): consider hooking Shared into the iterative execution model. Cache either the
37//  most executed, or after each iteration, and return a shared cache for each execution.
38#[derive(Debug)]
39pub struct SharedVTable;
40
41impl SharedVTable {
42    pub const ID: ArrayId = ArrayId::new_ref("vortex.shared");
43}
44
45impl VTable for SharedVTable {
46    type Array = SharedArray;
47    type Metadata = EmptyMetadata;
48
49    type ArrayVTable = Self;
50    type OperationsVTable = Self;
51    type ValidityVTable = Self;
52    type VisitorVTable = Self;
53
54    fn id(_array: &Self::Array) -> ArrayId {
55        Self::ID
56    }
57
58    fn metadata(_array: &Self::Array) -> VortexResult<Self::Metadata> {
59        Ok(EmptyMetadata)
60    }
61
62    fn serialize(_metadata: Self::Metadata) -> VortexResult<Option<Vec<u8>>> {
63        vortex_error::vortex_bail!("Shared array is not serializable")
64    }
65
66    fn deserialize(
67        _bytes: &[u8],
68        _dtype: &DType,
69        _len: usize,
70        _buffers: &[BufferHandle],
71        _session: &VortexSession,
72    ) -> VortexResult<Self::Metadata> {
73        vortex_error::vortex_bail!("Shared array is not serializable")
74    }
75
76    fn build(
77        dtype: &DType,
78        len: usize,
79        _metadata: &Self::Metadata,
80        _buffers: &[BufferHandle],
81        children: &dyn crate::serde::ArrayChildren,
82    ) -> VortexResult<SharedArray> {
83        let child = children.get(0, dtype, len)?;
84        Ok(SharedArray::new(child))
85    }
86
87    fn with_children(array: &mut Self::Array, children: Vec<ArrayRef>) -> VortexResult<()> {
88        vortex_error::vortex_ensure!(
89            children.len() == 1,
90            "SharedArray expects exactly 1 child, got {}",
91            children.len()
92        );
93        let child = children
94            .into_iter()
95            .next()
96            .vortex_expect("children length already validated");
97        array.set_source(child);
98        Ok(())
99    }
100
101    fn execute(array: &Self::Array, ctx: &mut ExecutionCtx) -> VortexResult<ArrayRef> {
102        Ok(array
103            .get_or_compute(|source| source.clone().execute::<Canonical>(ctx))?
104            .into_array())
105    }
106}
107
108impl BaseArrayVTable<SharedVTable> for SharedVTable {
109    fn len(array: &SharedArray) -> usize {
110        array.current_array_ref().len()
111    }
112
113    fn dtype(array: &SharedArray) -> &DType {
114        &array.dtype
115    }
116
117    fn stats(array: &SharedArray) -> StatsSetRef<'_> {
118        array.stats.to_ref(array.as_ref())
119    }
120
121    fn array_hash<H: std::hash::Hasher>(array: &SharedArray, state: &mut H, precision: Precision) {
122        let current = array.current_array_ref();
123        current.array_hash(state, precision);
124        array.dtype.hash(state);
125    }
126
127    fn array_eq(array: &SharedArray, other: &SharedArray, precision: Precision) -> bool {
128        let current = array.current_array_ref();
129        let other_current = other.current_array_ref();
130        current.array_eq(&other_current, precision) && array.dtype == other.dtype
131    }
132}
133
134impl OperationsVTable<SharedVTable> for SharedVTable {
135    fn scalar_at(array: &SharedArray, index: usize) -> VortexResult<Scalar> {
136        array.current_array_ref().scalar_at(index)
137    }
138}
139
140impl ValidityVTable<SharedVTable> for SharedVTable {
141    fn validity(array: &SharedArray) -> VortexResult<Validity> {
142        array.current_array_ref().validity()
143    }
144}
145
146impl VisitorVTable<SharedVTable> for SharedVTable {
147    fn visit_buffers(_array: &SharedArray, _visitor: &mut dyn ArrayBufferVisitor) {}
148
149    fn visit_children(array: &SharedArray, visitor: &mut dyn ArrayChildVisitor) {
150        visitor.visit_child("source", &array.current_array_ref());
151    }
152
153    fn nchildren(_array: &SharedArray) -> usize {
154        1
155    }
156
157    fn nth_child(array: &SharedArray, idx: usize) -> Option<ArrayRef> {
158        match idx {
159            0 => Some(array.current_array_ref()),
160            _ => None,
161        }
162    }
163}