Skip to main content

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    fn build(
87        _encoding: &Self::Encoding,
88        dtype: &DType,
89        row_count: u64,
90        _metadata: &<Self::Metadata as DeserializeMetadata>::Output,
91        _segment_ids: Vec<SegmentId>,
92        children: &dyn LayoutChildren,
93        _ctx: &ArrayContext,
94    ) -> VortexResult<Self::Layout> {
95        Ok(ChunkedLayout::new(
96            row_count,
97            dtype.clone(),
98            children.to_arc(),
99        ))
100    }
101
102    fn with_children(layout: &mut Self::Layout, children: Vec<LayoutRef>) -> VortexResult<()> {
103        let new_children = OwnedLayoutChildren::layout_children(children);
104
105        // Recalculate chunk offsets based on new children
106        let mut chunk_offsets = Vec::with_capacity(new_children.nchildren() + 1);
107        chunk_offsets.push(0);
108        for i in 0..new_children.nchildren() {
109            chunk_offsets.push(chunk_offsets[i] + new_children.child_row_count(i));
110        }
111
112        layout.children = new_children;
113        layout.chunk_offsets = chunk_offsets;
114        Ok(())
115    }
116}
117
118#[derive(Debug)]
119pub struct ChunkedLayoutEncoding;
120
121#[derive(Clone, Debug)]
122pub struct ChunkedLayout {
123    row_count: u64,
124    dtype: DType,
125    children: Arc<dyn LayoutChildren>,
126    chunk_offsets: Vec<u64>,
127}
128
129impl ChunkedLayout {
130    pub fn new(row_count: u64, dtype: DType, children: Arc<dyn LayoutChildren>) -> Self {
131        let mut chunk_offsets = Vec::with_capacity(children.nchildren() + 1);
132
133        chunk_offsets.push(0);
134        for i in 0..children.nchildren() {
135            chunk_offsets.push(chunk_offsets[i] + children.child_row_count(i));
136        }
137        assert_eq!(
138            chunk_offsets[children.nchildren()],
139            row_count,
140            "Row count mismatch"
141        );
142        Self {
143            row_count,
144            dtype,
145            children,
146            chunk_offsets,
147        }
148    }
149
150    pub fn children(&self) -> &Arc<dyn LayoutChildren> {
151        &self.children
152    }
153}