risingwave_stream/from_proto/
sink.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
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
// 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::sync::Arc;

use anyhow::anyhow;
use risingwave_common::catalog::{ColumnCatalog, Schema};
use risingwave_common::secret::LocalSecretManager;
use risingwave_common::types::DataType;
use risingwave_connector::match_sink_name_str;
use risingwave_connector::sink::catalog::{SinkFormatDesc, SinkId, SinkType};
use risingwave_connector::sink::file_sink::fs::FsSink;
use risingwave_connector::sink::{
    SinkError, SinkMetaClient, SinkParam, SinkWriterParam, CONNECTOR_TYPE_KEY, SINK_TYPE_OPTION,
};
use risingwave_pb::catalog::Table;
use risingwave_pb::plan_common::PbColumnCatalog;
use risingwave_pb::stream_plan::{SinkLogStoreType, SinkNode};
use risingwave_pb::telemetry::{PbTelemetryDatabaseObject, PbTelemetryEventStage};

use super::*;
use crate::common::log_store_impl::in_mem::BoundedInMemLogStoreFactory;
use crate::common::log_store_impl::kv_log_store::{
    KvLogStoreFactory, KvLogStoreMetrics, KvLogStorePkInfo, KV_LOG_STORE_V2_INFO,
};
use crate::executor::{SinkExecutor, StreamExecutorError};
use crate::telemetry::report_event;

pub struct SinkExecutorBuilder;

fn telemetry_sink_build(
    sink_id: &SinkId,
    connector_name: &str,
    sink_format_desc: &Option<SinkFormatDesc>,
) {
    let attr = sink_format_desc.as_ref().map(|f| {
        let mut builder = jsonbb::Builder::<Vec<u8>>::new();
        builder.begin_object();
        builder.add_string("format");
        builder.add_value(jsonbb::ValueRef::String(f.format.to_string().as_str()));
        builder.add_string("encode");
        builder.add_value(jsonbb::ValueRef::String(f.encode.to_string().as_str()));
        builder.end_object();
        builder.finish()
    });

    report_event(
        PbTelemetryEventStage::CreateStreamJob,
        "sink",
        sink_id.sink_id() as i64,
        Some(connector_name.to_string()),
        Some(PbTelemetryDatabaseObject::Sink),
        attr,
    )
}

fn resolve_pk_info(
    input_schema: &Schema,
    log_store_table: &Table,
) -> StreamResult<&'static KvLogStorePkInfo> {
    let predefined_column_len = log_store_table.columns.len() - input_schema.fields.len();

    #[expect(deprecated)]
    let info = match predefined_column_len {
        len if len
            == crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V1_INFO
                .predefined_column_len() =>
        {
            Ok(&crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V1_INFO)
        }
        len if len == KV_LOG_STORE_V2_INFO.predefined_column_len() => Ok(&KV_LOG_STORE_V2_INFO),
        other_len => Err(anyhow!(
            "invalid log store predefined len {}. log store table: {:?}, input_schema: {:?}",
            other_len,
            log_store_table,
            input_schema
        )),
    }?;
    validate_payload_schema(
        &log_store_table.columns[predefined_column_len..],
        input_schema,
    )?;
    Ok(info)
}

fn validate_payload_schema(
    log_store_payload_schema: &[PbColumnCatalog],
    input_schema: &Schema,
) -> StreamResult<()> {
    if log_store_payload_schema
        .iter()
        .zip_eq(input_schema.fields.iter())
        .map(|(log_store_col, input_field)| {
            let log_store_col_type = DataType::from(
                log_store_col
                    .column_desc
                    .as_ref()
                    .unwrap()
                    .column_type
                    .as_ref()
                    .unwrap(),
            );
            log_store_col_type.equals_datatype(&input_field.data_type)
        })
        .all(|equal| equal)
    {
        Ok(())
    } else {
        Err(anyhow!(
            "mismatch schema: log store: {:?}, input: {:?}",
            log_store_payload_schema,
            input_schema
        )
        .into())
    }
}

impl ExecutorBuilder for SinkExecutorBuilder {
    type Node = SinkNode;

    async fn new_boxed_executor(
        params: ExecutorParams,
        node: &Self::Node,
        state_store: impl StateStore,
    ) -> StreamResult<Executor> {
        let [input_executor]: [_; 1] = params.input.try_into().unwrap();
        let input_data_types = input_executor.info().schema.data_types();
        let chunk_size = params.env.config().developer.chunk_size;

        let sink_desc = node.sink_desc.as_ref().unwrap();
        let sink_type = SinkType::from_proto(sink_desc.get_sink_type().unwrap());
        let sink_id: SinkId = sink_desc.get_id().into();
        let sink_name = sink_desc.get_name().to_owned();
        let db_name = sink_desc.get_db_name().into();
        let sink_from_name = sink_desc.get_sink_from_name().into();
        let properties = sink_desc.get_properties().clone();
        let secret_refs = sink_desc.get_secret_refs().clone();
        let downstream_pk = sink_desc
            .downstream_pk
            .iter()
            .map(|i| *i as usize)
            .collect_vec();
        let columns = sink_desc
            .column_catalogs
            .clone()
            .into_iter()
            .map(ColumnCatalog::from)
            .collect_vec();

        let connector = {
            let sink_type = properties.get(CONNECTOR_TYPE_KEY).ok_or_else(|| {
                StreamExecutorError::from((
                    SinkError::Config(anyhow!("missing config: {}", CONNECTOR_TYPE_KEY)),
                    sink_id.sink_id,
                ))
            })?;

            match_sink_name_str!(
                sink_type.to_lowercase().as_str(),
                SinkType,
                Ok(SinkType::SINK_NAME),
                |other| {
                    Err(StreamExecutorError::from((
                        SinkError::Config(anyhow!("unsupported sink connector {}", other)),
                        sink_id.sink_id,
                    )))
                }
            )
        }?;
        let format_desc = match &sink_desc.format_desc {
            // Case A: new syntax `format ... encode ...`
            Some(f) => Some(
                f.clone()
                    .try_into()
                    .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?,
            ),
            None => match sink_desc.properties.get(SINK_TYPE_OPTION) {
                // Case B: old syntax `type = '...'`
                Some(t) => SinkFormatDesc::from_legacy_type(connector, t)
                    .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?,
                // Case C: no format + encode required
                None => None,
            },
        };

        let properties_with_secret =
            LocalSecretManager::global().fill_secrets(properties, secret_refs)?;

        let format_desc_with_secret = SinkParam::fill_secret_for_format_desc(format_desc)
            .map_err(|e| StreamExecutorError::from((e, sink_id.sink_id)))?;

        let sink_param = SinkParam {
            sink_id,
            sink_name: sink_name.clone(),
            properties: properties_with_secret,
            columns: columns
                .iter()
                .filter(|col| !col.is_hidden)
                .map(|col| col.column_desc.clone())
                .collect(),
            downstream_pk,
            sink_type,
            format_desc: format_desc_with_secret,
            db_name,
            sink_from_name,
        };

        let sink_write_param = SinkWriterParam {
            executor_id: params.executor_id,
            vnode_bitmap: params.vnode_bitmap.clone(),
            meta_client: params.env.meta_client().map(SinkMetaClient::MetaClient),
            extra_partition_col_idx: sink_desc.extra_partition_col_idx.map(|v| v as usize),

            actor_id: params.actor_context.id,
            sink_id,
            sink_name,
            connector: connector.to_string(),
        };

        let log_store_identity = format!(
            "sink[{}]-[{}]-executor[{}]",
            connector, sink_id.sink_id, params.executor_id
        );

        telemetry_sink_build(&sink_id, connector, &sink_param.format_desc);

        let exec = match node.log_store_type() {
            // Default value is the normal in memory log store to be backward compatible with the
            // previously unset value
            SinkLogStoreType::InMemoryLogStore | SinkLogStoreType::Unspecified => {
                let factory = BoundedInMemLogStoreFactory::new(1);
                SinkExecutor::new(
                    params.actor_context,
                    params.info.clone(),
                    input_executor,
                    sink_write_param,
                    sink_param,
                    columns,
                    factory,
                    chunk_size,
                    input_data_types,
                )
                .await?
                .boxed()
            }
            SinkLogStoreType::KvLogStore => {
                let metrics = KvLogStoreMetrics::new(
                    &params.executor_stats,
                    params.actor_context.id,
                    &sink_param,
                    connector,
                );

                let table = node.table.as_ref().unwrap().clone();
                let input_schema = input_executor.schema();
                let pk_info = resolve_pk_info(input_schema, &table)?;

                // TODO: support setting max row count in config
                let factory = KvLogStoreFactory::new(
                    state_store,
                    table,
                    params.vnode_bitmap.clone().map(Arc::new),
                    65536,
                    metrics,
                    log_store_identity,
                    pk_info,
                );

                SinkExecutor::new(
                    params.actor_context,
                    params.info.clone(),
                    input_executor,
                    sink_write_param,
                    sink_param,
                    columns,
                    factory,
                    chunk_size,
                    input_data_types,
                )
                .await?
                .boxed()
            }
        };

        Ok((params.info, exec).into())
    }
}