vortex_layout/layouts/chunked/
mod.rs

1// SPDX-License-Identifier: Apache-2.0
2// SPDX-FileCopyrightText: Copyright the Vortex contributors
3
4mod reader;
5pub mod writer;
6
7use std::sync::Arc;
8
9use vortex_array::ArrayContext;
10use vortex_array::DeserializeMetadata;
11use vortex_array::EmptyMetadata;
12use vortex_dtype::DType;
13use vortex_error::VortexResult;
14use vortex_session::VortexSession;
15
16use crate::LayoutChildType;
17use crate::LayoutEncodingRef;
18use crate::LayoutId;
19use crate::LayoutReaderRef;
20use crate::LayoutRef;
21use crate::VTable;
22use crate::children::LayoutChildren;
23use crate::children::OwnedLayoutChildren;
24use crate::layouts::chunked::reader::ChunkedReader;
25use crate::segments::SegmentId;
26use crate::segments::SegmentSource;
27use crate::vtable;
28
29vtable!(Chunked);
30
31impl VTable for ChunkedVTable {
32    type Layout = ChunkedLayout;
33    type Encoding = ChunkedLayoutEncoding;
34    type Metadata = EmptyMetadata;
35
36    fn id(_encoding: &Self::Encoding) -> LayoutId {
37        LayoutId::new_ref("vortex.chunked")
38    }
39
40    fn encoding(_layout: &Self::Layout) -> LayoutEncodingRef {
41        LayoutEncodingRef::new_ref(ChunkedLayoutEncoding.as_ref())
42    }
43
44    fn row_count(layout: &Self::Layout) -> u64 {
45        layout.row_count
46    }
47
48    fn dtype(layout: &Self::Layout) -> &DType {
49        &layout.dtype
50    }
51
52    fn metadata(_layout: &Self::Layout) -> Self::Metadata {
53        EmptyMetadata
54    }
55
56    fn segment_ids(_layout: &Self::Layout) -> Vec<SegmentId> {
57        vec![]
58    }
59
60    fn nchildren(layout: &Self::Layout) -> usize {
61        layout.children.nchildren()
62    }
63
64    fn child(layout: &Self::Layout, idx: usize) -> VortexResult<LayoutRef> {
65        layout.children.child(idx, Self::dtype(layout))
66    }
67
68    fn child_type(layout: &Self::Layout, idx: usize) -> LayoutChildType {
69        LayoutChildType::Chunk((idx, layout.chunk_offsets[idx]))
70    }
71
72    fn new_reader(
73        layout: &Self::Layout,
74        name: Arc<str>,
75        segment_source: Arc<dyn SegmentSource>,
76        session: &VortexSession,
77    ) -> VortexResult<LayoutReaderRef> {
78        Ok(Arc::new(ChunkedReader::new(
79            layout.clone(),
80            name,
81            segment_source,
82            session,
83        )))
84    }
85
86    #[cfg(gpu_unstable)]
87    fn new_gpu_reader(
88        layout: &Self::Layout,
89        name: Arc<str>,
90        segment_source: Arc<dyn SegmentSource>,
91        ctx: Arc<cudarc::driver::CudaContext>,
92    ) -> VortexResult<crate::gpu::GpuLayoutReaderRef> {
93        Ok(Arc::new(
94            crate::gpu::layouts::chunked::GpuChunkedLayoutReader::new(
95                layout.clone(),
96                name,
97                segment_source,
98                ctx,
99            ),
100        ))
101    }
102
103    fn build(
104        _encoding: &Self::Encoding,
105        dtype: &DType,
106        row_count: u64,
107        _metadata: &<Self::Metadata as DeserializeMetadata>::Output,
108        _segment_ids: Vec<SegmentId>,
109        children: &dyn LayoutChildren,
110        _ctx: ArrayContext,
111    ) -> VortexResult<Self::Layout> {
112        Ok(ChunkedLayout::new(
113            row_count,
114            dtype.clone(),
115            children.to_arc(),
116        ))
117    }
118
119    fn with_children(layout: &mut Self::Layout, children: Vec<LayoutRef>) -> VortexResult<()> {
120        let new_children = OwnedLayoutChildren::layout_children(children);
121
122        // Recalculate chunk offsets based on new children
123        let mut chunk_offsets = Vec::with_capacity(new_children.nchildren() + 1);
124        chunk_offsets.push(0);
125        for i in 0..new_children.nchildren() {
126            chunk_offsets.push(chunk_offsets[i] + new_children.child_row_count(i));
127        }
128
129        layout.children = new_children;
130        layout.chunk_offsets = chunk_offsets;
131        Ok(())
132    }
133}
134
135#[derive(Debug)]
136pub struct ChunkedLayoutEncoding;
137
138#[derive(Clone, Debug)]
139pub struct ChunkedLayout {
140    row_count: u64,
141    dtype: DType,
142    children: Arc<dyn LayoutChildren>,
143    chunk_offsets: Vec<u64>,
144}
145
146impl ChunkedLayout {
147    pub fn new(row_count: u64, dtype: DType, children: Arc<dyn LayoutChildren>) -> Self {
148        let mut chunk_offsets = Vec::with_capacity(children.nchildren() + 1);
149
150        chunk_offsets.push(0);
151        for i in 0..children.nchildren() {
152            chunk_offsets.push(chunk_offsets[i] + children.child_row_count(i));
153        }
154        assert_eq!(
155            chunk_offsets[children.nchildren()],
156            row_count,
157            "Row count mismatch"
158        );
159        Self {
160            row_count,
161            dtype,
162            children,
163            chunk_offsets,
164        }
165    }
166
167    pub fn children(&self) -> &Arc<dyn LayoutChildren> {
168        &self.children
169    }
170}