Skip to content

Commit ccef1c5

Browse files
authored
feat(sink): enable kafka sink with format plain encode protobuf (#12858)
1 parent fa66cbd commit ccef1c5

File tree

12 files changed

+256
-17
lines changed

12 files changed

+256
-17
lines changed

ci/scripts/e2e-kafka-sink-test.sh

Lines changed: 7 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -131,3 +131,10 @@ sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/drop_sink.slt'
131131
./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only --delete > /dev/null 2>&1
132132
./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-upsert --delete > /dev/null 2>&1
133133
./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-debezium --delete > /dev/null 2>&1
134+
135+
# test different encoding
136+
echo "testing protobuf"
137+
cp src/connector/src/test_data/proto_recursive/recursive.pb ./proto-recursive
138+
./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only-protobuf --create > /dev/null 2>&1
139+
sqllogictest -p 4566 -d dev 'e2e_test/sink/kafka/protobuf.slt'
140+
./.risingwave/bin/kafka/bin/kafka-topics.sh --bootstrap-server 127.0.0.1:29092 --topic test-rw-sink-append-only-protobuf --delete > /dev/null 2>&1

ci/workflows/pull-request.yml

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -82,7 +82,7 @@ steps:
8282
config: ci/docker-compose.yml
8383
mount-buildkite-agent: true
8484
- ./ci/plugins/upload-failure-logs
85-
timeout_in_minutes: 15
85+
timeout_in_minutes: 17
8686
retry: *auto-retry
8787

8888
- label: "end-to-end test (parallel)"

e2e_test/sink/kafka/protobuf.slt

Lines changed: 97 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,97 @@
1+
statement ok
2+
create table from_kafka with (
3+
connector = 'kafka',
4+
topic = 'test-rw-sink-append-only-protobuf',
5+
properties.bootstrap.server = '127.0.0.1:29092')
6+
format plain encode protobuf (
7+
schema.location = 'file:///risingwave/proto-recursive',
8+
message = 'recursive.AllTypes');
9+
10+
statement ok
11+
create table into_kafka (
12+
bool_field bool,
13+
string_field varchar,
14+
bytes_field bytea,
15+
float_field real,
16+
double_field double precision,
17+
int32_field int,
18+
int64_field bigint,
19+
sint32_field int,
20+
sint64_field bigint,
21+
sfixed32_field int,
22+
sfixed64_field bigint,
23+
nested_message_field struct<id int, name varchar>,
24+
repeated_int_field int[],
25+
timestamp_field timestamptz,
26+
oneof_int32 int);
27+
28+
statement ok
29+
insert into into_kafka values
30+
(true, 'Rising', 'a0', 3.5, 4.25, 22, 23, 24, null, 26, 27, row(1, ''), array[4, 0, 4], '2006-01-02 15:04:05-07:00', 42),
31+
(false, 'Wave', 'ZDF', 1.5, null, 11, 12, 13, 14, 15, 16, row(4, 'foo'), null, null, null);
32+
33+
statement ok
34+
flush;
35+
36+
statement ok
37+
create sink sink0 from into_kafka with (
38+
connector = 'kafka',
39+
topic = 'test-rw-sink-append-only-protobuf',
40+
properties.bootstrap.server = '127.0.0.1:29092')
41+
format plain encode protobuf (
42+
force_append_only = true,
43+
schema.location = 'file:///risingwave/proto-recursive',
44+
message = 'recursive.AllTypes');
45+
46+
sleep 2s
47+
48+
query TTTRRIIIIIITTTI
49+
select
50+
bool_field,
51+
string_field,
52+
bytes_field,
53+
float_field,
54+
double_field,
55+
int32_field,
56+
int64_field,
57+
sint32_field,
58+
sint64_field,
59+
sfixed32_field,
60+
sfixed64_field,
61+
nested_message_field,
62+
repeated_int_field,
63+
timestamp_field,
64+
oneof_int32 from from_kafka;
65+
----
66+
t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,) {4,0,4} (1136239445,0) 42
67+
f Wave \x5a4446 1.5 0 11 12 13 14 15 16 (4,foo) {} (0,0) 0
68+
69+
statement error failed to read file
70+
create sink sink_err from into_kafka with (
71+
connector = 'kafka',
72+
topic = 'test-rw-sink-append-only-protobuf',
73+
properties.bootstrap.server = '127.0.0.1:29092')
74+
format plain encode protobuf (
75+
force_append_only = true,
76+
schema.location = 'file:///risingwave/proto-recursiv',
77+
message = 'recursive.AllTypes');
78+
79+
statement error encode extra_column error: field not in proto
80+
create sink sink_err as select 1 as extra_column with (
81+
connector = 'kafka',
82+
topic = 'test-rw-sink-append-only-protobuf',
83+
properties.bootstrap.server = '127.0.0.1:29092')
84+
format plain encode protobuf (
85+
force_append_only = true,
86+
schema.location = 'file:///risingwave/proto-recursive',
87+
message = 'recursive.AllTypes');
88+
89+
statement error s3 URL not supported yet
90+
create sink sink_err from into_kafka with (
91+
connector = 'kafka',
92+
topic = 'test-rw-sink-append-only-protobuf',
93+
properties.bootstrap.server = '127.0.0.1:29092')
94+
format plain encode protobuf (
95+
force_append_only = true,
96+
schema.location = 's3:///risingwave/proto-recursive',
97+
message = 'recursive.AllTypes');

src/connector/src/lib.rs

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -46,6 +46,7 @@ pub mod error;
4646
mod macros;
4747

4848
pub mod parser;
49+
pub mod schema;
4950
pub mod sink;
5051
pub mod source;
5152

src/connector/src/parser/protobuf/parser.rs

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -73,7 +73,7 @@ impl ProtobufAccessBuilder {
7373
#[derive(Debug, Clone)]
7474
pub struct ProtobufParserConfig {
7575
confluent_wire_type: bool,
76-
message_descriptor: MessageDescriptor,
76+
pub(crate) message_descriptor: MessageDescriptor,
7777
}
7878

7979
impl ProtobufParserConfig {

src/connector/src/schema/mod.rs

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,21 @@
1+
// Copyright 2023 RisingWave Labs
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
pub mod protobuf;
16+
17+
const MESSAGE_NAME_KEY: &str = "message";
18+
const SCHEMA_LOCATION_KEY: &str = "schema.location";
19+
20+
#[derive(Debug)]
21+
pub struct SchemaFetchError(pub String);

src/connector/src/schema/protobuf.rs

Lines changed: 57 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,57 @@
1+
// Copyright 2023 RisingWave Labs
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
use std::collections::BTreeMap;
16+
17+
use prost_reflect::MessageDescriptor;
18+
19+
use super::{SchemaFetchError, MESSAGE_NAME_KEY, SCHEMA_LOCATION_KEY};
20+
use crate::aws_auth::AwsAuthProps;
21+
use crate::parser::{EncodingProperties, ProtobufParserConfig, ProtobufProperties};
22+
23+
/// `aws_auth_props` is only required when reading `s3://` URL.
24+
pub async fn fetch_descriptor(
25+
format_options: &BTreeMap<String, String>,
26+
aws_auth_props: Option<&AwsAuthProps>,
27+
) -> Result<MessageDescriptor, SchemaFetchError> {
28+
let row_schema_location = format_options
29+
.get(SCHEMA_LOCATION_KEY)
30+
.ok_or_else(|| SchemaFetchError(format!("{SCHEMA_LOCATION_KEY} required")))?
31+
.clone();
32+
let message_name = format_options
33+
.get(MESSAGE_NAME_KEY)
34+
.ok_or_else(|| SchemaFetchError(format!("{MESSAGE_NAME_KEY} required")))?
35+
.clone();
36+
37+
if row_schema_location.starts_with("s3") && aws_auth_props.is_none() {
38+
return Err(SchemaFetchError("s3 URL not supported yet".into()));
39+
}
40+
41+
let enc = EncodingProperties::Protobuf(ProtobufProperties {
42+
use_schema_registry: false,
43+
row_schema_location,
44+
message_name,
45+
aws_auth_props: aws_auth_props.cloned(),
46+
// name_strategy, topic, key_message_name, enable_upsert, client_config
47+
..Default::default()
48+
});
49+
// Ideally, we should extract the schema loading logic from source parser to this place,
50+
// and call this in both source and sink.
51+
// But right now this function calls into source parser for its schema loading functionality.
52+
// This reversed dependency will be fixed when we support schema registry.
53+
let conf = ProtobufParserConfig::new(enc)
54+
.await
55+
.map_err(|e| SchemaFetchError(e.to_string()))?;
56+
Ok(conf.message_descriptor)
57+
}

src/connector/src/sink/formatter/mod.rs

Lines changed: 37 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -28,7 +28,7 @@ pub use upsert::UpsertFormatter;
2828

2929
use super::catalog::{SinkEncode, SinkFormat, SinkFormatDesc};
3030
use super::encoder::KafkaConnectParams;
31-
use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode};
31+
use crate::sink::encoder::{JsonEncoder, ProtoEncoder, TimestampHandlingMode};
3232

3333
/// Transforms a `StreamChunk` into a sequence of key-value pairs according a specific format,
3434
/// for example append-only, upsert or debezium.
@@ -61,27 +61,28 @@ macro_rules! tri {
6161
};
6262
}
6363

64-
#[expect(clippy::enum_variant_names)]
6564
pub enum SinkFormatterImpl {
6665
AppendOnlyJson(AppendOnlyFormatter<JsonEncoder, JsonEncoder>),
66+
AppendOnlyProto(AppendOnlyFormatter<JsonEncoder, ProtoEncoder>),
6767
UpsertJson(UpsertFormatter<JsonEncoder, JsonEncoder>),
6868
DebeziumJson(DebeziumJsonFormatter),
6969
}
7070

7171
impl SinkFormatterImpl {
72-
pub fn new(
72+
pub async fn new(
7373
format_desc: &SinkFormatDesc,
7474
schema: Schema,
7575
pk_indices: Vec<usize>,
7676
db_name: String,
7777
sink_from_name: String,
7878
) -> Result<Self> {
79-
if format_desc.encode != SinkEncode::Json {
80-
return Err(SinkError::Config(anyhow!(
81-
"sink encode unsupported: {:?}",
79+
let err_unsupported = || {
80+
Err(SinkError::Config(anyhow!(
81+
"sink format/encode unsupported: {:?} {:?}",
82+
format_desc.format,
8283
format_desc.encode,
83-
)));
84-
}
84+
)))
85+
};
8586

8687
match format_desc.format {
8788
SinkFormat::AppendOnly => {
@@ -92,12 +93,32 @@ impl SinkFormatterImpl {
9293
TimestampHandlingMode::Milli,
9394
)
9495
});
95-
let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli);
9696

97-
let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder);
98-
Ok(SinkFormatterImpl::AppendOnlyJson(formatter))
97+
match format_desc.encode {
98+
SinkEncode::Json => {
99+
let val_encoder =
100+
JsonEncoder::new(schema, None, TimestampHandlingMode::Milli);
101+
let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder);
102+
Ok(SinkFormatterImpl::AppendOnlyJson(formatter))
103+
}
104+
SinkEncode::Protobuf => {
105+
// By passing `None` as `aws_auth_props`, reading from `s3://` not supported yet.
106+
let descriptor =
107+
crate::schema::protobuf::fetch_descriptor(&format_desc.options, None)
108+
.await
109+
.map_err(|e| SinkError::Config(anyhow!("{e:?}")))?;
110+
let val_encoder = ProtoEncoder::new(schema, None, descriptor)?;
111+
let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder);
112+
Ok(SinkFormatterImpl::AppendOnlyProto(formatter))
113+
}
114+
SinkEncode::Avro => err_unsupported(),
115+
}
99116
}
100117
SinkFormat::Debezium => {
118+
if format_desc.encode != SinkEncode::Json {
119+
return err_unsupported();
120+
}
121+
101122
Ok(SinkFormatterImpl::DebeziumJson(DebeziumJsonFormatter::new(
102123
schema,
103124
pk_indices,
@@ -107,6 +128,10 @@ impl SinkFormatterImpl {
107128
)))
108129
}
109130
SinkFormat::Upsert => {
131+
if format_desc.encode != SinkEncode::Json {
132+
return err_unsupported();
133+
}
134+
110135
let mut key_encoder = JsonEncoder::new(
111136
schema.clone(),
112137
Some(pk_indices),
@@ -146,6 +171,7 @@ macro_rules! dispatch_sink_formatter_impl {
146171
($impl:expr, $name:ident, $body:expr) => {
147172
match $impl {
148173
SinkFormatterImpl::AppendOnlyJson($name) => $body,
174+
SinkFormatterImpl::AppendOnlyProto($name) => $body,
149175
SinkFormatterImpl::UpsertJson($name) => $body,
150176
SinkFormatterImpl::DebeziumJson($name) => $body,
151177
}

src/connector/src/sink/kafka.rs

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -314,7 +314,8 @@ impl Sink for KafkaSink {
314314
self.pk_indices.clone(),
315315
self.db_name.clone(),
316316
self.sink_from_name.clone(),
317-
)?;
317+
)
318+
.await?;
318319
KafkaLogSinker::new(self.config.clone(), formatter).await
319320
}
320321

@@ -326,6 +327,15 @@ impl Sink for KafkaSink {
326327
self.format_desc.format
327328
)));
328329
}
330+
// Check for formatter constructor error, before it is too late for error reporting.
331+
SinkFormatterImpl::new(
332+
&self.format_desc,
333+
self.schema.clone(),
334+
self.pk_indices.clone(),
335+
self.db_name.clone(),
336+
self.sink_from_name.clone(),
337+
)
338+
.await?;
329339

330340
// Try Kafka connection.
331341
// There is no such interface for kafka producer to validate a connection

src/connector/src/sink/kinesis.rs

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -79,6 +79,15 @@ impl Sink for KinesisSink {
7979
"kinesis sink requires partition key (please define in `primary_key` field)",
8080
)));
8181
}
82+
// Check for formatter constructor error, before it is too late for error reporting.
83+
SinkFormatterImpl::new(
84+
&self.format_desc,
85+
self.schema.clone(),
86+
self.pk_indices.clone(),
87+
self.db_name.clone(),
88+
self.sink_from_name.clone(),
89+
)
90+
.await?;
8291

8392
// check reachability
8493
let client = self.config.common.build_client().await?;
@@ -145,7 +154,8 @@ impl KinesisSinkWriter {
145154
sink_from_name: String,
146155
) -> Result<Self> {
147156
let formatter =
148-
SinkFormatterImpl::new(format_desc, schema, pk_indices, db_name, sink_from_name)?;
157+
SinkFormatterImpl::new(format_desc, schema, pk_indices, db_name, sink_from_name)
158+
.await?;
149159
let client = config
150160
.common
151161
.build_client()

src/connector/src/sink/pulsar.rs

Lines changed: 11 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -180,6 +180,15 @@ impl Sink for PulsarSink {
180180
self.format_desc.format
181181
)));
182182
}
183+
// Check for formatter constructor error, before it is too late for error reporting.
184+
SinkFormatterImpl::new(
185+
&self.format_desc,
186+
self.schema.clone(),
187+
self.downstream_pk.clone(),
188+
self.db_name.clone(),
189+
self.sink_from_name.clone(),
190+
)
191+
.await?;
183192

184193
// Validate pulsar connection.
185194
let pulsar = self.config.common.build_client().await?;
@@ -211,7 +220,8 @@ impl PulsarSinkWriter {
211220
sink_from_name: String,
212221
) -> Result<Self> {
213222
let formatter =
214-
SinkFormatterImpl::new(format_desc, schema, downstream_pk, db_name, sink_from_name)?;
223+
SinkFormatterImpl::new(format_desc, schema, downstream_pk, db_name, sink_from_name)
224+
.await?;
215225
let pulsar = config.common.build_client().await?;
216226
let producer = build_pulsar_producer(&pulsar, &config).await?;
217227
Ok(Self {

0 commit comments

Comments
 (0)