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, DeserializeMetadata, EmptyMetadata};
10use vortex_dtype::DType;
11use vortex_error::VortexResult;
12
13use crate::children::LayoutChildren;
14use crate::layouts::chunked::reader::ChunkedReader;
15use crate::segments::{SegmentId, SegmentSource};
16use crate::{
17    LayoutChildType, LayoutEncodingRef, LayoutId, LayoutReaderRef, LayoutRef, VTable, vtable,
18};
19
20vtable!(Chunked);
21
22impl VTable for ChunkedVTable {
23    type Layout = ChunkedLayout;
24    type Encoding = ChunkedLayoutEncoding;
25    type Metadata = EmptyMetadata;
26
27    fn id(_encoding: &Self::Encoding) -> LayoutId {
28        LayoutId::new_ref("vortex.chunked")
29    }
30
31    fn encoding(_layout: &Self::Layout) -> LayoutEncodingRef {
32        LayoutEncodingRef::new_ref(ChunkedLayoutEncoding.as_ref())
33    }
34
35    fn row_count(layout: &Self::Layout) -> u64 {
36        layout.row_count
37    }
38
39    fn dtype(layout: &Self::Layout) -> &DType {
40        &layout.dtype
41    }
42
43    fn metadata(_layout: &Self::Layout) -> Self::Metadata {
44        EmptyMetadata
45    }
46
47    fn segment_ids(_layout: &Self::Layout) -> Vec<SegmentId> {
48        vec![]
49    }
50
51    fn nchildren(layout: &Self::Layout) -> usize {
52        layout.children.nchildren()
53    }
54
55    fn child(layout: &Self::Layout, idx: usize) -> VortexResult<LayoutRef> {
56        layout.children.child(idx, Self::dtype(layout))
57    }
58
59    fn child_type(layout: &Self::Layout, idx: usize) -> LayoutChildType {
60        LayoutChildType::Chunk((idx, layout.chunk_offsets[idx]))
61    }
62
63    fn new_reader(
64        layout: &Self::Layout,
65        name: Arc<str>,
66        segment_source: Arc<dyn SegmentSource>,
67    ) -> VortexResult<LayoutReaderRef> {
68        Ok(Arc::new(ChunkedReader::new(
69            layout.clone(),
70            name,
71            segment_source,
72        )))
73    }
74
75    fn build(
76        _encoding: &Self::Encoding,
77        dtype: &DType,
78        row_count: u64,
79        _metadata: &<Self::Metadata as DeserializeMetadata>::Output,
80        _segment_ids: Vec<SegmentId>,
81        children: &dyn LayoutChildren,
82        _ctx: ArrayContext,
83    ) -> VortexResult<Self::Layout> {
84        Ok(ChunkedLayout::new(
85            row_count,
86            dtype.clone(),
87            children.to_arc(),
88        ))
89    }
90}
91
92#[derive(Debug)]
93pub struct ChunkedLayoutEncoding;
94
95#[derive(Clone, Debug)]
96pub struct ChunkedLayout {
97    row_count: u64,
98    dtype: DType,
99    children: Arc<dyn LayoutChildren>,
100    chunk_offsets: Vec<u64>,
101}
102
103impl ChunkedLayout {
104    pub fn new(row_count: u64, dtype: DType, children: Arc<dyn LayoutChildren>) -> Self {
105        let mut chunk_offsets = Vec::with_capacity(children.nchildren() + 1);
106
107        chunk_offsets.push(0);
108        for i in 0..children.nchildren() {
109            chunk_offsets.push(chunk_offsets[i] + children.child_row_count(i));
110        }
111        assert_eq!(
112            chunk_offsets[children.nchildren()],
113            row_count,
114            "Row count mismatch"
115        );
116        Self {
117            row_count,
118            dtype,
119            children,
120            chunk_offsets,
121        }
122    }
123}