risingwave_connector/source/reader/
fs_reader.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
// 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.

#![deprecated = "will be replaced by new fs source (list + fetch)"]

use std::sync::Arc;

use anyhow::Context;
use futures::stream::pending;
use futures::StreamExt;
use risingwave_common::catalog::ColumnId;

use crate::error::ConnectorResult;
use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig};
use crate::source::{
    create_split_reader, BoxChunkSourceStream, ConnectorProperties, ConnectorState,
    SourceColumnDesc, SourceContext, SplitReader,
};
use crate::{dispatch_source_prop, WithOptionsSecResolved};

#[derive(Clone, Debug)]
pub struct FsSourceReader {
    pub config: ConnectorProperties,
    pub columns: Vec<SourceColumnDesc>,
    pub properties: WithOptionsSecResolved,
    pub parser_config: SpecificParserConfig,
}

impl FsSourceReader {
    #[allow(clippy::too_many_arguments)]
    pub fn new(
        properties: WithOptionsSecResolved,
        columns: Vec<SourceColumnDesc>,
        parser_config: SpecificParserConfig,
    ) -> ConnectorResult<Self> {
        // Store the connector node address to properties for later use.
        let config = ConnectorProperties::extract(properties.clone(), false)?;

        Ok(Self {
            config,
            columns,
            properties,
            parser_config,
        })
    }

    fn get_target_columns(
        &self,
        column_ids: Vec<ColumnId>,
    ) -> ConnectorResult<Vec<SourceColumnDesc>> {
        column_ids
            .iter()
            .map(|id| {
                self.columns
                    .iter()
                    .find(|c| c.column_id == *id)
                    .with_context(|| {
                        format!("Failed to find column id: {} in source: {:?}", id, self)
                    })
                    .cloned()
            })
            .try_collect()
            .map_err(Into::into)
    }

    pub async fn to_stream(
        &self,
        state: ConnectorState,
        column_ids: Vec<ColumnId>,
        source_ctx: Arc<SourceContext>,
    ) -> ConnectorResult<BoxChunkSourceStream> {
        let config = self.config.clone();
        let columns = self.get_target_columns(column_ids)?;

        let parser_config = ParserConfig {
            specific: self.parser_config.clone(),
            common: CommonParserConfig {
                rw_columns: columns,
            },
        };
        let stream = match state {
            None => pending().boxed(),
            Some(splits) => {
                dispatch_source_prop!(config, prop, {
                    create_split_reader(*prop, splits, parser_config, source_ctx, None)
                        .await?
                        .into_stream()
                })
            }
        };
        Ok(stream)
    }
}