Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat(sink): add Google Pub/Sub support #16363

Merged
merged 10 commits into from
May 8, 2024
2 changes: 1 addition & 1 deletion src/connector/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ gcp-bigquery-client = "0.18.0"
glob = "0.3"
google-cloud-bigquery = { version = "0.8.0", features = ["auth"] }
google-cloud-gax = "0.17.0"
google-cloud-googleapis = "0.12.0"
google-cloud-googleapis = { version = "0.12", features = ["pubsub"] }
google-cloud-pubsub = "0.24"
Comment on lines +63 to 64
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is there any overlap between the deps? Shall we just keep one of them?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, google-cloud-pubsub depends on google-cloud-googleapis but does not re-export, which is required for publishing messages.

http = "0.2"
icelake = { workspace = true }
Expand Down
293 changes: 293 additions & 0 deletions src/connector/src/sink/google_pubsub.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,293 @@
// 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::HashMap;
use std::usize;

use anyhow::{anyhow, Context};
use google_cloud_gax::conn::Environment;
use google_cloud_googleapis::pubsub::v1::PubsubMessage;
use google_cloud_pubsub::apiv1;
use google_cloud_pubsub::client::google_cloud_auth::credentials::CredentialsFile;
use google_cloud_pubsub::client::google_cloud_auth::project;
use google_cloud_pubsub::client::google_cloud_auth::token::DefaultTokenSourceProvider;
use google_cloud_pubsub::client::{Client, ClientConfig};
use google_cloud_pubsub::publisher::Publisher;
use risingwave_common::array::StreamChunk;
use risingwave_common::catalog::Schema;
use risingwave_common::session_config::sink_decouple::SinkDecouple;
use serde_derive::Deserialize;
use serde_with::serde_as;
use tonic::Status;
use with_options::WithOptions;

use super::catalog::desc::SinkDesc;
use super::catalog::SinkFormatDesc;
use super::formatter::SinkFormatterImpl;
use super::log_store::DeliveryFutureManagerAddFuture;
use super::writer::{
AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink,
};
use super::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkParam, SinkWriterParam};
use crate::dispatch_sink_formatter_str_key_impl;

pub const PUBSUB_SINK: &str = "google_pubsub";

#[serde_as]
#[derive(Clone, Debug, Deserialize, WithOptions)]
pub struct GooglePubSubConfig {
/// The Google Pub/Sub Project ID
#[serde(rename = "pubsub.project_id")]
pub project_id: String,

/// Specifies the Pub/Sub topic to publish messages
#[serde(rename = "pubsub.topic")]
pub topic: String,

/// The Google Pub/Sub endpoint URL
#[serde(rename = "pubsub.endpoint")]
pub endpoint: String,

/// use the connector with a pubsub emulator
/// <https://cloud.google.com/pubsub/docs/emulator>
#[serde(rename = "pubsub.emulator_host")]
pub emulator_host: Option<String>,

/// A JSON string containing the service account credentials for authorization,
/// see the [service-account](https://developers.google.com/workspace/guides/create-credentials#create_credentials_for_a_service_account) credentials guide.
/// The provided account credential must have the
/// `pubsub.publisher` [role](https://cloud.google.com/pubsub/docs/access-control#roles)
#[serde(rename = "pubsub.credentials")]
pub credentials: Option<String>,

// accept "append-only"
pub r#type: String,
}

impl GooglePubSubConfig {
fn from_hashmap(values: HashMap<String, String>) -> Result<Self> {
serde_json::from_value::<GooglePubSubConfig>(serde_json::to_value(values).unwrap())
.map_err(|e| SinkError::Config(anyhow!(e)))
}
}

#[derive(Clone, Debug)]
pub struct GooglePubSubSink {
pub config: GooglePubSubConfig,
is_append_only: bool,

schema: Schema,
pk_indices: Vec<usize>,
format_desc: SinkFormatDesc,
db_name: String,
sink_from_name: String,
}

impl Sink for GooglePubSubSink {
type Coordinator = DummySinkCommitCoordinator;
type LogSinker = AsyncTruncateLogSinkerOf<GooglePubSubSinkWriter>;

const SINK_NAME: &'static str = PUBSUB_SINK;

fn is_sink_decouple(desc: &SinkDesc, user_specified: &SinkDecouple) -> Result<bool> {
match user_specified {
SinkDecouple::Default => Ok(desc.sink_type.is_append_only()),
SinkDecouple::Disable => Ok(false),
SinkDecouple::Enable => Ok(true),
}
}

async fn validate(&self) -> Result<()> {
if !self.is_append_only {
return Err(SinkError::GooglePubSub(anyhow!(
"Google Pub/Sub sink only support append-only mode"
)));
}

let conf = &self.config;
if matches!((&conf.emulator_host, &conf.credentials), (None, None)) {
return Err(SinkError::GooglePubSub(anyhow!(
"Configure at least one of `pubsub.emulator_host` and `pubsub.credentials` in the Google Pub/Sub sink"
)));
}

Ok(())
}

async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result<Self::LogSinker> {
Ok(GooglePubSubSinkWriter::new(
self.config.clone(),
self.schema.clone(),
self.pk_indices.clone(),
&self.format_desc,
self.db_name.clone(),
self.sink_from_name.clone(),
)
.await?
.into_log_sinker(usize::MAX))
}
}

impl TryFrom<SinkParam> for GooglePubSubSink {
type Error = SinkError;

fn try_from(param: SinkParam) -> std::result::Result<Self, Self::Error> {
let schema = param.schema();
let config = GooglePubSubConfig::from_hashmap(param.properties)?;

let format_desc = param
.format_desc
.ok_or_else(|| SinkError::Config(anyhow!("missing FORMAT ... ENCODE ...")))?;
Ok(Self {
config,
is_append_only: param.sink_type.is_append_only(),

schema,
pk_indices: param.downstream_pk,
format_desc,
db_name: param.db_name,
sink_from_name: param.sink_from_name,
})
}
}

struct GooglePubSubPayloadWriter {
publisher: Publisher,
}

impl GooglePubSubSinkWriter {
pub async fn new(
config: GooglePubSubConfig,
schema: Schema,
pk_indices: Vec<usize>,
format_desc: &SinkFormatDesc,
db_name: String,
sink_from_name: String,
) -> Result<Self> {
let environment = if let Some(ref cred) = config.credentials {
let auth_config = project::Config {
audience: Some(apiv1::conn_pool::AUDIENCE),
scopes: Some(&apiv1::conn_pool::SCOPES),
sub: None,
};
let cred_file = CredentialsFile::new_from_str(cred).await.map_err(|e| {
SinkError::GooglePubSub(anyhow!(
"Failed to create Google Cloud Pub/Sub credentials file: {}",
e
))
})?;
let provider =
DefaultTokenSourceProvider::new_with_credentials(auth_config, Box::new(cred_file))
.await
.map_err(|e| {
SinkError::GooglePubSub(anyhow!(
"Failed to create Google Cloud Pub/Sub token source provider: {}",
e
))
})?;
Environment::GoogleCloud(Box::new(provider))
} else if let Some(emu_host) = config.emulator_host {
Environment::Emulator(emu_host)
} else {
return Err(SinkError::GooglePubSub(anyhow!(
"Missing emulator_host or credentials in Google Pub/Sub sink"
)));
};

let client_config = ClientConfig {
endpoint: config.endpoint,
project_id: Some(config.project_id),
environment,
..Default::default()
};
let client = Client::new(client_config)
.await
.map_err(|e| SinkError::GooglePubSub(anyhow!(e)))?;

let topic = async {
let topic = client.topic(&config.topic);
if !topic.exists(None).await? {
topic.create(None, None).await?;
}
Ok(topic)
}
.await
.map_err(|e: Status| SinkError::GooglePubSub(anyhow!(e)))?;

let formatter = SinkFormatterImpl::new(
format_desc,
schema,
pk_indices,
db_name,
sink_from_name,
topic.fully_qualified_name(),
)
.await?;

let publisher = topic.new_publisher(None);
let payload_writer = GooglePubSubPayloadWriter { publisher };

Ok(Self {
payload_writer,
formatter,
})
}
}

pub struct GooglePubSubSinkWriter {
payload_writer: GooglePubSubPayloadWriter,
formatter: SinkFormatterImpl,
}

impl AsyncTruncateSinkWriter for GooglePubSubSinkWriter {
async fn write_chunk<'a>(
&'a mut self,
chunk: StreamChunk,
_add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>,
) -> Result<()> {
dispatch_sink_formatter_str_key_impl!(
&self.formatter,
formatter,
self.payload_writer.write_chunk(chunk, formatter).await
)
}
}

impl FormattedSink for GooglePubSubPayloadWriter {
type K = String;
type V = Vec<u8>;

async fn write_one(&mut self, k: Option<Self::K>, v: Option<Self::V>) -> Result<()> {
let ordering_key = k.unwrap_or_default();
match v {
Some(data) => {
let msg = PubsubMessage {
data,
ordering_key,
..Default::default()
};
let awaiter = self.publisher.publish(msg).await;
awaiter
.get()
.await
.context("Google Pub/Sub sink error")
.map_err(SinkError::GooglePubSub)
.map(|_| ())
}
None => Err(SinkError::GooglePubSub(anyhow!(
"Google Pub/Sub sink error: missing value to publish"
))),
}
}
}
8 changes: 8 additions & 0 deletions src/connector/src/sink/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ pub mod doris_starrocks_connector;
pub mod elasticsearch;
pub mod encoder;
pub mod formatter;
pub mod google_pubsub;
pub mod iceberg;
pub mod kafka;
pub mod kinesis;
Expand Down Expand Up @@ -86,6 +87,7 @@ macro_rules! for_all_sinks {
{ ClickHouse, $crate::sink::clickhouse::ClickHouseSink },
{ Iceberg, $crate::sink::iceberg::IcebergSink },
{ Mqtt, $crate::sink::mqtt::MqttSink },
{ GooglePubSub, $crate::sink::google_pubsub::GooglePubSubSink },
{ Nats, $crate::sink::nats::NatsSink },
{ Jdbc, $crate::sink::remote::JdbcSink },
{ ElasticSearch, $crate::sink::remote::ElasticSearchSink },
Expand Down Expand Up @@ -525,6 +527,12 @@ pub enum SinkError {
#[backtrace]
anyhow::Error,
),
#[error("Google Pub/Sub error: {0}")]
GooglePubSub(
#[source]
#[backtrace]
anyhow::Error,
),
#[error("Doris/Starrocks connect error: {0}")]
DorisStarrocksConnect(
#[source]
Expand Down
25 changes: 25 additions & 0 deletions src/connector/with_options_sink.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -113,6 +113,31 @@ DorisConfig:
- name: r#type
field_type: String
required: true
GooglePubSubConfig:
fields:
- name: pubsub.project_id
field_type: String
comments: The Google Pub/Sub Project ID
required: true
- name: pubsub.topic
field_type: String
comments: Specifies the Pub/Sub topic to publish messages
required: true
- name: pubsub.endpoint
field_type: String
comments: The Google Pub/Sub endpoint URL
required: true
- name: pubsub.emulator_host
field_type: String
comments: use the connector with a pubsub emulator <https://cloud.google.com/pubsub/docs/emulator>
required: false
- name: pubsub.credentials
field_type: String
comments: A JSON string containing the service account credentials for authorization, see the [service-account](https://developers.google.com/workspace/guides/create-credentials#create_credentials_for_a_service_account) credentials guide. The provided account credential must have the `pubsub.publisher` [role](https://cloud.google.com/pubsub/docs/access-control#roles)
required: false
- name: r#type
field_type: String
required: true
IcebergConfig:
fields:
- name: connector
Expand Down
8 changes: 6 additions & 2 deletions src/frontend/src/handler/create_sink.rs
Original file line number Diff line number Diff line change
Expand Up @@ -736,6 +736,7 @@ fn bind_sink_format_desc(value: ConnectorSchema) -> Result<SinkFormatDesc> {

static CONNECTORS_COMPATIBLE_FORMATS: LazyLock<HashMap<String, HashMap<Format, Vec<Encode>>>> =
LazyLock::new(|| {
use risingwave_connector::sink::google_pubsub::GooglePubSubSink;
use risingwave_connector::sink::kafka::KafkaSink;
use risingwave_connector::sink::kinesis::KinesisSink;
use risingwave_connector::sink::mqtt::MqttSink;
Expand All @@ -744,6 +745,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock<HashMap<String, HashMap<Format, V
use risingwave_connector::sink::Sink as _;

convert_args!(hashmap!(
GooglePubSubSink::SINK_NAME => hashmap!(
Format::Plain => vec![Encode::Json],
),
KafkaSink::SINK_NAME => hashmap!(
Format::Plain => vec![Encode::Json, Encode::Protobuf],
Format::Upsert => vec![Encode::Json, Encode::Avro],
Expand All @@ -763,8 +767,8 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock<HashMap<String, HashMap<Format, V
Format::Debezium => vec![Encode::Json],
),
RedisSink::SINK_NAME => hashmap!(
Format::Plain => vec![Encode::Json,Encode::Template],
Format::Upsert => vec![Encode::Json,Encode::Template],
Format::Plain => vec![Encode::Json, Encode::Template],
Format::Upsert => vec![Encode::Json, Encode::Template],
),
))
});
Expand Down
Loading