risingwave_connector/source/
base.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
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
// 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::collections::{BTreeMap, HashMap};
use std::sync::Arc;

use anyhow::anyhow;
use async_trait::async_trait;
use aws_sdk_s3::types::Object;
use bytes::Bytes;
use enum_as_inner::EnumAsInner;
use futures::stream::BoxStream;
use futures::Stream;
use itertools::Itertools;
use risingwave_common::array::StreamChunk;
use risingwave_common::bail;
use risingwave_common::catalog::TableId;
use risingwave_common::secret::LocalSecretManager;
use risingwave_common::types::{JsonbVal, Scalar};
use risingwave_pb::catalog::{PbSource, PbStreamSourceInfo};
use risingwave_pb::plan_common::ExternalTableDesc;
use risingwave_pb::source::ConnectorSplit;
use serde::de::DeserializeOwned;
use tokio::sync::mpsc;

use super::cdc::DebeziumCdcMeta;
use super::datagen::DatagenMeta;
use super::google_pubsub::GooglePubsubMeta;
use super::kafka::KafkaMeta;
use super::kinesis::KinesisMeta;
use super::monitor::SourceMetrics;
use super::nexmark::source::message::NexmarkMeta;
use super::{AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR};
use crate::error::ConnectorResult as Result;
use crate::parser::schema_change::SchemaChangeEnvelope;
use crate::parser::ParserConfig;
use crate::source::filesystem::FsPageItem;
use crate::source::monitor::EnumeratorMetrics;
use crate::source::SplitImpl::{CitusCdc, MongodbCdc, MysqlCdc, PostgresCdc, SqlServerCdc};
use crate::with_options::WithOptions;
use crate::{
    dispatch_source_prop, dispatch_split_impl, for_all_sources, impl_connector_properties,
    impl_split, match_source_name_str, WithOptionsSecResolved,
};

const SPLIT_TYPE_FIELD: &str = "split_type";
const SPLIT_INFO_FIELD: &str = "split_info";
pub const UPSTREAM_SOURCE_KEY: &str = "connector";

pub trait TryFromBTreeMap: Sized + UnknownFields {
    /// Used to initialize the source properties from the raw untyped `WITH` options.
    fn try_from_btreemap(
        props: BTreeMap<String, String>,
        deny_unknown_fields: bool,
    ) -> Result<Self>;
}

/// Represents `WITH` options for sources.
///
/// Each instance should add a `#[derive(with_options::WithOptions)]` marker.
pub trait SourceProperties: TryFromBTreeMap + Clone + WithOptions + std::fmt::Debug {
    const SOURCE_NAME: &'static str;
    type Split: SplitMetaData
        + TryFrom<SplitImpl, Error = crate::error::ConnectorError>
        + Into<SplitImpl>;
    type SplitEnumerator: SplitEnumerator<Properties = Self, Split = Self::Split>;
    type SplitReader: SplitReader<Split = Self::Split, Properties = Self>;

    /// Load additional info from `PbSource`. Currently only used by CDC.
    fn init_from_pb_source(&mut self, _source: &PbSource) {}

    /// Load additional info from `ExternalTableDesc`. Currently only used by CDC.
    fn init_from_pb_cdc_table_desc(&mut self, _table_desc: &ExternalTableDesc) {}
}

pub trait UnknownFields {
    /// Unrecognized fields in the `WITH` clause.
    fn unknown_fields(&self) -> HashMap<String, String>;
}

impl<P: DeserializeOwned + UnknownFields> TryFromBTreeMap for P {
    fn try_from_btreemap(
        props: BTreeMap<String, String>,
        deny_unknown_fields: bool,
    ) -> Result<Self> {
        let json_value = serde_json::to_value(props)?;
        let res = serde_json::from_value::<P>(json_value)?;

        if !deny_unknown_fields || res.unknown_fields().is_empty() {
            Ok(res)
        } else {
            bail!(
                "Unknown fields in the WITH clause: {:?}",
                res.unknown_fields()
            )
        }
    }
}

pub async fn create_split_reader<P: SourceProperties>(
    prop: P,
    splits: Vec<SplitImpl>,
    parser_config: ParserConfig,
    source_ctx: SourceContextRef,
    columns: Option<Vec<Column>>,
) -> Result<P::SplitReader> {
    let splits = splits.into_iter().map(P::Split::try_from).try_collect()?;
    P::SplitReader::new(prop, splits, parser_config, source_ctx, columns).await
}

/// [`SplitEnumerator`] fetches the split metadata from the external source service.
/// NOTE: It runs in the meta server, so probably it should be moved to the `meta` crate.
#[async_trait]
pub trait SplitEnumerator: Sized {
    type Split: SplitMetaData + Send;
    type Properties;

    async fn new(properties: Self::Properties, context: SourceEnumeratorContextRef)
        -> Result<Self>;
    async fn list_splits(&mut self) -> Result<Vec<Self::Split>>;
}

pub type SourceContextRef = Arc<SourceContext>;
pub type SourceEnumeratorContextRef = Arc<SourceEnumeratorContext>;

/// The max size of a chunk yielded by source stream.
pub const MAX_CHUNK_SIZE: usize = 1024;

#[derive(Debug, Clone)]
pub struct SourceCtrlOpts {
    /// The max size of a chunk yielded by source stream.
    pub chunk_size: usize,
    /// Rate limit of source
    pub rate_limit: Option<u32>,
}

// The options in `SourceCtrlOpts` are so important that we don't want to impl `Default` for it,
// so that we can prevent any unintentional use of the default value.
impl !Default for SourceCtrlOpts {}

#[derive(Debug)]
pub struct SourceEnumeratorContext {
    pub info: SourceEnumeratorInfo,
    pub metrics: Arc<EnumeratorMetrics>,
}

impl SourceEnumeratorContext {
    /// Create a dummy `SourceEnumeratorContext` for testing purpose, or for the situation
    /// where the real context doesn't matter.
    pub fn dummy() -> SourceEnumeratorContext {
        SourceEnumeratorContext {
            info: SourceEnumeratorInfo { source_id: 0 },
            metrics: Arc::new(EnumeratorMetrics::default()),
        }
    }
}

#[derive(Clone, Debug)]
pub struct SourceEnumeratorInfo {
    pub source_id: u32,
}

#[derive(Debug)]
pub struct SourceContext {
    pub actor_id: u32,
    pub source_id: TableId,
    pub fragment_id: u32,
    pub source_name: String,
    pub metrics: Arc<SourceMetrics>,
    pub source_ctrl_opts: SourceCtrlOpts,
    pub connector_props: ConnectorProperties,
    // source parser put schema change event into this channel
    pub schema_change_tx:
        Option<mpsc::Sender<(SchemaChangeEnvelope, tokio::sync::oneshot::Sender<()>)>>,
}

impl SourceContext {
    pub fn new(
        actor_id: u32,
        source_id: TableId,
        fragment_id: u32,
        source_name: String,
        metrics: Arc<SourceMetrics>,
        source_ctrl_opts: SourceCtrlOpts,
        connector_props: ConnectorProperties,
        schema_change_channel: Option<
            mpsc::Sender<(SchemaChangeEnvelope, tokio::sync::oneshot::Sender<()>)>,
        >,
    ) -> Self {
        Self {
            actor_id,
            source_id,
            fragment_id,
            source_name,
            metrics,
            source_ctrl_opts,
            connector_props,
            schema_change_tx: schema_change_channel,
        }
    }

    /// Create a dummy `SourceContext` for testing purpose, or for the situation
    /// where the real context doesn't matter.
    pub fn dummy() -> Self {
        Self::new(
            0,
            TableId::new(0),
            0,
            "dummy".to_string(),
            Arc::new(SourceMetrics::default()),
            SourceCtrlOpts {
                chunk_size: MAX_CHUNK_SIZE,
                rate_limit: None,
            },
            ConnectorProperties::default(),
            None,
        )
    }
}

#[derive(Clone, Copy, Debug, PartialEq, Eq, Default)]
pub enum SourceFormat {
    #[default]
    Invalid,
    Native,
    None,
    Debezium,
    DebeziumMongo,
    Maxwell,
    Canal,
    Upsert,
    Plain,
}

/// Refer to [`crate::parser::EncodingProperties`]
#[derive(Clone, Copy, Debug, PartialEq, Eq, Default)]
pub enum SourceEncode {
    #[default]
    Invalid,
    Native,
    None,
    Avro,
    Csv,
    Protobuf,
    Json,
    Bytes,
    Parquet,
}

#[derive(Clone, Copy, Debug, PartialEq, Eq, Default)]
pub struct SourceStruct {
    pub format: SourceFormat,
    pub encode: SourceEncode,
}

impl SourceStruct {
    pub fn new(format: SourceFormat, encode: SourceEncode) -> Self {
        Self { format, encode }
    }
}

// Only return valid (format, encode)
pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result<SourceStruct> {
    use risingwave_pb::plan_common::{PbEncodeType, PbFormatType, RowFormatType};

    // old version meta.
    if let Ok(format) = info.get_row_format() {
        let (format, encode) = match format {
            RowFormatType::Json => (SourceFormat::Plain, SourceEncode::Json),
            RowFormatType::Protobuf => (SourceFormat::Plain, SourceEncode::Protobuf),
            RowFormatType::DebeziumJson => (SourceFormat::Debezium, SourceEncode::Json),
            RowFormatType::Avro => (SourceFormat::Plain, SourceEncode::Avro),
            RowFormatType::Maxwell => (SourceFormat::Maxwell, SourceEncode::Json),
            RowFormatType::CanalJson => (SourceFormat::Canal, SourceEncode::Json),
            RowFormatType::Csv => (SourceFormat::Plain, SourceEncode::Csv),
            RowFormatType::Native => (SourceFormat::Native, SourceEncode::Native),
            RowFormatType::DebeziumAvro => (SourceFormat::Debezium, SourceEncode::Avro),
            RowFormatType::UpsertJson => (SourceFormat::Upsert, SourceEncode::Json),
            RowFormatType::UpsertAvro => (SourceFormat::Upsert, SourceEncode::Avro),
            RowFormatType::DebeziumMongoJson => (SourceFormat::DebeziumMongo, SourceEncode::Json),
            RowFormatType::Bytes => (SourceFormat::Plain, SourceEncode::Bytes),
            RowFormatType::RowUnspecified => unreachable!(),
        };
        return Ok(SourceStruct::new(format, encode));
    }
    let source_format = info.get_format()?;
    let source_encode = info.get_row_encode()?;
    let (format, encode) = match (source_format, source_encode) {
        (PbFormatType::Plain, PbEncodeType::Json) => (SourceFormat::Plain, SourceEncode::Json),
        (PbFormatType::Plain, PbEncodeType::Protobuf) => {
            (SourceFormat::Plain, SourceEncode::Protobuf)
        }
        (PbFormatType::Debezium, PbEncodeType::Json) => {
            (SourceFormat::Debezium, SourceEncode::Json)
        }
        (PbFormatType::Plain, PbEncodeType::Avro) => (SourceFormat::Plain, SourceEncode::Avro),
        (PbFormatType::Maxwell, PbEncodeType::Json) => (SourceFormat::Maxwell, SourceEncode::Json),
        (PbFormatType::Canal, PbEncodeType::Json) => (SourceFormat::Canal, SourceEncode::Json),
        (PbFormatType::Plain, PbEncodeType::Csv) => (SourceFormat::Plain, SourceEncode::Csv),
        (PbFormatType::Plain, PbEncodeType::Parquet) => {
            (SourceFormat::Plain, SourceEncode::Parquet)
        }
        (PbFormatType::Native, PbEncodeType::Native) => {
            (SourceFormat::Native, SourceEncode::Native)
        }
        (PbFormatType::None, PbEncodeType::None) => (SourceFormat::None, SourceEncode::None),
        (PbFormatType::Debezium, PbEncodeType::Avro) => {
            (SourceFormat::Debezium, SourceEncode::Avro)
        }
        (PbFormatType::Upsert, PbEncodeType::Json) => (SourceFormat::Upsert, SourceEncode::Json),
        (PbFormatType::Upsert, PbEncodeType::Avro) => (SourceFormat::Upsert, SourceEncode::Avro),
        (PbFormatType::DebeziumMongo, PbEncodeType::Json) => {
            (SourceFormat::DebeziumMongo, SourceEncode::Json)
        }
        (PbFormatType::Plain, PbEncodeType::Bytes) => (SourceFormat::Plain, SourceEncode::Bytes),
        (PbFormatType::Upsert, PbEncodeType::Protobuf) => {
            (SourceFormat::Upsert, SourceEncode::Protobuf)
        }
        (format, encode) => {
            bail!(
                "Unsupported combination of format {:?} and encode {:?}",
                format,
                encode
            );
        }
    };
    Ok(SourceStruct::new(format, encode))
}

/// Stream of [`SourceMessage`].
pub type BoxSourceStream = BoxStream<'static, crate::error::ConnectorResult<Vec<SourceMessage>>>;

// Manually expand the trait alias to improve IDE experience.
pub trait ChunkSourceStream:
    Stream<Item = crate::error::ConnectorResult<StreamChunk>> + Send + 'static
{
}
impl<T> ChunkSourceStream for T where
    T: Stream<Item = crate::error::ConnectorResult<StreamChunk>> + Send + 'static
{
}

pub type BoxChunkSourceStream = BoxStream<'static, crate::error::ConnectorResult<StreamChunk>>;
pub type BoxTryStream<M> = BoxStream<'static, crate::error::ConnectorResult<M>>;

/// [`SplitReader`] is a new abstraction of the external connector read interface which is
/// responsible for parsing, it is used to read messages from the outside and transform them into a
/// stream of parsed [`StreamChunk`]
#[async_trait]
pub trait SplitReader: Sized + Send {
    type Properties;
    type Split: SplitMetaData;

    async fn new(
        properties: Self::Properties,
        state: Vec<Self::Split>,
        parser_config: ParserConfig,
        source_ctx: SourceContextRef,
        columns: Option<Vec<Column>>,
    ) -> crate::error::ConnectorResult<Self>;

    fn into_stream(self) -> BoxChunkSourceStream;

    fn backfill_info(&self) -> HashMap<SplitId, BackfillInfo> {
        HashMap::new()
    }

    async fn seek_to_latest(&mut self) -> Result<Vec<SplitImpl>> {
        Err(anyhow!("seek_to_latest is not supported for this connector").into())
    }
}

/// Information used to determine whether we should start and finish source backfill.
///
/// XXX: if a connector cannot provide the latest offsets (but we want to make it shareable),
/// perhaps we should ban blocking DDL for it.
#[derive(Debug, Clone)]
pub enum BackfillInfo {
    HasDataToBackfill {
        /// The last available offsets for each split (**inclusive**).
        ///
        /// This will be used to determine whether source backfill is finished when
        /// there are no _new_ messages coming from upstream `SourceExecutor`. Otherwise,
        /// blocking DDL cannot finish until new messages come.
        ///
        /// When there are upstream messages, we will use the latest offsets from the upstream.
        latest_offset: String,
    },
    /// If there are no messages in the split at all, we don't need to start backfill.
    /// In this case, there will be no message from the backfill stream too.
    /// If we started backfill, we cannot finish it until new messages come.
    /// So we mark this a special case for optimization.
    NoDataToBackfill,
}

for_all_sources!(impl_connector_properties);

impl Default for ConnectorProperties {
    fn default() -> Self {
        ConnectorProperties::Test(Box::default())
    }
}

impl ConnectorProperties {
    pub fn is_new_fs_connector_hash_map(with_properties: &HashMap<String, String>) -> bool {
        with_properties
            .get(UPSTREAM_SOURCE_KEY)
            .map(|s| {
                s.eq_ignore_ascii_case(OPENDAL_S3_CONNECTOR)
                    || s.eq_ignore_ascii_case(POSIX_FS_CONNECTOR)
                    || s.eq_ignore_ascii_case(GCS_CONNECTOR)
                    || s.eq_ignore_ascii_case(AZBLOB_CONNECTOR)
            })
            .unwrap_or(false)
    }
}

impl ConnectorProperties {
    /// Creates typed source properties from the raw `WITH` properties.
    ///
    /// It checks the `connector` field, and them dispatches to the corresponding type's `try_from_btreemap` method.
    ///
    /// `deny_unknown_fields`: Since `WITH` options are persisted in meta, we do not deny unknown fields when restoring from
    /// existing data to avoid breaking backwards compatibility. We only deny unknown fields when creating new sources.
    pub fn extract(
        with_properties: WithOptionsSecResolved,
        deny_unknown_fields: bool,
    ) -> Result<Self> {
        let (options, secret_refs) = with_properties.into_parts();
        let mut options_with_secret =
            LocalSecretManager::global().fill_secrets(options, secret_refs)?;
        let connector = options_with_secret
            .remove(UPSTREAM_SOURCE_KEY)
            .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?;
        match_source_name_str!(
            connector.to_lowercase().as_str(),
            PropType,
            PropType::try_from_btreemap(options_with_secret, deny_unknown_fields)
                .map(ConnectorProperties::from),
            |other| bail!("connector '{}' is not supported", other)
        )
    }

    pub fn enable_split_scale_in(&self) -> bool {
        // enable split scale in just for Kinesis
        matches!(self, ConnectorProperties::Kinesis(_))
    }

    /// Load additional info from `PbSource`. Currently only used by CDC.
    pub fn init_from_pb_source(&mut self, source: &PbSource) {
        dispatch_source_prop!(self, prop, prop.init_from_pb_source(source))
    }

    /// Load additional info from `ExternalTableDesc`. Currently only used by CDC.
    pub fn init_from_pb_cdc_table_desc(&mut self, cdc_table_desc: &ExternalTableDesc) {
        dispatch_source_prop!(self, prop, prop.init_from_pb_cdc_table_desc(cdc_table_desc))
    }

    pub fn support_multiple_splits(&self) -> bool {
        matches!(self, ConnectorProperties::Kafka(_))
            || matches!(self, ConnectorProperties::OpendalS3(_))
            || matches!(self, ConnectorProperties::Gcs(_))
            || matches!(self, ConnectorProperties::Azblob(_))
    }
}

for_all_sources!(impl_split);

impl From<&SplitImpl> for ConnectorSplit {
    fn from(split: &SplitImpl) -> Self {
        dispatch_split_impl!(split, inner, SourcePropType, {
            ConnectorSplit {
                split_type: String::from(SourcePropType::SOURCE_NAME),
                encoded_split: inner.encode_to_bytes().to_vec(),
            }
        })
    }
}

impl TryFrom<&ConnectorSplit> for SplitImpl {
    type Error = crate::error::ConnectorError;

    fn try_from(split: &ConnectorSplit) -> std::result::Result<Self, Self::Error> {
        match_source_name_str!(
            split.split_type.to_lowercase().as_str(),
            PropType,
            {
                <PropType as SourceProperties>::Split::restore_from_bytes(
                    split.encoded_split.as_ref(),
                )
                .map(Into::into)
            },
            |other| bail!("connector '{}' is not supported", other)
        )
    }
}

impl SplitImpl {
    fn restore_from_json_inner(split_type: &str, value: JsonbVal) -> Result<Self> {
        match_source_name_str!(
            split_type.to_lowercase().as_str(),
            PropType,
            <PropType as SourceProperties>::Split::restore_from_json(value).map(Into::into),
            |other| bail!("connector '{}' is not supported", other)
        )
    }

    pub fn is_cdc_split(&self) -> bool {
        matches!(
            self,
            MysqlCdc(_) | PostgresCdc(_) | MongodbCdc(_) | CitusCdc(_) | SqlServerCdc(_)
        )
    }

    /// Get the current split offset.
    pub fn get_cdc_split_offset(&self) -> String {
        match self {
            MysqlCdc(split) => split.start_offset().clone().unwrap_or_default(),
            PostgresCdc(split) => split.start_offset().clone().unwrap_or_default(),
            MongodbCdc(split) => split.start_offset().clone().unwrap_or_default(),
            CitusCdc(split) => split.start_offset().clone().unwrap_or_default(),
            SqlServerCdc(split) => split.start_offset().clone().unwrap_or_default(),
            _ => unreachable!("get_cdc_split_offset() is only for cdc split"),
        }
    }
}

impl SplitMetaData for SplitImpl {
    fn id(&self) -> SplitId {
        dispatch_split_impl!(self, inner, IgnoreType, inner.id())
    }

    fn encode_to_json(&self) -> JsonbVal {
        use serde_json::json;
        let inner = self.encode_to_json_inner().take();
        json!({ SPLIT_TYPE_FIELD: self.get_type(), SPLIT_INFO_FIELD: inner}).into()
    }

    fn restore_from_json(value: JsonbVal) -> Result<Self> {
        let mut value = value.take();
        let json_obj = value.as_object_mut().unwrap();
        let split_type = json_obj
            .remove(SPLIT_TYPE_FIELD)
            .unwrap()
            .as_str()
            .unwrap()
            .to_string();
        let inner_value = json_obj.remove(SPLIT_INFO_FIELD).unwrap();
        Self::restore_from_json_inner(&split_type, inner_value.into())
    }

    fn update_offset(&mut self, last_seen_offset: String) -> Result<()> {
        dispatch_split_impl!(
            self,
            inner,
            IgnoreType,
            inner.update_offset(last_seen_offset)
        )
    }
}

impl SplitImpl {
    pub fn get_type(&self) -> String {
        dispatch_split_impl!(self, _ignored, PropType, {
            PropType::SOURCE_NAME.to_string()
        })
    }

    pub fn update_in_place(&mut self, last_seen_offset: String) -> Result<()> {
        dispatch_split_impl!(self, inner, IgnoreType, {
            inner.update_offset(last_seen_offset)?
        });
        Ok(())
    }

    pub fn encode_to_json_inner(&self) -> JsonbVal {
        dispatch_split_impl!(self, inner, IgnoreType, inner.encode_to_json())
    }
}

pub type DataType = risingwave_common::types::DataType;

#[derive(Clone, Debug)]
pub struct Column {
    pub name: String,
    pub data_type: DataType,
    /// This field is only used by datagen.
    pub is_visible: bool,
}

/// Split id resides in every source message, use `Arc` to avoid copying.
pub type SplitId = Arc<str>;

/// The message pumped from the external source service.
/// The third-party message structs will eventually be transformed into this struct.
#[derive(Debug, Clone)]
pub struct SourceMessage {
    pub key: Option<Vec<u8>>,
    pub payload: Option<Vec<u8>>,
    pub offset: String, // TODO: use `Arc<str>`
    pub split_id: SplitId,
    pub meta: SourceMeta,
}

impl SourceMessage {
    /// Create a dummy `SourceMessage` with all fields unset for testing purposes.
    pub fn dummy() -> Self {
        Self {
            key: None,
            payload: None,
            offset: "".to_string(),
            split_id: "".into(),
            meta: SourceMeta::Empty,
        }
    }
}

#[derive(Debug, Clone)]
pub enum SourceMeta {
    Kafka(KafkaMeta),
    Kinesis(KinesisMeta),
    Nexmark(NexmarkMeta),
    GooglePubsub(GooglePubsubMeta),
    Datagen(DatagenMeta),
    DebeziumCdc(DebeziumCdcMeta),
    // For the source that doesn't have meta data.
    Empty,
}

/// Implement Eq manually to ignore the `meta` field.
impl PartialEq for SourceMessage {
    fn eq(&self, other: &Self) -> bool {
        self.offset == other.offset
            && self.split_id == other.split_id
            && self.payload == other.payload
    }
}
impl Eq for SourceMessage {}

/// The metadata of a split.
pub trait SplitMetaData: Sized {
    fn id(&self) -> SplitId;
    fn encode_to_bytes(&self) -> Bytes {
        self.encode_to_json()
            .as_scalar_ref()
            .value_serialize()
            .into()
    }
    fn restore_from_bytes(bytes: &[u8]) -> Result<Self> {
        Self::restore_from_json(JsonbVal::value_deserialize(bytes).unwrap())
    }

    /// Encode the whole split metadata to a JSON object
    fn encode_to_json(&self) -> JsonbVal;
    fn restore_from_json(value: JsonbVal) -> Result<Self>;
    fn update_offset(&mut self, last_seen_offset: String) -> crate::error::ConnectorResult<()>;
}

/// [`ConnectorState`] maintains the consuming splits' info. In specific split readers,
/// `ConnectorState` cannot be [`None`] and contains one(for mq split readers) or many(for fs
/// split readers) [`SplitImpl`]. If no split is assigned to source executor, `ConnectorState` is
/// [`None`] and the created source stream will be a pending stream.
pub type ConnectorState = Option<Vec<SplitImpl>>;

#[derive(Debug, Clone, Default)]
pub struct FsFilterCtrlCtx;
pub type FsFilterCtrlCtxRef = Arc<FsFilterCtrlCtx>;

#[async_trait]
pub trait FsListInner: Sized {
    // fixme: better to implement as an Iterator, but the last page still have some contents
    async fn get_next_page<T: for<'a> From<&'a Object>>(&mut self) -> Result<(Vec<T>, bool)>;
    fn filter_policy(&self, ctx: &FsFilterCtrlCtx, page_num: usize, item: &FsPageItem) -> bool;
}

#[cfg(test)]
mod tests {
    use maplit::*;
    use nexmark::event::EventType;

    use super::*;
    use crate::source::cdc::{DebeziumCdcSplit, Mysql};
    use crate::source::kafka::KafkaSplit;

    #[test]
    fn test_split_impl_get_fn() -> Result<()> {
        let split = KafkaSplit::new(0, Some(0), Some(0), "demo".to_string());
        let split_impl = SplitImpl::Kafka(split.clone());
        let get_value = split_impl.into_kafka().unwrap();
        println!("{:?}", get_value);
        assert_eq!(split.encode_to_bytes(), get_value.encode_to_bytes());
        assert_eq!(split.encode_to_json(), get_value.encode_to_json());

        Ok(())
    }

    #[test]
    fn test_cdc_split_state() -> Result<()> {
        let offset_str = "{\"sourcePartition\":{\"server\":\"RW_CDC_mydb.products\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1670407377,\"file\":\"binlog.000001\",\"pos\":98587,\"row\":2,\"server_id\":1,\"event\":2}}";
        let split = DebeziumCdcSplit::<Mysql>::new(1001, Some(offset_str.to_string()), None);
        let split_impl = SplitImpl::MysqlCdc(split);
        let encoded_split = split_impl.encode_to_bytes();
        let restored_split_impl = SplitImpl::restore_from_bytes(encoded_split.as_ref())?;
        assert_eq!(
            split_impl.encode_to_bytes(),
            restored_split_impl.encode_to_bytes()
        );
        assert_eq!(
            split_impl.encode_to_json(),
            restored_split_impl.encode_to_json()
        );

        let encoded_split = split_impl.encode_to_json();
        let restored_split_impl = SplitImpl::restore_from_json(encoded_split)?;
        assert_eq!(
            split_impl.encode_to_bytes(),
            restored_split_impl.encode_to_bytes()
        );
        assert_eq!(
            split_impl.encode_to_json(),
            restored_split_impl.encode_to_json()
        );
        Ok(())
    }

    #[test]
    fn test_extract_nexmark_config() {
        let props = convert_args!(btreemap!(
            "connector" => "nexmark",
            "nexmark.table.type" => "Person",
            "nexmark.split.num" => "1",
        ));

        let props =
            ConnectorProperties::extract(WithOptionsSecResolved::without_secrets(props), true)
                .unwrap();

        if let ConnectorProperties::Nexmark(props) = props {
            assert_eq!(props.table_type, Some(EventType::Person));
            assert_eq!(props.split_num, 1);
        } else {
            panic!("extract nexmark config failed");
        }
    }

    #[test]
    fn test_extract_kafka_config() {
        let props = convert_args!(btreemap!(
            "connector" => "kafka",
            "properties.bootstrap.server" => "b1,b2",
            "topic" => "test",
            "scan.startup.mode" => "earliest",
            "broker.rewrite.endpoints" => r#"{"b-1:9092":"dns-1", "b-2:9092":"dns-2"}"#,
        ));

        let props =
            ConnectorProperties::extract(WithOptionsSecResolved::without_secrets(props), true)
                .unwrap();
        if let ConnectorProperties::Kafka(k) = props {
            let btreemap = btreemap! {
                "b-1:9092".to_string() => "dns-1".to_string(),
                "b-2:9092".to_string() => "dns-2".to_string(),
            };
            assert_eq!(k.privatelink_common.broker_rewrite_map, Some(btreemap));
        } else {
            panic!("extract kafka config failed");
        }
    }

    #[test]
    fn test_extract_cdc_properties() {
        let user_props_mysql = convert_args!(btreemap!(
            "connector" => "mysql-cdc",
            "database.hostname" => "127.0.0.1",
            "database.port" => "3306",
            "database.user" => "root",
            "database.password" => "123456",
            "database.name" => "mydb",
            "table.name" => "products",
        ));

        let user_props_postgres = convert_args!(btreemap!(
            "connector" => "postgres-cdc",
            "database.hostname" => "127.0.0.1",
            "database.port" => "5432",
            "database.user" => "root",
            "database.password" => "654321",
            "schema.name" => "public",
            "database.name" => "mypgdb",
            "table.name" => "orders",
        ));

        let conn_props = ConnectorProperties::extract(
            WithOptionsSecResolved::without_secrets(user_props_mysql),
            true,
        )
        .unwrap();
        if let ConnectorProperties::MysqlCdc(c) = conn_props {
            assert_eq!(c.properties.get("database.hostname").unwrap(), "127.0.0.1");
            assert_eq!(c.properties.get("database.port").unwrap(), "3306");
            assert_eq!(c.properties.get("database.user").unwrap(), "root");
            assert_eq!(c.properties.get("database.password").unwrap(), "123456");
            assert_eq!(c.properties.get("database.name").unwrap(), "mydb");
            assert_eq!(c.properties.get("table.name").unwrap(), "products");
        } else {
            panic!("extract cdc config failed");
        }

        let conn_props = ConnectorProperties::extract(
            WithOptionsSecResolved::without_secrets(user_props_postgres),
            true,
        )
        .unwrap();
        if let ConnectorProperties::PostgresCdc(c) = conn_props {
            assert_eq!(c.properties.get("database.hostname").unwrap(), "127.0.0.1");
            assert_eq!(c.properties.get("database.port").unwrap(), "5432");
            assert_eq!(c.properties.get("database.user").unwrap(), "root");
            assert_eq!(c.properties.get("database.password").unwrap(), "654321");
            assert_eq!(c.properties.get("schema.name").unwrap(), "public");
            assert_eq!(c.properties.get("database.name").unwrap(), "mypgdb");
            assert_eq!(c.properties.get("table.name").unwrap(), "orders");
        } else {
            panic!("extract cdc config failed");
        }
    }
}