risingwave_connector/source/filesystem/
file_common.rs

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
// Copyright 2024 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::fmt::Debug;
use std::hash::Hash;
use std::marker::PhantomData;

use aws_sdk_s3::types::Object;
use risingwave_common::types::{JsonbVal, Timestamptz};
use serde::{Deserialize, Serialize};
use strum::Display;

use super::opendal_source::OpendalSource;
use crate::error::ConnectorResult;
use crate::source::{SplitId, SplitMetaData};

///  [`FsSplit`] Describes a file or a split of a file. A file is a generic concept,
/// and can be a local file, a distributed file system, or am object in S3 bucket.
#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct FsSplit {
    pub name: String,
    pub offset: usize,
    pub size: usize,
}

impl From<&Object> for FsSplit {
    fn from(value: &Object) -> Self {
        Self {
            name: value.key().unwrap().to_owned(),
            offset: 0,
            size: value.size().unwrap_or_default() as usize,
        }
    }
}

impl SplitMetaData for FsSplit {
    fn id(&self) -> SplitId {
        self.name.as_str().into()
    }

    fn restore_from_json(value: JsonbVal) -> ConnectorResult<Self> {
        serde_json::from_value(value.take()).map_err(Into::into)
    }

    fn encode_to_json(&self) -> JsonbVal {
        serde_json::to_value(self.clone()).unwrap().into()
    }

    fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> {
        let offset = last_seen_offset.parse().unwrap();
        self.offset = offset;
        Ok(())
    }
}

impl FsSplit {
    pub fn new(name: String, start: usize, size: usize) -> Self {
        Self {
            name,
            offset: start,
            size,
        }
    }
}

///  [`OpendalFsSplit`] Describes a file or a split of a file. A file is a generic concept,
/// and can be a local file, a distributed file system, or am object in S3 bucket.
#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)]
pub struct OpendalFsSplit<Src: OpendalSource> {
    pub name: String,
    pub offset: usize,
    // For Parquet encoding, the `size` represents the number of rows, while for other encodings, the `size` denotes the file size.
    pub size: usize,
    _marker: PhantomData<Src>,
}

impl<Src: OpendalSource> From<&Object> for OpendalFsSplit<Src> {
    fn from(value: &Object) -> Self {
        Self {
            name: value.key().unwrap().to_owned(),
            offset: 0,
            size: value.size().unwrap_or_default() as usize,
            _marker: PhantomData,
        }
    }
}

impl<Src: OpendalSource> SplitMetaData for OpendalFsSplit<Src> {
    fn id(&self) -> SplitId {
        self.name.as_str().into()
    }

    fn restore_from_json(value: JsonbVal) -> ConnectorResult<Self> {
        serde_json::from_value(value.take()).map_err(Into::into)
    }

    fn encode_to_json(&self) -> JsonbVal {
        serde_json::to_value(self.clone()).unwrap().into()
    }

    fn update_offset(&mut self, last_seen_offset: String) -> ConnectorResult<()> {
        let offset = last_seen_offset.parse().unwrap();
        self.offset = offset;
        Ok(())
    }
}

impl<Src: OpendalSource> OpendalFsSplit<Src> {
    pub fn new(name: String, start: usize, size: usize) -> Self {
        Self {
            name,
            offset: start,
            size,
            _marker: PhantomData,
        }
    }

    pub fn empty_split() -> Self {
        Self {
            name: "empty_split".to_string(),
            offset: 0,
            size: 0,
            _marker: PhantomData,
        }
    }
}

#[derive(Clone, Debug)]
pub struct FsPageItem {
    pub name: String,
    pub size: i64,
    pub timestamp: Timestamptz,
}

pub type FsPage = Vec<FsPageItem>;

#[derive(Debug, Default, Clone, PartialEq, Display, Deserialize)]
pub enum CompressionFormat {
    #[default]
    None,

    #[serde(rename = "gzip", alias = "gz")]
    Gzip,
}