Skip to content

Commit

Permalink
test(source): add csv format test for Kafka (risingwavelabs#9875)
Browse files Browse the repository at this point in the history
  • Loading branch information
lmatz authored May 18, 2023
1 parent 05113ae commit ca18ca8
Show file tree
Hide file tree
Showing 5 changed files with 47 additions and 8 deletions.
25 changes: 25 additions & 0 deletions e2e_test/source/basic/kafka.slt
Original file line number Diff line number Diff line change
@@ -1,3 +1,20 @@
# We don't support CSV header for Kafka
statement error CSV HEADER is not supported when creating table with Kafka connector
create table s0 (v1 int, v2 varchar) with (
connector = 'kafka',
topic = 'kafka_1_csv_topic',
properties.bootstrap.server = '127.0.0.1:29092',
scan.startup.mode = 'earliest'
) row format csv DELIMITED BY ',';

statement ok
create table s0 (v1 int, v2 varchar) with (
connector = 'kafka',
topic = 'kafka_1_csv_topic',
properties.bootstrap.server = '127.0.0.1:29092',
scan.startup.mode = 'earliest'
) row format csv WITHOUT HEADER DELIMITED BY ',';

# expect fail with invalid broker address
statement error
create table s1 (v1 int, v2 varchar) with (
Expand Down Expand Up @@ -322,6 +339,14 @@ flush;
# Wait enough time to ensure SourceExecutor consumes all Kafka data.
sleep 1s

query IT rowsort
select v1, v2 from s0;
----
1 1
2 22
3 333
4 4444

query IT rowsort
select * from s1
----
Expand Down
5 changes: 5 additions & 0 deletions scripts/source/test_data/kafka_1_csv_topic.1
Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
v1,v2
1,"1"
2,"22"
3,"333"
4,"4444"
2 changes: 1 addition & 1 deletion src/connector/src/parser/csv_parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ pub struct CsvParserConfig {
pub has_header: bool,
}

/// Parser for JSON format
/// Parser for CSV format
#[derive(Debug)]
pub struct CsvParser {
rw_columns: Vec<SourceColumnDesc>,
Expand Down
22 changes: 15 additions & 7 deletions src/frontend/src/handler/create_source.rs
Original file line number Diff line number Diff line change
Expand Up @@ -423,12 +423,20 @@ pub(crate) async fn resolve_source_schema(
}
}

SourceSchema::Csv(csv_info) => StreamSourceInfo {
row_format: RowFormatType::Csv as i32,
csv_delimiter: csv_info.delimiter as i32,
csv_has_header: csv_info.has_header,
..Default::default()
},
SourceSchema::Csv(csv_info) => {
if is_kafka && csv_info.has_header {
return Err(RwError::from(ProtocolError(
"CSV HEADER is not supported when creating table with Kafka connector"
.to_owned(),
)));
}
StreamSourceInfo {
row_format: RowFormatType::Csv as i32,
csv_delimiter: csv_info.delimiter as i32,
csv_has_header: csv_info.has_header,
..Default::default()
}
}

SourceSchema::Native => StreamSourceInfo {
row_format: RowFormatType::Native as i32,
Expand Down Expand Up @@ -559,7 +567,7 @@ pub(super) fn bind_source_watermark(
static CONNECTORS_COMPATIBLE_FORMATS: LazyLock<HashMap<String, Vec<RowFormatType>>> = LazyLock::new(
|| {
convert_args!(hashmap!(
KAFKA_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson, RowFormatType::DebeziumAvro,RowFormatType::DebeziumMongoJson, RowFormatType::UpsertJson, RowFormatType::UpsertAvro],
KAFKA_CONNECTOR => vec![RowFormatType::Csv, RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson, RowFormatType::DebeziumAvro,RowFormatType::DebeziumMongoJson, RowFormatType::UpsertJson, RowFormatType::UpsertAvro],
PULSAR_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson],
KINESIS_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson],
GOOGLE_PUBSUB_CONNECTOR => vec![RowFormatType::Json, RowFormatType::Protobuf, RowFormatType::DebeziumJson, RowFormatType::Avro, RowFormatType::Maxwell, RowFormatType::CanalJson],
Expand Down
1 change: 1 addition & 0 deletions src/source/src/source_desc.rs
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ impl SourceDescBuilder {
PbRowFormatType::UpsertJson => SourceFormat::UpsertJson,
PbRowFormatType::UpsertAvro => SourceFormat::UpsertAvro,
PbRowFormatType::DebeziumMongoJson => SourceFormat::DebeziumMongoJson,
PbRowFormatType::Csv => SourceFormat::Csv,
_ => unreachable!(),
};

Expand Down

0 comments on commit ca18ca8

Please sign in to comment.