risingwave_batch/executor/
s3_file_scan.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
// 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 futures_async_stream::try_stream;
use futures_util::stream::StreamExt;
use risingwave_common::catalog::{Field, Schema};
use risingwave_connector::source::iceberg::{new_s3_operator, read_parquet_file};
use risingwave_pb::batch_plan::file_scan_node;
use risingwave_pb::batch_plan::file_scan_node::StorageType;
use risingwave_pb::batch_plan::plan_node::NodeBody;

use crate::error::BatchError;
use crate::executor::{BoxedExecutor, BoxedExecutorBuilder, DataChunk, Executor, ExecutorBuilder};
use crate::task::BatchTaskContext;

#[derive(PartialEq, Debug)]
pub enum FileFormat {
    Parquet,
}

/// S3 file scan executor. Currently only support parquet file format.
pub struct S3FileScanExecutor {
    file_format: FileFormat,
    file_location: Vec<String>,
    s3_region: String,
    s3_access_key: String,
    s3_secret_key: String,
    batch_size: usize,
    schema: Schema,
    identity: String,
}

impl Executor for S3FileScanExecutor {
    fn schema(&self) -> &risingwave_common::catalog::Schema {
        &self.schema
    }

    fn identity(&self) -> &str {
        &self.identity
    }

    fn execute(self: Box<Self>) -> super::BoxedDataChunkStream {
        self.do_execute().boxed()
    }
}

impl S3FileScanExecutor {
    pub fn new(
        file_format: FileFormat,
        file_location: Vec<String>,
        s3_region: String,
        s3_access_key: String,
        s3_secret_key: String,
        batch_size: usize,
        schema: Schema,
        identity: String,
    ) -> Self {
        Self {
            file_format,
            file_location,
            s3_region,
            s3_access_key,
            s3_secret_key,
            batch_size,
            schema,
            identity,
        }
    }

    #[try_stream(ok = DataChunk, error = BatchError)]
    async fn do_execute(self: Box<Self>) {
        assert_eq!(self.file_format, FileFormat::Parquet);
        for file in self.file_location {
            let op = new_s3_operator(
                self.s3_region.clone(),
                self.s3_access_key.clone(),
                self.s3_secret_key.clone(),
                file.clone(),
            )?;
            let chunk_stream = read_parquet_file(op, file, None, None, self.batch_size, 0).await?;
            #[for_await]
            for stream_chunk in chunk_stream {
                let stream_chunk = stream_chunk?;
                let (data_chunk, _) = stream_chunk.into_parts();
                yield data_chunk;
            }
        }
    }
}

pub struct FileScanExecutorBuilder {}

#[async_trait::async_trait]
impl BoxedExecutorBuilder for FileScanExecutorBuilder {
    async fn new_boxed_executor<C: BatchTaskContext>(
        source: &ExecutorBuilder<'_, C>,
        _inputs: Vec<BoxedExecutor>,
    ) -> crate::error::Result<BoxedExecutor> {
        let file_scan_node = try_match_expand!(
            source.plan_node().get_node_body().unwrap(),
            NodeBody::FileScan
        )?;

        assert_eq!(file_scan_node.storage_type, StorageType::S3 as i32);

        Ok(Box::new(S3FileScanExecutor::new(
            match file_scan_node::FileFormat::try_from(file_scan_node.file_format).unwrap() {
                file_scan_node::FileFormat::Parquet => FileFormat::Parquet,
                file_scan_node::FileFormat::Unspecified => unreachable!(),
            },
            file_scan_node.file_location.clone(),
            file_scan_node.s3_region.clone(),
            file_scan_node.s3_access_key.clone(),
            file_scan_node.s3_secret_key.clone(),
            source.context.get_config().developer.chunk_size,
            Schema::from_iter(file_scan_node.columns.iter().map(Field::from)),
            source.plan_node().get_identity().clone(),
        )))
    }
}