-
Notifications
You must be signed in to change notification settings - Fork 592
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
feat(sink): enable kafka sink with
format plain encode protobuf
(#1…
- Loading branch information
1 parent
fa66cbd
commit ccef1c5
Showing
12 changed files
with
256 additions
and
17 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,97 @@ | ||
statement ok | ||
create table from_kafka with ( | ||
connector = 'kafka', | ||
topic = 'test-rw-sink-append-only-protobuf', | ||
properties.bootstrap.server = '127.0.0.1:29092') | ||
format plain encode protobuf ( | ||
schema.location = 'file:///risingwave/proto-recursive', | ||
message = 'recursive.AllTypes'); | ||
|
||
statement ok | ||
create table into_kafka ( | ||
bool_field bool, | ||
string_field varchar, | ||
bytes_field bytea, | ||
float_field real, | ||
double_field double precision, | ||
int32_field int, | ||
int64_field bigint, | ||
sint32_field int, | ||
sint64_field bigint, | ||
sfixed32_field int, | ||
sfixed64_field bigint, | ||
nested_message_field struct<id int, name varchar>, | ||
repeated_int_field int[], | ||
timestamp_field timestamptz, | ||
oneof_int32 int); | ||
|
||
statement ok | ||
insert into into_kafka values | ||
(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), | ||
(false, 'Wave', 'ZDF', 1.5, null, 11, 12, 13, 14, 15, 16, row(4, 'foo'), null, null, null); | ||
|
||
statement ok | ||
flush; | ||
|
||
statement ok | ||
create sink sink0 from into_kafka with ( | ||
connector = 'kafka', | ||
topic = 'test-rw-sink-append-only-protobuf', | ||
properties.bootstrap.server = '127.0.0.1:29092') | ||
format plain encode protobuf ( | ||
force_append_only = true, | ||
schema.location = 'file:///risingwave/proto-recursive', | ||
message = 'recursive.AllTypes'); | ||
|
||
sleep 2s | ||
|
||
query TTTRRIIIIIITTTI | ||
select | ||
bool_field, | ||
string_field, | ||
bytes_field, | ||
float_field, | ||
double_field, | ||
int32_field, | ||
int64_field, | ||
sint32_field, | ||
sint64_field, | ||
sfixed32_field, | ||
sfixed64_field, | ||
nested_message_field, | ||
repeated_int_field, | ||
timestamp_field, | ||
oneof_int32 from from_kafka; | ||
---- | ||
t Rising \x6130 3.5 4.25 22 23 24 0 26 27 (1,) {4,0,4} (1136239445,0) 42 | ||
f Wave \x5a4446 1.5 0 11 12 13 14 15 16 (4,foo) {} (0,0) 0 | ||
|
||
statement error failed to read file | ||
create sink sink_err from into_kafka with ( | ||
connector = 'kafka', | ||
topic = 'test-rw-sink-append-only-protobuf', | ||
properties.bootstrap.server = '127.0.0.1:29092') | ||
format plain encode protobuf ( | ||
force_append_only = true, | ||
schema.location = 'file:///risingwave/proto-recursiv', | ||
message = 'recursive.AllTypes'); | ||
|
||
statement error encode extra_column error: field not in proto | ||
create sink sink_err as select 1 as extra_column with ( | ||
connector = 'kafka', | ||
topic = 'test-rw-sink-append-only-protobuf', | ||
properties.bootstrap.server = '127.0.0.1:29092') | ||
format plain encode protobuf ( | ||
force_append_only = true, | ||
schema.location = 'file:///risingwave/proto-recursive', | ||
message = 'recursive.AllTypes'); | ||
|
||
statement error s3 URL not supported yet | ||
create sink sink_err from into_kafka with ( | ||
connector = 'kafka', | ||
topic = 'test-rw-sink-append-only-protobuf', | ||
properties.bootstrap.server = '127.0.0.1:29092') | ||
format plain encode protobuf ( | ||
force_append_only = true, | ||
schema.location = 's3:///risingwave/proto-recursive', | ||
message = 'recursive.AllTypes'); |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -46,6 +46,7 @@ pub mod error; | |
mod macros; | ||
|
||
pub mod parser; | ||
pub mod schema; | ||
pub mod sink; | ||
pub mod source; | ||
|
||
|
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,21 @@ | ||
// Copyright 2023 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. | ||
|
||
pub mod protobuf; | ||
|
||
const MESSAGE_NAME_KEY: &str = "message"; | ||
const SCHEMA_LOCATION_KEY: &str = "schema.location"; | ||
|
||
#[derive(Debug)] | ||
pub struct SchemaFetchError(pub String); |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,57 @@ | ||
// Copyright 2023 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; | ||
|
||
use prost_reflect::MessageDescriptor; | ||
|
||
use super::{SchemaFetchError, MESSAGE_NAME_KEY, SCHEMA_LOCATION_KEY}; | ||
use crate::aws_auth::AwsAuthProps; | ||
use crate::parser::{EncodingProperties, ProtobufParserConfig, ProtobufProperties}; | ||
|
||
/// `aws_auth_props` is only required when reading `s3://` URL. | ||
pub async fn fetch_descriptor( | ||
format_options: &BTreeMap<String, String>, | ||
aws_auth_props: Option<&AwsAuthProps>, | ||
) -> Result<MessageDescriptor, SchemaFetchError> { | ||
let row_schema_location = format_options | ||
.get(SCHEMA_LOCATION_KEY) | ||
.ok_or_else(|| SchemaFetchError(format!("{SCHEMA_LOCATION_KEY} required")))? | ||
.clone(); | ||
let message_name = format_options | ||
.get(MESSAGE_NAME_KEY) | ||
.ok_or_else(|| SchemaFetchError(format!("{MESSAGE_NAME_KEY} required")))? | ||
.clone(); | ||
|
||
if row_schema_location.starts_with("s3") && aws_auth_props.is_none() { | ||
return Err(SchemaFetchError("s3 URL not supported yet".into())); | ||
} | ||
|
||
let enc = EncodingProperties::Protobuf(ProtobufProperties { | ||
use_schema_registry: false, | ||
row_schema_location, | ||
message_name, | ||
aws_auth_props: aws_auth_props.cloned(), | ||
// name_strategy, topic, key_message_name, enable_upsert, client_config | ||
..Default::default() | ||
}); | ||
// Ideally, we should extract the schema loading logic from source parser to this place, | ||
// and call this in both source and sink. | ||
// But right now this function calls into source parser for its schema loading functionality. | ||
// This reversed dependency will be fixed when we support schema registry. | ||
let conf = ProtobufParserConfig::new(enc) | ||
.await | ||
.map_err(|e| SchemaFetchError(e.to_string()))?; | ||
Ok(conf.message_descriptor) | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.