-
Notifications
You must be signed in to change notification settings - Fork 595
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
Changes from 9 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
35e690a
feat(sink): add Google Pub/Sub sink
jetjinser 25272cd
feat(sink): make google_pubsub sink config more consistent
jetjinser 986a7a1
test: update `with_options_sink` test for google pubsub sink
jetjinser c02bfb0
feat(sink): use formatter for google pubsub sink
jetjinser aa85312
chore: remove unnecessary comments
jetjinser 164a18b
fix: change error type to GooglePubSub for Google Pub/Sub sink
jetjinser aeeb127
refactor: remove encode check for google pubsub sink
jetjinser 1a4df2c
refactor: use unwrap instead of expect
jetjinser 94896a4
refactor: use google-cloud-gax for authentication
jetjinser 00ca033
refactor: use `context` to add context to errors
jetjinser File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
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" | ||
))), | ||
} | ||
} | ||
} |
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.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
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?
There was a problem hiding this comment.
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 ongoogle-cloud-googleapis
but does not re-export, which is required for publishing messages.