From 8fdcfb8ded255b5e59d8ab59c86dea8535416a30 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 25 Oct 2023 11:21:44 +0800 Subject: [PATCH 1/5] feat(sink): monitor error in remote sink response stream (#13028) --- Cargo.lock | 1 + src/common/src/util/future_utils.rs | 37 +++- src/common/src/util/mod.rs | 4 +- src/connector/src/sink/remote.rs | 276 +++++++++++++++++++++++----- src/jni_core/Cargo.toml | 1 + src/jni_core/src/lib.rs | 7 +- src/rpc_client/src/lib.rs | 31 +--- 7 files changed, 285 insertions(+), 72 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 32cb936bdaae3..99a5a675c0342 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -7621,6 +7621,7 @@ dependencies = [ name = "risingwave_jni_core" version = "0.1.0" dependencies = [ + "anyhow", "bytes", "cfg-or-panic", "futures", diff --git a/src/common/src/util/future_utils.rs b/src/common/src/util/future_utils.rs index 75c38488457ac..20844d8cd15d4 100644 --- a/src/common/src/util/future_utils.rs +++ b/src/common/src/util/future_utils.rs @@ -13,9 +13,11 @@ // limitations under the License. use std::future::pending; +use std::pin::{pin, Pin}; -use futures::future::Either; -use futures::{Future, FutureExt, Stream}; +use futures::future::{select, Either}; +use futures::stream::Peekable; +use futures::{Future, FutureExt, Stream, StreamExt}; /// Convert a list of streams into a [`Stream`] of results from the streams. pub fn select_all( @@ -43,3 +45,34 @@ pub fn drop_either_future( Either::Right((right, _)) => Either::Right(right), } } + +/// Await on a future while monitoring on a peekable stream that may return error. +/// The peekable stream is polled at a higher priority than the future. +/// +/// When the peekable stream returns with a error and end of stream, the future will +/// return the error immediately. Otherwise, it will keep polling the given future. +/// +/// Return: +/// - Ok(output) as the output of the given future. +/// - Err(None) to indicate that the stream has reached the end. +/// - Err(e) to indicate that the stream returns an error. +pub async fn await_future_with_monitor_error_stream( + peek_stream: &mut Peekable> + Unpin>, + future: F, +) -> Result> { + // Poll the response stream to early see the error + match select(pin!(Pin::new(&mut *peek_stream).peek()), pin!(future)).await { + Either::Left((response_result, send_future)) => match response_result { + None => Err(None), + Some(Err(_)) => { + let err = match peek_stream.next().now_or_never() { + Some(Some(Err(err))) => err, + _ => unreachable!("peek has output, peek output not None, have check err"), + }; + Err(Some(err)) + } + Some(Ok(_)) => Ok(send_future.await), + }, + Either::Right((output, _)) => Ok(output), + } +} diff --git a/src/common/src/util/mod.rs b/src/common/src/util/mod.rs index f4140b558faa7..e1f85263e1415 100644 --- a/src/common/src/util/mod.rs +++ b/src/common/src/util/mod.rs @@ -45,7 +45,9 @@ pub mod tracing; pub mod value_encoding; pub mod worker_util; -pub use future_utils::{drop_either_future, pending_on_none, select_all}; +pub use future_utils::{ + await_future_with_monitor_error_stream, drop_either_future, pending_on_none, select_all, +}; #[macro_use] pub mod match_util; diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index ad182e734a33a..310213262b2ad 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -13,17 +13,23 @@ // limitations under the License. use std::collections::HashMap; +use std::fmt::Formatter; +use std::future::Future; use std::marker::PhantomData; use std::ops::Deref; +use std::time::Instant; use anyhow::anyhow; use async_trait::async_trait; +use futures::stream::Peekable; +use futures::{StreamExt, TryFutureExt, TryStreamExt}; use itertools::Itertools; use jni::objects::{JByteArray, JValue, JValueOwned}; use prost::Message; use risingwave_common::array::StreamChunk; use risingwave_common::error::anyhow_error; use risingwave_common::types::DataType; +use risingwave_common::util::await_future_with_monitor_error_stream; use risingwave_jni_core::jvm_runtime::JVM; use risingwave_pb::connector_service::sink_coordinator_stream_request::{ CommitMetadata, StartCoordinator, @@ -43,15 +49,17 @@ use risingwave_pb::connector_service::{ }; use tokio::sync::mpsc; use tokio::sync::mpsc::{Receiver, Sender}; +use tokio_stream::wrappers::ReceiverStream; use tracing::warn; use super::encoder::{JsonEncoder, RowEncoder}; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::encoder::TimestampHandlingMode; +use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkMetrics, - SinkParam, SinkWriterParam, + DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkCommitCoordinator, SinkError, + SinkMetrics, SinkParam, SinkWriterParam, }; use crate::ConnectorParams; @@ -101,18 +109,12 @@ impl TryFrom for RemoteSink { impl Sink for RemoteSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf>; + type LogSinker = RemoteLogSinker; const SINK_NAME: &'static str = R::SINK_NAME; async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { - Ok(RemoteSinkWriter::new( - self.param.clone(), - writer_param.connector_params, - writer_param.sink_metrics.clone(), - ) - .await? - .into_log_sinker(writer_param.sink_metrics)) + RemoteLogSinker::new(self.param.clone(), writer_param).await } async fn validate(&self) -> Result<()> { @@ -192,6 +194,139 @@ impl Sink for RemoteSink { } } +pub struct RemoteLogSinker { + writer: RemoteSinkWriter, + sink_metrics: SinkMetrics, +} + +impl RemoteLogSinker { + async fn new(sink_param: SinkParam, writer_param: SinkWriterParam) -> Result { + let writer = RemoteSinkWriter::new( + sink_param, + writer_param.connector_params, + writer_param.sink_metrics.clone(), + ) + .await?; + let sink_metrics = writer_param.sink_metrics; + Ok(RemoteLogSinker { + writer, + sink_metrics, + }) + } +} + +/// Await the given future while monitoring on error of the receiver stream. +async fn await_future_with_monitor_receiver_err>>( + receiver: &mut SinkWriterStreamJniReceiver, + future: F, +) -> Result { + match await_future_with_monitor_error_stream(&mut receiver.response_stream, future).await { + Ok(result) => result, + Err(None) => Err(SinkError::Remote(anyhow!("end of remote receiver stream"))), + Err(Some(err)) => Err(SinkError::Internal(err)), + } +} + +impl LogSinker for RemoteLogSinker { + async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { + // Note: this is a total copy of the implementation of LogSinkerOf, + // except that we monitor the future of `log_reader.next_item` with await_future_with_monitor_receiver_err + // to monitor the error in the response stream. + + let mut sink_writer = self.writer; + let sink_metrics = self.sink_metrics; + #[derive(Debug)] + enum LogConsumerState { + /// Mark that the log consumer is not initialized yet + Uninitialized, + + /// Mark that a new epoch has begun. + EpochBegun { curr_epoch: u64 }, + + /// Mark that the consumer has just received a barrier + BarrierReceived { prev_epoch: u64 }, + } + + let mut state = LogConsumerState::Uninitialized; + + log_reader.init().await?; + + loop { + let (epoch, item): (u64, LogStoreReadItem) = await_future_with_monitor_receiver_err( + &mut sink_writer.stream_handle.response_rx, + log_reader.next_item().map_err(SinkError::Internal), + ) + .await?; + if let LogStoreReadItem::UpdateVnodeBitmap(_) = &item { + match &state { + LogConsumerState::BarrierReceived { .. } => {} + _ => unreachable!( + "update vnode bitmap can be accepted only right after \ + barrier, but current state is {:?}", + state + ), + } + } + // begin_epoch when not previously began + state = match state { + LogConsumerState::Uninitialized => { + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::EpochBegun { curr_epoch } => { + assert!( + epoch >= curr_epoch, + "new epoch {} should not be below the current epoch {}", + epoch, + curr_epoch + ); + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + LogConsumerState::BarrierReceived { prev_epoch } => { + assert!( + epoch > prev_epoch, + "new epoch {} should be greater than prev epoch {}", + epoch, + prev_epoch + ); + sink_writer.begin_epoch(epoch).await?; + LogConsumerState::EpochBegun { curr_epoch: epoch } + } + }; + match item { + LogStoreReadItem::StreamChunk { chunk, .. } => { + if let Err(e) = sink_writer.write_batch(chunk).await { + sink_writer.abort().await?; + return Err(e); + } + } + LogStoreReadItem::Barrier { is_checkpoint } => { + let prev_epoch = match state { + LogConsumerState::EpochBegun { curr_epoch } => curr_epoch, + _ => unreachable!("epoch must have begun before handling barrier"), + }; + if is_checkpoint { + let start_time = Instant::now(); + sink_writer.barrier(true).await?; + sink_metrics + .sink_commit_duration_metrics + .observe(start_time.elapsed().as_millis() as f64); + log_reader + .truncate(TruncateOffset::Barrier { epoch }) + .await?; + } else { + sink_writer.barrier(false).await?; + } + state = LogConsumerState::BarrierReceived { prev_epoch } + } + LogStoreReadItem::UpdateVnodeBitmap(vnode_bitmap) => { + sink_writer.update_vnode_bitmap(vnode_bitmap).await?; + } + } + } + } +} + #[derive(Debug)] pub struct CoordinatedRemoteSink(pub RemoteSink); @@ -286,14 +421,11 @@ impl SinkCoordinatorStreamJniHandle { } } -const DEFAULT_CHANNEL_SIZE: usize = 16; -#[derive(Debug)] -pub struct SinkWriterStreamJniHandle { +struct SinkWriterStreamJniSender { request_tx: Sender, - response_rx: Receiver, } -impl SinkWriterStreamJniHandle { +impl SinkWriterStreamJniSender { pub async fn start_epoch(&mut self, epoch: u64) -> Result<()> { self.request_tx .send(SinkWriterStreamRequest { @@ -316,33 +448,29 @@ impl SinkWriterStreamJniHandle { .map_err(|err| SinkError::Internal(err.into())) } - pub async fn barrier(&mut self, epoch: u64) -> Result<()> { + pub async fn barrier(&mut self, epoch: u64, is_checkpoint: bool) -> Result<()> { self.request_tx .send(SinkWriterStreamRequest { request: Some(SinkRequest::Barrier(Barrier { epoch, - is_checkpoint: false, + is_checkpoint, })), }) .await .map_err(|err| SinkError::Internal(err.into())) } +} - pub async fn commit(&mut self, epoch: u64) -> Result { - self.request_tx - .send(SinkWriterStreamRequest { - request: Some(SinkRequest::Barrier(Barrier { - epoch, - is_checkpoint: true, - })), - }) - .await - .map_err(|err| SinkError::Internal(err.into()))?; +struct SinkWriterStreamJniReceiver { + response_stream: Peekable>>, +} - match self.response_rx.recv().await { - Some(SinkWriterStreamResponse { +impl SinkWriterStreamJniReceiver { + async fn next_commit_response(&mut self) -> Result { + match self.response_stream.try_next().await { + Ok(Some(SinkWriterStreamResponse { response: Some(sink_writer_stream_response::Response::Commit(rsp)), - }) => Ok(rsp), + })) => Ok(rsp), msg => Err(SinkError::Internal(anyhow!( "should get Sync response but get {:?}", msg @@ -351,6 +479,53 @@ impl SinkWriterStreamJniHandle { } } +const DEFAULT_CHANNEL_SIZE: usize = 16; +struct SinkWriterStreamJniHandle { + request_tx: SinkWriterStreamJniSender, + response_rx: SinkWriterStreamJniReceiver, +} + +impl std::fmt::Debug for SinkWriterStreamJniHandle { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.debug_struct("SinkWriterStreamJniHandle").finish() + } +} + +impl SinkWriterStreamJniHandle { + async fn start_epoch(&mut self, epoch: u64) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.start_epoch(epoch), + ) + .await + } + + async fn write_batch(&mut self, epoch: u64, batch_id: u64, payload: Payload) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.write_batch(epoch, batch_id, payload), + ) + .await + } + + async fn barrier(&mut self, epoch: u64) -> Result<()> { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.barrier(epoch, false), + ) + .await + } + + async fn commit(&mut self, epoch: u64) -> Result { + await_future_with_monitor_receiver_err( + &mut self.response_rx, + self.request_tx.barrier(epoch, true), + ) + .await?; + self.response_rx.next_commit_response().await + } +} + pub type RemoteSinkWriter = RemoteSinkWriterInner<(), R>; pub type CoordinatedRemoteSinkWriter = RemoteSinkWriterInner, R>; @@ -374,10 +549,7 @@ impl RemoteSinkWriterInner { let (request_tx, request_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); let (response_tx, response_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); - let mut stream_handle = SinkWriterStreamJniHandle { - request_tx, - response_rx, - }; + let mut response_stream = ReceiverStream::new(response_rx).peekable(); std::thread::spawn(move || { let mut env = JVM.get_or_init().unwrap().attach_current_thread().unwrap(); @@ -388,7 +560,10 @@ impl RemoteSinkWriterInner { "(JJ)V", &[ JValue::from(&request_rx as *const Receiver as i64), - JValue::from(&response_tx as *const Sender as i64), + JValue::from( + &response_tx as *const Sender> + as i64, + ), ], ); @@ -410,8 +585,7 @@ impl RemoteSinkWriterInner { }; // First request - stream_handle - .request_tx + request_tx .send(sink_writer_stream_request) .await .map_err(|err| { @@ -423,17 +597,18 @@ impl RemoteSinkWriterInner { })?; // First response - match stream_handle.response_rx.recv().await { - Some(SinkWriterStreamResponse { + match response_stream.try_next().await { + Ok(Some(SinkWriterStreamResponse { response: Some(sink_writer_stream_response::Response::Start(_)), - }) => {} - msg => { + })) => {} + Ok(msg) => { return Err(SinkError::Internal(anyhow!( "should get start response for connector `{}` but get {:?}", R::SINK_NAME, msg ))); } + Err(e) => return Err(SinkError::Internal(e)), }; tracing::trace!( @@ -444,6 +619,11 @@ impl RemoteSinkWriterInner { let schema = param.schema(); + let stream_handle = SinkWriterStreamJniHandle { + request_tx: SinkWriterStreamJniSender { request_tx }, + response_rx: SinkWriterStreamJniReceiver { response_stream }, + }; + Ok(Self { properties: param.properties, epoch: None, @@ -458,7 +638,7 @@ impl RemoteSinkWriterInner { #[cfg(test)] fn for_test( - response_receiver: Receiver, + response_receiver: Receiver>, request_sender: Sender, ) -> RemoteSinkWriter { use risingwave_common::catalog::{Field, Schema}; @@ -480,8 +660,12 @@ impl RemoteSinkWriterInner { ]); let stream_handle = SinkWriterStreamJniHandle { - request_tx: request_sender, - response_rx: response_receiver, + request_tx: SinkWriterStreamJniSender { + request_tx: request_sender, + }, + response_rx: SinkWriterStreamJniReceiver { + response_stream: ReceiverStream::new(response_receiver).peekable(), + }, }; RemoteSinkWriter { @@ -828,12 +1012,12 @@ mod test { // test commit response_sender - .send(SinkWriterStreamResponse { + .send(Ok(SinkWriterStreamResponse { response: Some(Response::Commit(CommitResponse { epoch: 2022, metadata: None, })), - }) + })) .await .expect("test failed: failed to sync epoch"); sink.barrier(true).await.unwrap(); diff --git a/src/jni_core/Cargo.toml b/src/jni_core/Cargo.toml index 69c11a7f21e24..77cafd155000d 100644 --- a/src/jni_core/Cargo.toml +++ b/src/jni_core/Cargo.toml @@ -10,6 +10,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +anyhow = "1" bytes = "1" cfg-or-panic = "0.2" futures = { version = "0.3", default-features = false, features = ["alloc"] } diff --git a/src/jni_core/src/lib.rs b/src/jni_core/src/lib.rs index 29bbf76929b45..4815cd7368370 100644 --- a/src/jni_core/src/lib.rs +++ b/src/jni_core/src/lib.rs @@ -902,14 +902,17 @@ pub extern "system" fn Java_com_risingwave_java_binding_Binding_sendSinkWriterRe 'a, >( env: EnvParam<'a>, - channel: Pointer<'a, Sender>, + channel: Pointer<'a, Sender>>, msg: JByteArray<'a>, ) -> jboolean { execute_and_catch(env, move |env| { let sink_writer_stream_response: SinkWriterStreamResponse = Message::decode(to_guarded_slice(&msg, env)?.deref())?; - match channel.as_ref().blocking_send(sink_writer_stream_response) { + match channel + .as_ref() + .blocking_send(Ok(sink_writer_stream_response)) + { Ok(_) => Ok(JNI_TRUE), Err(e) => { tracing::info!("send error. {:?}", e); diff --git a/src/rpc_client/src/lib.rs b/src/rpc_client/src/lib.rs index be871521d3bc4..6afa67ef88efe 100644 --- a/src/rpc_client/src/lib.rs +++ b/src/rpc_client/src/lib.rs @@ -30,12 +30,11 @@ use std::any::type_name; use std::fmt::{Debug, Formatter}; use std::future::Future; use std::iter::repeat; -use std::pin::pin; use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; -use futures::future::{select, try_join_all, Either}; +use futures::future::try_join_all; use futures::stream::{BoxStream, Peekable}; use futures::{Stream, StreamExt}; use moka::future::Cache; @@ -58,13 +57,12 @@ mod sink_coordinate_client; mod stream_client; mod tracing; -use std::pin::Pin; - pub use compactor_client::{CompactorClient, GrpcCompactorProxyClient}; pub use compute_client::{ComputeClient, ComputeClientPool, ComputeClientPoolRef}; pub use connector_client::{ConnectorClient, SinkCoordinatorStreamHandle, SinkWriterStreamHandle}; pub use hummock_meta_client::{CompactionEventItem, HummockMetaClient}; pub use meta_client::{MetaClient, SinkCoordinationRpcClient}; +use risingwave_common::util::await_future_with_monitor_error_stream; pub use sink_coordinate_client::CoordinatorStreamHandle; pub use stream_client::{StreamClient, StreamClientPool, StreamClientPoolRef}; @@ -240,25 +238,16 @@ impl BidiStreamHandle { } pub async fn send_request(&mut self, request: REQ) -> Result<()> { - // Poll the response stream to early see the error - let send_request_result = match select( - pin!(self.request_sender.send(request)), - pin!(Pin::new(&mut self.response_stream).peek()), + match await_future_with_monitor_error_stream( + &mut self.response_stream, + self.request_sender.send(request), ) .await { - Either::Left((result, _)) => result, - Either::Right((response_result, send_future)) => match response_result { - None => { - return Err(anyhow!("end of response stream").into()); - } - Some(Err(e)) => { - return Err(e.clone().into()); - } - Some(Ok(_)) => send_future.await, - }, - }; - send_request_result - .map_err(|_| anyhow!("unable to send request {}", type_name::()).into()) + Ok(send_result) => send_result + .map_err(|_| anyhow!("unable to send request {}", type_name::()).into()), + Err(None) => Err(anyhow!("end of response stream").into()), + Err(Some(e)) => Err(e.into()), + } } } From 8221d3a16cec7122504e49423ae74a400d902ff1 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Wed, 25 Oct 2023 11:55:34 +0800 Subject: [PATCH 2/5] feat: add `WAIT` command (#13027) --- proto/ddl_service.proto | 5 +++++ src/frontend/src/handler/mod.rs | 2 ++ src/frontend/src/handler/wait.rs | 31 +++++++++++++++++++++++++++++ src/frontend/src/meta_client.rs | 6 ++++++ src/frontend/src/test_utils.rs | 4 ++++ src/meta/service/src/ddl_service.rs | 5 +++++ src/meta/src/rpc/ddl_controller.rs | 16 +++++++++++++++ src/rpc_client/src/meta_client.rs | 7 +++++++ src/sqlparser/src/ast/mod.rs | 6 ++++++ src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 1 + src/utils/pgwire/src/pg_response.rs | 2 ++ 12 files changed, 86 insertions(+) create mode 100644 src/frontend/src/handler/wait.rs diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 27c9f2ee82f83..1efc933a7d033 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -314,6 +314,10 @@ message GetTablesResponse { map tables = 1; } +message WaitRequest {} + +message WaitResponse {} + service DdlService { rpc CreateDatabase(CreateDatabaseRequest) returns (CreateDatabaseResponse); rpc DropDatabase(DropDatabaseRequest) returns (DropDatabaseResponse); @@ -343,4 +347,5 @@ service DdlService { rpc ListConnections(ListConnectionsRequest) returns (ListConnectionsResponse); rpc DropConnection(DropConnectionRequest) returns (DropConnectionResponse); rpc GetTables(GetTablesRequest) returns (GetTablesResponse); + rpc Wait(WaitRequest) returns (WaitResponse); } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 149f39bead330..174ed23e03ec5 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -73,6 +73,7 @@ mod show; mod transaction; pub mod util; pub mod variable; +mod wait; /// The [`PgResponseBuilder`] used by RisingWave. pub type RwPgResponseBuilder = PgResponseBuilder; @@ -419,6 +420,7 @@ pub async fn handle( } } Statement::Flush => flush::handle_flush(handler_args).await, + Statement::Wait => wait::handle_wait(handler_args).await, Statement::SetVariable { local: _, variable, diff --git a/src/frontend/src/handler/wait.rs b/src/frontend/src/handler/wait.rs new file mode 100644 index 0000000000000..83f2784ec8c17 --- /dev/null +++ b/src/frontend/src/handler/wait.rs @@ -0,0 +1,31 @@ +// 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 pgwire::pg_response::{PgResponse, StatementType}; +use risingwave_common::error::Result; + +use super::RwPgResponse; +use crate::handler::HandlerArgs; +use crate::session::SessionImpl; + +pub(super) async fn handle_wait(handler_args: HandlerArgs) -> Result { + do_wait(&handler_args.session).await?; + Ok(PgResponse::empty_result(StatementType::WAIT)) +} + +pub(crate) async fn do_wait(session: &SessionImpl) -> Result<()> { + let client = session.env().meta_client(); + client.wait().await?; + Ok(()) +} diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index ae90c2e345f9f..d37c5dec127f1 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -43,6 +43,8 @@ pub trait FrontendMetaClient: Send + Sync { async fn flush(&self, checkpoint: bool) -> Result; + async fn wait(&self) -> Result<()>; + async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result>; async fn list_table_fragments( @@ -111,6 +113,10 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0.flush(checkpoint).await } + async fn wait(&self) -> Result<()> { + self.0.wait().await + } + async fn cancel_creating_jobs(&self, infos: PbJobs) -> Result> { self.0.cancel_creating_jobs(infos).await } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 20eb252fc5053..cf915ae35713d 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -773,6 +773,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { }) } + async fn wait(&self) -> RpcResult<()> { + Ok(()) + } + async fn cancel_creating_jobs(&self, _infos: PbJobs) -> RpcResult> { Ok(vec![]) } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 2fa5f50e15666..061ff93589163 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -732,6 +732,11 @@ impl DdlService for DdlServiceImpl { } Ok(Response::new(GetTablesResponse { tables })) } + + async fn wait(&self, _request: Request) -> Result, Status> { + self.ddl_controller.wait().await; + Ok(Response::new(WaitResponse {})) + } } impl DdlServiceImpl { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 04b9729c5a5b8..36615bd93b757 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -15,6 +15,7 @@ use std::cmp::Ordering; use std::num::NonZeroUsize; use std::sync::Arc; +use std::time::Duration; use itertools::Itertools; use risingwave_common::config::DefaultParallelism; @@ -29,6 +30,7 @@ use risingwave_pb::ddl_service::alter_relation_name_request::Relation; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::stream_plan::StreamFragmentGraph as StreamFragmentGraphProto; use tokio::sync::Semaphore; +use tokio::time::sleep; use tracing::log::warn; use tracing::Instrument; @@ -1094,4 +1096,18 @@ impl DdlController { } } } + + pub async fn wait(&self) { + for _ in 0..30 * 60 { + if self + .catalog_manager + .list_creating_background_mvs() + .await + .is_empty() + { + break; + } + sleep(Duration::from_secs(1)).await; + } + } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 827860d1af7b3..95b746ea33e6c 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -698,6 +698,12 @@ impl MetaClient { Ok(resp.snapshot.unwrap()) } + pub async fn wait(&self) -> Result<()> { + let request = WaitRequest {}; + self.inner.wait(request).await?; + Ok(()) + } + pub async fn cancel_creating_jobs(&self, jobs: PbJobs) -> Result> { let request = CancelCreatingJobsRequest { jobs: Some(jobs) }; let resp = self.inner.cancel_creating_jobs(request).await?; @@ -1719,6 +1725,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, list_connections, ListConnectionsRequest, ListConnectionsResponse } ,{ ddl_client, drop_connection, DropConnectionRequest, DropConnectionResponse } ,{ ddl_client, get_tables, GetTablesRequest, GetTablesResponse } + ,{ ddl_client, wait, WaitRequest, WaitResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index ecae5a9663a88..5d802bae99cdc 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1294,6 +1294,9 @@ pub enum Statement { /// /// Note: RisingWave specific statement. Flush, + /// WAIT for ALL running stream jobs to finish. + /// It will block the current session the condition is met. + Wait, } impl fmt::Display for Statement { @@ -1787,6 +1790,9 @@ impl fmt::Display for Statement { Statement::Flush => { write!(f, "FLUSH") } + Statement::Wait => { + write!(f, "WAIT") + } Statement::Begin { modes } => { write!(f, "BEGIN")?; if !modes.is_empty() { diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index 5c2fedb0ea547..4188f06f76ae3 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -540,6 +540,7 @@ define_keywords!( VIEWS, VIRTUAL, VOLATILE, + WAIT, WATERMARK, WHEN, WHENEVER, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index ee054f7d17031..5cc094a204268 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -259,6 +259,7 @@ impl Parser { Keyword::PREPARE => Ok(self.parse_prepare()?), Keyword::COMMENT => Ok(self.parse_comment()?), Keyword::FLUSH => Ok(Statement::Flush), + Keyword::WAIT => Ok(Statement::Wait), _ => self.expected( "an SQL statement", Token::Word(w).with_location(token.location), diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 29ea77f83b71b..eeec929732f50 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -92,6 +92,7 @@ pub enum StatementType { ROLLBACK, SET_TRANSACTION, CANCEL_COMMAND, + WAIT, } impl std::fmt::Display for StatementType { @@ -278,6 +279,7 @@ impl StatementType { }, Statement::Explain { .. } => Ok(StatementType::EXPLAIN), Statement::Flush => Ok(StatementType::FLUSH), + Statement::Wait => Ok(StatementType::WAIT), _ => Err("unsupported statement type".to_string()), } } From 6495d90ac0e7077a7b57490cc618cce3399b3f38 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 25 Oct 2023 12:50:17 +0800 Subject: [PATCH 3/5] feat(sink): async truncate for kinesis, pulsar, nats, redis and clickhouse sink (#12930) --- src/connector/src/sink/blackhole.rs | 3 + src/connector/src/sink/clickhouse.rs | 34 +++---- src/connector/src/sink/kafka.rs | 116 ++++++++++-------------- src/connector/src/sink/kinesis.rs | 40 ++++---- src/connector/src/sink/mod.rs | 9 +- src/connector/src/sink/nats.rs | 35 +++---- src/connector/src/sink/pulsar.rs | 131 ++++++++++++++------------- src/connector/src/sink/redis.rs | 43 +++++---- src/connector/src/sink/remote.rs | 1 + src/connector/src/sink/writer.rs | 101 +++++++++++++++++---- 10 files changed, 284 insertions(+), 229 deletions(-) diff --git a/src/connector/src/sink/blackhole.rs b/src/connector/src/sink/blackhole.rs index 1f1ace3b0d104..60b0506604c97 100644 --- a/src/connector/src/sink/blackhole.rs +++ b/src/connector/src/sink/blackhole.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use async_trait::async_trait; + use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; use crate::sink::{ DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkError, SinkParam, SinkWriterParam, @@ -45,6 +47,7 @@ impl Sink for BlackHoleSink { } } +#[async_trait] impl LogSinker for BlackHoleSink { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { log_reader.init().await?; diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 2bddf8026216f..f4fdf9b761f38 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -29,7 +29,11 @@ use serde_derive::Deserialize; use serde_with::serde_as; use super::{DummySinkCommitCoordinator, SinkWriterParam}; -use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::catalog::desc::SinkDesc; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; use crate::sink::{ Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; @@ -243,10 +247,14 @@ impl ClickHouseSink { } impl Sink for ClickHouseSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = CLICKHOUSE_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { // For upsert clickhouse sink, the primary key must be defined. if !self.is_append_only && self.pk_indices.is_empty() { @@ -277,7 +285,7 @@ impl Sink for ClickHouseSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(ClickHouseSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -285,7 +293,7 @@ impl Sink for ClickHouseSink { self.is_append_only, ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } } pub struct ClickHouseSinkWriter { @@ -496,24 +504,18 @@ impl ClickHouseSinkWriter { } } -#[async_trait::async_trait] -impl SinkWriter for ClickHouseSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for ClickHouseSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { if self.is_append_only { self.append_only(chunk).await } else { self.upsert(chunk).await } } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - // clickhouse no transactional guarantees, so we do nothing here. - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) - } } #[derive(ClickHouseRow, Deserialize, Clone)] diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index a204a8d121706..f77b2b0a88c36 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -14,20 +14,18 @@ use std::collections::HashMap; use std::fmt::Debug; -use std::pin::pin; use std::sync::Arc; use std::time::Duration; use anyhow::anyhow; -use futures::future::{select, Either}; use futures::{Future, FutureExt, TryFuture}; use rdkafka::error::KafkaError; use rdkafka::message::ToBytes; use rdkafka::producer::{DeliveryFuture, FutureProducer, FutureRecord}; use rdkafka::types::RDKafkaErrorCode; use rdkafka::ClientConfig; +use risingwave_common::array::StreamChunk; use risingwave_common::catalog::Schema; -use risingwave_common::util::drop_either_future; use serde_derive::{Deserialize, Serialize}; use serde_with::{serde_as, DisplayFromStr}; use strum_macros::{Display, EnumString}; @@ -37,11 +35,11 @@ use super::{Sink, SinkError, SinkParam}; use crate::common::KafkaCommon; use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::log_store::{ - DeliveryFutureManager, DeliveryFutureManagerAddFuture, LogReader, LogStoreReadItem, +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, }; -use crate::sink::writer::FormattedSink; -use crate::sink::{DummySinkCommitCoordinator, LogSinker, Result, SinkWriterParam}; +use crate::sink::{DummySinkCommitCoordinator, Result, SinkWriterParam}; use crate::source::kafka::{KafkaProperties, KafkaSplitEnumerator, PrivateLinkProducerContext}; use crate::source::{SourceEnumeratorContext, SplitEnumerator}; use crate::{ @@ -299,7 +297,7 @@ impl TryFrom for KafkaSink { impl Sink for KafkaSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = KafkaLogSinker; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = KAFKA_SINK; @@ -316,7 +314,18 @@ impl Sink for KafkaSink { self.sink_from_name.clone(), ) .await?; - KafkaLogSinker::new(self.config.clone(), formatter).await + let max_delivery_buffer_size = (self + .config + .rdkafka_properties + .queue_buffering_max_messages + .as_ref() + .cloned() + .unwrap_or(KAFKA_WRITER_MAX_QUEUE_SIZE) as f32 + * KAFKA_WRITER_MAX_QUEUE_SIZE_RATIO) as usize; + + Ok(KafkaSinkWriter::new(self.config.clone(), formatter) + .await? + .into_log_sinker(max_delivery_buffer_size)) } async fn validate(&self) -> Result<()> { @@ -370,16 +379,15 @@ struct KafkaPayloadWriter<'a> { config: &'a KafkaConfig, } -type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; +pub type KafkaSinkDeliveryFuture = impl TryFuture + Unpin + 'static; -pub struct KafkaLogSinker { +pub struct KafkaSinkWriter { formatter: SinkFormatterImpl, inner: FutureProducer, - future_manager: DeliveryFutureManager, config: KafkaConfig, } -impl KafkaLogSinker { +impl KafkaSinkWriter { async fn new(config: KafkaConfig, formatter: SinkFormatterImpl) -> Result { let inner: FutureProducer = { let mut c = ClientConfig::new(); @@ -403,19 +411,29 @@ impl KafkaLogSinker { c.create_with_context(producer_ctx).await? }; - let max_delivery_buffer_size = (config - .rdkafka_properties - .queue_buffering_max_messages - .as_ref() - .cloned() - .unwrap_or(KAFKA_WRITER_MAX_QUEUE_SIZE) as f32 - * KAFKA_WRITER_MAX_QUEUE_SIZE_RATIO) as usize; - - Ok(KafkaLogSinker { + Ok(KafkaSinkWriter { formatter, inner, config: config.clone(), - future_manager: DeliveryFutureManager::new(max_delivery_buffer_size), + }) + } +} + +impl AsyncTruncateSinkWriter for KafkaSinkWriter { + type DeliveryFuture = KafkaSinkDeliveryFuture; + + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { + let mut payload_writer = KafkaPayloadWriter { + inner: &mut self.inner, + add_future, + config: &self.config, + }; + dispatch_sink_formatter_impl!(&self.formatter, formatter, { + payload_writer.write_chunk(chunk, formatter).await }) } } @@ -537,50 +555,6 @@ impl<'a> FormattedSink for KafkaPayloadWriter<'a> { } } -impl LogSinker for KafkaLogSinker { - async fn consume_log_and_sink(mut self, mut log_reader: impl LogReader) -> Result<()> { - log_reader.init().await?; - loop { - let select_result = drop_either_future( - select( - pin!(log_reader.next_item()), - pin!(self.future_manager.next_truncate_offset()), - ) - .await, - ); - match select_result { - Either::Left(item_result) => { - let (epoch, item) = item_result?; - match item { - LogStoreReadItem::StreamChunk { chunk_id, chunk } => { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { - let mut writer = KafkaPayloadWriter { - inner: &self.inner, - add_future: self - .future_manager - .start_write_chunk(epoch, chunk_id), - config: &self.config, - }; - writer.write_chunk(chunk, formatter).await?; - }) - } - LogStoreReadItem::Barrier { - is_checkpoint: _is_checkpoint, - } => { - self.future_manager.add_barrier(epoch); - } - LogStoreReadItem::UpdateVnodeBitmap(_) => {} - } - } - Either::Right(offset_result) => { - let offset = offset_result?; - log_reader.truncate(offset).await?; - } - } - } - } -} - #[cfg(test)] mod test { use maplit::hashmap; @@ -748,7 +722,7 @@ mod test { let kafka_config = KafkaConfig::from_hashmap(properties)?; // Create the actual sink writer to Kafka - let mut sink = KafkaLogSinker::new( + let sink = KafkaSinkWriter::new( kafka_config.clone(), SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( // We do not specify primary key for this schema @@ -759,12 +733,16 @@ mod test { .await .unwrap(); + use crate::sink::log_store::DeliveryFutureManager; + + let mut future_manager = DeliveryFutureManager::new(usize::MAX); + for i in 0..10 { println!("epoch: {}", i); for j in 0..100 { let mut writer = KafkaPayloadWriter { inner: &sink.inner, - add_future: sink.future_manager.start_write_chunk(i, j), + add_future: future_manager.start_write_chunk(i, j), config: &sink.config, }; match writer diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index dd8518af39948..605edde3b1eb0 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -30,8 +30,12 @@ use super::catalog::SinkFormatDesc; use super::SinkParam; use crate::common::KinesisCommon; use crate::dispatch_sink_formatter_impl; +use crate::sink::catalog::desc::SinkDesc; use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, +}; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkError, SinkWriterParam}; pub const KINESIS_SINK: &str = "kinesis"; @@ -67,10 +71,14 @@ impl TryFrom for KinesisSink { impl Sink for KinesisSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = KINESIS_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { // Kinesis requires partition key. There is no builtin support for round-robin as in kafka/pulsar. // https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#Streams-PutRecord-request-PartitionKey @@ -103,7 +111,7 @@ impl Sink for KinesisSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(KinesisSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -113,7 +121,7 @@ impl Sink for KinesisSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } } @@ -214,20 +222,16 @@ impl FormattedSink for KinesisSinkPayloadWriter { } } -#[async_trait::async_trait] -impl SinkWriter for KinesisSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { - dispatch_sink_formatter_impl!(&self.formatter, formatter, { +impl AsyncTruncateSinkWriter for KinesisSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { + dispatch_sink_formatter_impl!( + &self.formatter, + formatter, self.payload_writer.write_chunk(chunk, formatter).await - }) - } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - // Kinesis offers no transactional guarantees, so we do nothing here. - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) + ) } } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 7769a87f4e715..6afd08778cd96 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -34,7 +34,6 @@ pub mod utils; pub mod writer; use std::collections::HashMap; -use std::future::Future; use ::clickhouse::error::Error as ClickHouseError; use ::redis::RedisError; @@ -278,11 +277,9 @@ pub trait Sink: TryFrom { } } -pub trait LogSinker: Send + 'static { - fn consume_log_and_sink( - self, - log_reader: impl LogReader, - ) -> impl Future> + Send + 'static; +#[async_trait] +pub trait LogSinker: 'static { + async fn consume_log_and_sink(self, log_reader: impl LogReader) -> Result<()>; } #[async_trait] diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 8e3f3e2c18022..2f810eed786a9 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -25,10 +25,14 @@ use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; use super::utils::chunk_to_json; -use super::{DummySinkCommitCoordinator, SinkWriter, SinkWriterParam}; +use super::{DummySinkCommitCoordinator, SinkWriterParam}; use crate::common::NatsCommon; +use crate::sink::catalog::desc::SinkDesc; use crate::sink::encoder::{JsonEncoder, TimestampHandlingMode}; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; use crate::sink::{Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY}; pub const NATS_SINK: &str = "nats"; @@ -88,10 +92,14 @@ impl TryFrom for NatsSink { impl Sink for NatsSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = NATS_SINK; + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + async fn validate(&self) -> Result<()> { if !self.is_append_only { return Err(SinkError::Nats(anyhow!( @@ -110,11 +118,11 @@ impl Sink for NatsSink { Ok(()) } - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok( NatsSinkWriter::new(self.config.clone(), self.schema.clone()) .await? - .into_log_sinker(writer_param.sink_metrics), + .into_log_sinker(usize::MAX), ) } } @@ -153,17 +161,12 @@ impl NatsSinkWriter { } } -#[async_trait::async_trait] -impl SinkWriter for NatsSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for NatsSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { self.append_only(chunk).await } - - async fn begin_epoch(&mut self, _epoch_id: u64) -> Result<()> { - Ok(()) - } - - async fn barrier(&mut self, _is_checkpoint: bool) -> Result<()> { - Ok(()) - } } diff --git a/src/connector/src/sink/pulsar.rs b/src/connector/src/sink/pulsar.rs index f980b2ad9f9b1..9eb57c1ae0771 100644 --- a/src/connector/src/sink/pulsar.rs +++ b/src/connector/src/sink/pulsar.rs @@ -12,14 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, VecDeque}; +use std::collections::HashMap; use std::fmt::Debug; use std::time::Duration; use anyhow::anyhow; -use async_trait::async_trait; -use futures::future::try_join_all; -use futures::TryFutureExt; +use futures::{FutureExt, TryFuture, TryFutureExt}; use pulsar::producer::{Message, SendFuture}; use pulsar::{Producer, ProducerOptions, Pulsar, TokioExecutor}; use risingwave_common::array::StreamChunk; @@ -28,10 +26,15 @@ use serde::Deserialize; use serde_with::{serde_as, DisplayFromStr}; use super::catalog::{SinkFormat, SinkFormatDesc}; -use super::{Sink, SinkError, SinkParam, SinkWriter, SinkWriterParam}; +use super::{Sink, SinkError, SinkParam, SinkWriterParam}; use crate::common::PulsarCommon; -use crate::sink::formatter::SinkFormatterImpl; -use crate::sink::writer::{FormattedSink, LogSinkerOf, SinkWriterExt}; +use crate::sink::catalog::desc::SinkDesc; +use crate::sink::encoder::SerTo; +use crate::sink::formatter::{SinkFormatter, SinkFormatterImpl}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, FormattedSink, +}; use crate::sink::{DummySinkCommitCoordinator, Result}; use crate::{deserialize_duration_from_string, dispatch_sink_formatter_impl}; @@ -155,11 +158,15 @@ impl TryFrom for PulsarSink { impl Sink for PulsarSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = PULSAR_SINK; - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + fn default_sink_decouple(desc: &SinkDesc) -> bool { + desc.sink_type.is_append_only() + } + + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(PulsarSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -169,7 +176,7 @@ impl Sink for PulsarSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(PULSAR_SEND_FUTURE_BUFFER_MAX_SIZE)) } async fn validate(&self) -> Result<()> { @@ -199,15 +206,26 @@ impl Sink for PulsarSink { } pub struct PulsarSinkWriter { - payload_writer: PulsarPayloadWriter, formatter: SinkFormatterImpl, -} - -struct PulsarPayloadWriter { pulsar: Pulsar, producer: Producer, config: PulsarConfig, - send_future_buffer: VecDeque, +} + +struct PulsarPayloadWriter<'w> { + producer: &'w mut Producer, + config: &'w PulsarConfig, + add_future: DeliveryFutureManagerAddFuture<'w, PulsarDeliveryFuture>, +} + +pub type PulsarDeliveryFuture = impl TryFuture + Unpin + 'static; + +fn may_delivery_future(future: SendFuture) -> PulsarDeliveryFuture { + future.map(|result| { + result + .map(|_| ()) + .map_err(|e: pulsar::Error| SinkError::Pulsar(anyhow!(e))) + }) } impl PulsarSinkWriter { @@ -226,17 +244,14 @@ impl PulsarSinkWriter { let producer = build_pulsar_producer(&pulsar, &config).await?; Ok(Self { formatter, - payload_writer: PulsarPayloadWriter { - pulsar, - producer, - config, - send_future_buffer: VecDeque::new(), - }, + pulsar, + producer, + config, }) } } -impl PulsarPayloadWriter { +impl<'w> PulsarPayloadWriter<'w> { async fn send_message(&mut self, message: Message) -> Result<()> { let mut success_flag = false; let mut connection_err = None; @@ -247,17 +262,10 @@ impl PulsarPayloadWriter { // a SendFuture holding the message receipt // or error after sending is returned Ok(send_future) => { - // Check if send_future_buffer is greater than the preset limit - while self.send_future_buffer.len() >= PULSAR_SEND_FUTURE_BUFFER_MAX_SIZE { - self.send_future_buffer - .pop_front() - .expect("Expect the SendFuture not to be None") - .map_err(|e| SinkError::Pulsar(anyhow!(e))) - .await?; - } - + self.add_future + .add_future_may_await(may_delivery_future(send_future)) + .await?; success_flag = true; - self.send_future_buffer.push_back(send_future); break; } // error upon sending @@ -295,24 +303,9 @@ impl PulsarPayloadWriter { self.send_message(message).await?; Ok(()) } - - async fn commit_inner(&mut self) -> Result<()> { - self.producer - .send_batch() - .map_err(pulsar_to_sink_err) - .await?; - try_join_all( - self.send_future_buffer - .drain(..) - .map(|send_future| send_future.map_err(|e| SinkError::Pulsar(anyhow!(e)))), - ) - .await?; - - Ok(()) - } } -impl FormattedSink for PulsarPayloadWriter { +impl<'w> FormattedSink for PulsarPayloadWriter<'w> { type K = String; type V = Vec; @@ -321,23 +314,33 @@ impl FormattedSink for PulsarPayloadWriter { } } -#[async_trait] -impl SinkWriter for PulsarSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for PulsarSinkWriter { + type DeliveryFuture = PulsarDeliveryFuture; + + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { dispatch_sink_formatter_impl!(&self.formatter, formatter, { - self.payload_writer.write_chunk(chunk, formatter).await + let mut payload_writer = PulsarPayloadWriter { + producer: &mut self.producer, + add_future, + config: &self.config, + }; + // TODO: we can call `payload_writer.write_chunk(chunk, formatter)`, + // but for an unknown reason, this will greatly increase the compile time, + // by nearly 4x. May investigate it later. + for r in formatter.format_chunk(&chunk) { + let (key, value) = r?; + payload_writer + .write_inner( + key.map(SerTo::ser_to).transpose()?, + value.map(SerTo::ser_to).transpose()?, + ) + .await?; + } + Ok(()) }) } - - async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { - Ok(()) - } - - async fn barrier(&mut self, is_checkpoint: bool) -> Result { - if is_checkpoint { - self.payload_writer.commit_inner().await?; - } - - Ok(()) - } } diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index 6120075a049df..af3ec3b981620 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -29,8 +29,11 @@ use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_impl; -use crate::sink::writer::{LogSinkerOf, SinkWriterExt}; -use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; +use crate::sink::log_store::DeliveryFutureManagerAddFuture; +use crate::sink::writer::{ + AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, +}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriterParam}; pub const REDIS_SINK: &str = "redis"; pub const KEY_FORMAT: &str = "key_format"; @@ -99,11 +102,11 @@ impl TryFrom for RedisSink { impl Sink for RedisSink { type Coordinator = DummySinkCommitCoordinator; - type LogSinker = LogSinkerOf; + type LogSinker = AsyncTruncateLogSinkerOf; const SINK_NAME: &'static str = "redis"; - async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + async fn new_log_sinker(&self, _writer_param: SinkWriterParam) -> Result { Ok(RedisSinkWriter::new( self.config.clone(), self.schema.clone(), @@ -113,7 +116,7 @@ impl Sink for RedisSink { self.sink_from_name.clone(), ) .await? - .into_log_sinker(writer_param.sink_metrics)) + .into_log_sinker(usize::MAX)) } async fn validate(&self) -> Result<()> { @@ -257,25 +260,16 @@ impl RedisSinkWriter { } } -#[async_trait] -impl SinkWriter for RedisSinkWriter { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { +impl AsyncTruncateSinkWriter for RedisSinkWriter { + async fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> Result<()> { dispatch_sink_formatter_impl!(&self.formatter, formatter, { self.payload_writer.write_chunk(chunk, formatter).await }) } - - async fn begin_epoch(&mut self, epoch: u64) -> Result<()> { - self.epoch = epoch; - Ok(()) - } - - async fn barrier(&mut self, is_checkpoint: bool) -> Result<()> { - if is_checkpoint { - self.payload_writer.commit().await?; - } - Ok(()) - } } #[cfg(test)] @@ -290,6 +284,7 @@ mod test { use super::*; use crate::sink::catalog::{SinkEncode, SinkFormat}; + use crate::sink::log_store::DeliveryFutureManager; #[tokio::test] async fn test_write() { @@ -326,8 +321,10 @@ mod test { ], ); + let mut manager = DeliveryFutureManager::new(0); + redis_sink_writer - .write_batch(chunk_a) + .write_chunk(chunk_a, manager.start_write_chunk(0, 0)) .await .expect("failed to write batch"); let expected_a = @@ -383,6 +380,8 @@ mod test { .await .unwrap(); + let mut future_manager = DeliveryFutureManager::new(0); + let chunk_a = StreamChunk::new( vec![Op::Insert, Op::Insert, Op::Insert], vec![ @@ -392,7 +391,7 @@ mod test { ); redis_sink_writer - .write_batch(chunk_a) + .write_chunk(chunk_a, future_manager.start_write_chunk(0, 0)) .await .expect("failed to write batch"); let expected_a = vec![ diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 310213262b2ad..3c52cb720dbd4 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -227,6 +227,7 @@ async fn await_future_with_monitor_receiver_err> } } +#[async_trait] impl LogSinker for RemoteLogSinker { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { // Note: this is a total copy of the implementation of LogSinkerOf, diff --git a/src/connector/src/sink/writer.rs b/src/connector/src/sink/writer.rs index 37ad452831b2e..64261bb42ab48 100644 --- a/src/connector/src/sink/writer.rs +++ b/src/connector/src/sink/writer.rs @@ -12,17 +12,25 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::future::{Future, Ready}; +use std::pin::pin; use std::sync::Arc; use std::time::Instant; use async_trait::async_trait; +use futures::future::{select, Either}; +use futures::TryFuture; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; +use risingwave_common::util::drop_either_future; use crate::sink::encoder::SerTo; use crate::sink::formatter::SinkFormatter; -use crate::sink::log_store::{LogReader, LogStoreReadItem, TruncateOffset}; -use crate::sink::{LogSinker, Result, SinkMetrics}; +use crate::sink::log_store::{ + DeliveryFutureManager, DeliveryFutureManagerAddFuture, LogReader, LogStoreReadItem, + TruncateOffset, +}; +use crate::sink::{LogSinker, Result, SinkError, SinkMetrics}; #[async_trait] pub trait SinkWriter: Send + 'static { @@ -48,22 +56,17 @@ pub trait SinkWriter: Send + 'static { } } -// TODO: remove this trait after KafkaSinkWriter implements SinkWriter -#[async_trait] -// An old version of SinkWriter for backward compatibility -pub trait SinkWriterV1: Send + 'static { - async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()>; - - // the following interface is for transactions, if not supported, return Ok(()) - // start a transaction with epoch number. Note that epoch number should be increasing. - async fn begin_epoch(&mut self, epoch: u64) -> Result<()>; +pub type DummyDeliveryFuture = Ready>; - // commits the current transaction and marks all messages in the transaction success. - async fn commit(&mut self) -> Result<()>; +pub trait AsyncTruncateSinkWriter: Send + 'static { + type DeliveryFuture: TryFuture + Unpin + Send + 'static = + DummyDeliveryFuture; - // aborts the current transaction because some error happens. we should rollback to the last - // commit point. - async fn abort(&mut self) -> Result<()>; + fn write_chunk<'a>( + &'a mut self, + chunk: StreamChunk, + add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, + ) -> impl Future> + Send + 'a; } /// A free-form sink that may output in multiple formats and encodings. Examples include kafka, @@ -104,12 +107,12 @@ pub trait FormattedSink { } } -pub struct LogSinkerOf> { +pub struct LogSinkerOf { writer: W, sink_metrics: SinkMetrics, } -impl> LogSinkerOf { +impl LogSinkerOf { pub fn new(writer: W, sink_metrics: SinkMetrics) -> Self { LogSinkerOf { writer, @@ -118,6 +121,7 @@ impl> LogSinkerOf { } } +#[async_trait] impl> LogSinker for LogSinkerOf { async fn consume_log_and_sink(self, mut log_reader: impl LogReader) -> Result<()> { let mut sink_writer = self.writer; @@ -222,3 +226,64 @@ where } } } + +pub struct AsyncTruncateLogSinkerOf { + writer: W, + future_manager: DeliveryFutureManager, +} + +impl AsyncTruncateLogSinkerOf { + pub fn new(writer: W, max_future_count: usize) -> Self { + AsyncTruncateLogSinkerOf { + writer, + future_manager: DeliveryFutureManager::new(max_future_count), + } + } +} + +#[async_trait] +impl LogSinker for AsyncTruncateLogSinkerOf { + async fn consume_log_and_sink(mut self, mut log_reader: impl LogReader) -> Result<()> { + log_reader.init().await?; + loop { + let select_result = drop_either_future( + select( + pin!(log_reader.next_item()), + pin!(self.future_manager.next_truncate_offset()), + ) + .await, + ); + match select_result { + Either::Left(item_result) => { + let (epoch, item) = item_result?; + match item { + LogStoreReadItem::StreamChunk { chunk_id, chunk } => { + let add_future = self.future_manager.start_write_chunk(epoch, chunk_id); + self.writer.write_chunk(chunk, add_future).await?; + } + LogStoreReadItem::Barrier { + is_checkpoint: _is_checkpoint, + } => { + self.future_manager.add_barrier(epoch); + } + LogStoreReadItem::UpdateVnodeBitmap(_) => {} + } + } + Either::Right(offset_result) => { + let offset = offset_result?; + log_reader.truncate(offset).await?; + } + } + } + } +} + +#[easy_ext::ext(AsyncTruncateSinkWriterExt)] +impl T +where + T: AsyncTruncateSinkWriter + Sized, +{ + pub fn into_log_sinker(self, max_future_count: usize) -> AsyncTruncateLogSinkerOf { + AsyncTruncateLogSinkerOf::new(self, max_future_count) + } +} From 3e89c83a5cd2aec90002591531a8e44a7cd5c826 Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Wed, 25 Oct 2023 13:03:39 +0800 Subject: [PATCH 4/5] feat: add iceberg cdc test (#12956) Co-authored-by: ZENOTME --- .gitignore | 3 + ci/scripts/e2e-iceberg-cdc.sh | 91 +++++++++++ ci/workflows/integration-tests.yml | 5 + ci/workflows/pull-request.yml | 15 ++ docker/docker-compose.yml | 1 + e2e_test/iceberg/main.py | 8 +- e2e_test/iceberg/test_case/cdc/load.slt | 46 ++++++ e2e_test/iceberg/test_case/cdc/mysql_cdc.sql | 21 +++ .../test_case/cdc/mysql_cdc_insert.sql | 7 + .../test_case/cdc/no_partition_cdc.toml | 25 +++ .../test_case/cdc/no_partition_cdc_init.toml | 31 ++++ integration_tests/iceberg-cdc/README.md | 5 + .../iceberg-cdc/docker-compose.yaml | 142 ++++++++++++++++++ .../iceberg-cdc/mysql_prepare.sql | 15 ++ integration_tests/iceberg-cdc/python/check.py | 25 +++ .../iceberg-cdc/python/config.ini | 8 + integration_tests/iceberg-cdc/python/init.py | 103 +++++++++++++ .../iceberg-cdc/python/pyproject.toml | 16 ++ integration_tests/iceberg-cdc/run_test.sh | 19 +++ .../iceberg-cdc/spark/.gitignore | 3 + .../iceberg-cdc/spark/spark-connect-server.sh | 21 +++ integration_tests/scripts/run_demos.py | 9 ++ 22 files changed, 616 insertions(+), 3 deletions(-) create mode 100755 ci/scripts/e2e-iceberg-cdc.sh create mode 100644 e2e_test/iceberg/test_case/cdc/load.slt create mode 100644 e2e_test/iceberg/test_case/cdc/mysql_cdc.sql create mode 100644 e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql create mode 100644 e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml create mode 100644 e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml create mode 100644 integration_tests/iceberg-cdc/README.md create mode 100644 integration_tests/iceberg-cdc/docker-compose.yaml create mode 100644 integration_tests/iceberg-cdc/mysql_prepare.sql create mode 100644 integration_tests/iceberg-cdc/python/check.py create mode 100644 integration_tests/iceberg-cdc/python/config.ini create mode 100644 integration_tests/iceberg-cdc/python/init.py create mode 100644 integration_tests/iceberg-cdc/python/pyproject.toml create mode 100755 integration_tests/iceberg-cdc/run_test.sh create mode 100644 integration_tests/iceberg-cdc/spark/.gitignore create mode 100755 integration_tests/iceberg-cdc/spark/spark-connect-server.sh diff --git a/.gitignore b/.gitignore index 19fb6643dd8a6..375738f67093e 100644 --- a/.gitignore +++ b/.gitignore @@ -74,4 +74,7 @@ simulation-it-test.tar.zst # hummock-trace .trace +# spark binary +e2e_test/iceberg/spark-*-bin* + **/poetry.lock \ No newline at end of file diff --git a/ci/scripts/e2e-iceberg-cdc.sh b/ci/scripts/e2e-iceberg-cdc.sh new file mode 100755 index 0000000000000..081f5bbd2afcb --- /dev/null +++ b/ci/scripts/e2e-iceberg-cdc.sh @@ -0,0 +1,91 @@ +#!/usr/bin/env bash + +# Exits as soon as any line fails. +set -euo pipefail + +source ci/scripts/common.sh + +# prepare environment +export CONNECTOR_RPC_ENDPOINT="localhost:50051" +export CONNECTOR_LIBS_PATH="./connector-node/libs" + +while getopts 'p:' opt; do + case ${opt} in + p ) + profile=$OPTARG + ;; + \? ) + echo "Invalid Option: -$OPTARG" 1>&2 + exit 1 + ;; + : ) + echo "Invalid option: $OPTARG requires an argument" 1>&2 + ;; + esac +done +shift $((OPTIND -1)) + +download_and_prepare_rw "$profile" source + +echo "--- Download connector node package" +buildkite-agent artifact download risingwave-connector.tar.gz ./ +mkdir ./connector-node +tar xf ./risingwave-connector.tar.gz -C ./connector-node + +echo "--- e2e, ci-1cn-1fe, iceberg cdc" + +node_port=50051 +node_timeout=10 + +wait_for_connector_node_start() { + start_time=$(date +%s) + while : + do + if nc -z localhost $node_port; then + echo "Port $node_port is listened! Connector Node is up!" + break + fi + + current_time=$(date +%s) + elapsed_time=$((current_time - start_time)) + if [ $elapsed_time -ge $node_timeout ]; then + echo "Timeout waiting for port $node_port to be listened!" + exit 1 + fi + sleep 0.1 + done + sleep 2 +} + +echo "--- starting risingwave cluster with connector node" + +RUST_LOG="info,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +cargo make ci-start ci-1cn-1fe-with-recovery +./connector-node/start-service.sh -p $node_port > .risingwave/log/connector-node.log 2>&1 & +echo "waiting for connector node to start" +wait_for_connector_node_start + +# prepare minio iceberg sink +echo "--- preparing iceberg" +.risingwave/bin/mcli -C .risingwave/config/mcli mb hummock-minio/icebergdata + +cd e2e_test/iceberg +bash ./start_spark_connect_server.sh + +# Don't remove the `--quiet` option since poetry has a bug when printing output, see +# https://github.com/python-poetry/poetry/issues/3412 +"$HOME"/.local/bin/poetry update --quiet + +# 1. import data to mysql +mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc.sql + +# 2. create table and sink +"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc_init.toml + +# 3. insert new data to mysql +mysql --host=mysql --port=3306 -u root -p123456 < ./test_case/cdc/mysql_cdc_insert.sql + +sleep 20 + +# 4. check change +"$HOME"/.local/bin/poetry run python main.py -t ./test_case/cdc/no_partition_cdc.toml \ No newline at end of file diff --git a/ci/workflows/integration-tests.yml b/ci/workflows/integration-tests.yml index 4bd0ec1a000b1..455f29b210ec1 100644 --- a/ci/workflows/integration-tests.yml +++ b/ci/workflows/integration-tests.yml @@ -29,6 +29,7 @@ steps: - "postgres-cdc" - "mysql-sink" - "postgres-sink" + - "iceberg-cdc" # - "iceberg-sink" - "debezium-mysql" format: @@ -79,6 +80,10 @@ steps: # testcase: "iceberg-sink" # format: "protobuf" # skip: true + - with: + testcase: "iceberg-cdc" + format: "protobuf" + skip: true - with: testcase: "debezium-mysql" format: "protobuf" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 985bd0be4b822..3aaa09f0d7716 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -209,6 +209,21 @@ steps: timeout_in_minutes: 10 retry: *auto-retry + - label: "end-to-end iceberg cdc test" + if: build.pull_request.labels includes "ci/run-e2e-iceberg-sink-tests" + command: "ci/scripts/e2e-iceberg-cdc.sh -p ci-dev" + depends_on: + - "build" + - "build-other" + plugins: + - docker-compose#v4.9.0: + run: sink-test-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 10 + retry: *auto-retry + - label: "end-to-end pulsar sink test" if: build.pull_request.labels includes "ci/run-e2e-pulsar-sink-tests" command: "ci/scripts/e2e-pulsar-sink-test.sh -p ci-dev" diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index d25c94daf2670..4dbd5fe5bb28d 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -260,6 +260,7 @@ services: MINIO_PROMETHEUS_URL: "http://prometheus-0:9500" MINIO_ROOT_PASSWORD: hummockadmin MINIO_ROOT_USER: hummockadmin + MINIO_DOMAIN: "minio-0" container_name: minio-0 healthcheck: test: diff --git a/e2e_test/iceberg/main.py b/e2e_test/iceberg/main.py index fa07aa367a9b3..3f3120227e6e7 100644 --- a/e2e_test/iceberg/main.py +++ b/e2e_test/iceberg/main.py @@ -42,14 +42,16 @@ def init_iceberg_table(args,init_sqls): spark.sql(sql) -def init_risingwave_mv(args,slt): +def execute_slt(args,slt): + if slt is None or slt == "": + return rw_config = args['risingwave'] cmd = f"sqllogictest -p {rw_config['port']} -d {rw_config['db']} {slt}" print(f"Command line is [{cmd}]") subprocess.run(cmd, shell=True, check=True) - time.sleep(10) + time.sleep(30) def verify_result(args,verify_sql,verify_schema,verify_data): @@ -110,6 +112,6 @@ def drop_table(args,drop_sqls): print({section: dict(config[section]) for section in config.sections()}) init_iceberg_table(config,init_sqls) - init_risingwave_mv(config,slt) + execute_slt(config,slt) verify_result(config,verify_sql,verify_schema,verify_data) drop_table(config,drop_sqls) diff --git a/e2e_test/iceberg/test_case/cdc/load.slt b/e2e_test/iceberg/test_case/cdc/load.slt new file mode 100644 index 0000000000000..caefd1326bbda --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/load.slt @@ -0,0 +1,46 @@ +# CDC source basic test + +# enable cdc backfill in ci +statement ok +set cdc_backfill='true'; + +statement ok +create table products ( id INT, + name STRING, + description STRING, + PRIMARY KEY (id) +) with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'my@db', + table.name = 'products', + server.id = '5085' +); + + +statement ok +CREATE SINK s1 AS select * from products WITH ( + connector = 'iceberg', + type = 'upsert', + force_append_only = 'false', + database.name = 'demo', + table.name = 'demo_db.demo_table', + catalog.type = 'storage', + warehouse.path = 's3://icebergdata/demo', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + primary_key = 'id' +); + +query I +select count(*) from products; +---- +8 + +statement ok +flush; diff --git a/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql b/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql new file mode 100644 index 0000000000000..b7b6f13af83cf --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/mysql_cdc.sql @@ -0,0 +1,21 @@ +DROP DATABASE IF EXISTS `my@db`; +CREATE DATABASE `my@db`; + +USE `my@db`; + +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512) +); + +ALTER TABLE products AUTO_INCREMENT = 101; + +INSERT INTO products VALUES (default,"101","101"), +(default,"102","102"), +(default,"103","103"), +(default,"104","104"), +(default,"105","105"), +(default,"106","106"), +(default,"107","107"), +(default,"108","108") diff --git a/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql b/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql new file mode 100644 index 0000000000000..641d6220ea8dc --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/mysql_cdc_insert.sql @@ -0,0 +1,7 @@ +USE `my@db`; + +INSERT INTO products VALUES (default,"109","109"), +(default,"110","110"), +(default,"111","111"), +(default,"112","112"), +(default,"113","113"); diff --git a/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml b/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml new file mode 100644 index 0000000000000..5ab9647b12eb0 --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/no_partition_cdc.toml @@ -0,0 +1,25 @@ +init_sqls = [] + +slt = '' + +verify_schema = ['int','string','string'] + +verify_sql = 'SELECT * FROM demo_db.demo_table ORDER BY id ASC' + +verify_data = """ +101,101,101 +102,102,102 +103,103,103 +104,104,104 +105,105,105 +106,106,106 +107,107,107 +108,108,108 +109,109,109 +110,110,110 +111,111,111 +112,112,112 +113,113,113 +""" + +drop_sqls = [] diff --git a/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml b/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml new file mode 100644 index 0000000000000..17e5f7497aae5 --- /dev/null +++ b/e2e_test/iceberg/test_case/cdc/no_partition_cdc_init.toml @@ -0,0 +1,31 @@ +init_sqls = [ + 'CREATE SCHEMA IF NOT EXISTS demo_db', + 'DROP TABLE IF EXISTS demo_db.demo_table', + ''' + CREATE TABLE demo_db.demo_table ( + id int, + name string, + description string + ) USING iceberg + TBLPROPERTIES ('format-version'='2'); + ''' +] + +slt = 'test_case/cdc/load.slt' + +verify_schema = ['int','string','string'] + +verify_sql = 'SELECT * FROM demo_db.demo_table ORDER BY id ASC' + +verify_data = """ +101,101,101 +102,102,102 +103,103,103 +104,104,104 +105,105,105 +106,106,106 +107,107,107 +108,108,108 +""" + +drop_sqls = [] diff --git a/integration_tests/iceberg-cdc/README.md b/integration_tests/iceberg-cdc/README.md new file mode 100644 index 0000000000000..56f40172c3dfa --- /dev/null +++ b/integration_tests/iceberg-cdc/README.md @@ -0,0 +1,5 @@ +# Iceberg CDC Integration Tests +`mysql -> rw -> iceberg` + +# How to run +./run_test.sh \ No newline at end of file diff --git a/integration_tests/iceberg-cdc/docker-compose.yaml b/integration_tests/iceberg-cdc/docker-compose.yaml new file mode 100644 index 0000000000000..8e9ad1062ef38 --- /dev/null +++ b/integration_tests/iceberg-cdc/docker-compose.yaml @@ -0,0 +1,142 @@ +version: '3.8' + +services: + compactor-0: + extends: + file: ../../docker/docker-compose.yml + service: compactor-0 + compute-node-0: + extends: + file: ../../docker/docker-compose.yml + service: compute-node-0 + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-0 + frontend-node-0: + extends: + file: ../../docker/docker-compose.yml + service: frontend-node-0 + meta-node-0: + extends: + file: ../../docker/docker-compose.yml + service: meta-node-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + mc: + depends_on: + - minio-0 + image: minio/mc + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + entrypoint: > + /bin/sh -c " + until (/usr/bin/mc config host add minio http://minio-0:9301 hummockadmin hummockadmin) do echo '...waiting...' && sleep 1; done; + /usr/bin/mc rm -r --force minio/icebergdata; + /usr/bin/mc mb minio/icebergdata; + /usr/bin/mc anonymous set public minio/icebergdata; + tail -f /dev/null + " + + mysql: + image: mysql:8.0 + expose: + - 3306 + ports: + - "3306:3306" + environment: + - MYSQL_ROOT_PASSWORD=123456 + - MYSQL_USER=mysqluser + - MYSQL_PASSWORD=mysqlpw + - MYSQL_DATABASE=mydb + healthcheck: + test: [ "CMD-SHELL", "mysqladmin ping -h 127.0.0.1 -u root -p123456" ] + interval: 5s + timeout: 5s + retries: 5 + container_name: mysql + prepare_mysql: + image: mysql:8.0 + depends_on: + - mysql + command: + - /bin/sh + - -c + - "mysql -p123456 -h mysql mydb < mysql_prepare.sql" + volumes: + - "./mysql_prepare.sql:/mysql_prepare.sql" + container_name: prepare_mysql + restart: on-failure + + rest: + image: tabulario/iceberg-rest:0.6.0 + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + - CATALOG_CATOLOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog + - CATALOG_URI=jdbc:sqlite:file:/tmp/iceberg_rest_mode=memory + - CATALOG_WAREHOUSE=s3://icebergdata/demo + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio-0:9301 + depends_on: + - minio-0 + # let the rest access minio through: hummock001.minio-0 + links: + - minio-0:icebergdata.minio-0 + expose: + - 8181 + ports: + - "8181:8181" + + spark: + depends_on: + - minio-0 + - rest + image: ghcr.io/icelake-io/icelake-spark:latest + environment: + - AWS_ACCESS_KEY_ID=hummockadmin + - AWS_SECRET_ACCESS_KEY=hummockadmin + - AWS_REGION=us-east-1 + - SPARK_HOME=/opt/spark + - PATH=/usr/local/sbin:/usr/local/bin:/usr/sbin:/usr/bin:/sbin:/bin:/opt/spark/bin:/opt/spark/sbin + user: root + links: + - minio-0:icebergdata.minio-0 + expose: + - 15002 + ports: + - "15002:15002" + healthcheck: + test: netstat -ltn | grep -c 15002 + interval: 1s + retries: 1200 + volumes: + - ./spark:/spark + command: [ "bash", "/spark/spark-connect-server.sh" ] + +volumes: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + spark: + external: false diff --git a/integration_tests/iceberg-cdc/mysql_prepare.sql b/integration_tests/iceberg-cdc/mysql_prepare.sql new file mode 100644 index 0000000000000..3e5a236a41205 --- /dev/null +++ b/integration_tests/iceberg-cdc/mysql_prepare.sql @@ -0,0 +1,15 @@ +-- mysql -p123456 -uroot -h 127.0.0.1 mydb < mysql_prepare.sql +-- +-- Mysql +USE mydb; + +CREATE TABLE user_behaviors ( + user_id VARCHAR(60), + target_id VARCHAR(60), + target_type VARCHAR(60), + event_timestamp VARCHAR(100), + behavior_type VARCHAR(60), + parent_target_type VARCHAR(60), + parent_target_id VARCHAR(60), + PRIMARY KEY(user_id, target_id, event_timestamp) +); diff --git a/integration_tests/iceberg-cdc/python/check.py b/integration_tests/iceberg-cdc/python/check.py new file mode 100644 index 0000000000000..699fa4df29c30 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/check.py @@ -0,0 +1,25 @@ +from pyspark.sql import SparkSession +import configparser +import psycopg2 + +def check_spark_table(args): + expect_row_count = 0 + rw_config = args['risingwave'] + with psycopg2.connect(database=rw_config['db'], user=rw_config['user'], host=rw_config['host'], + port=rw_config['port']) as conn: + with conn.cursor() as cursor: + cursor.execute("SELECT COUNT(*) FROM user_behaviors") + expect_row_count = cursor.fetchone()[0] + print(f"expect_row_count is {expect_row_count}") + spark_config = args['spark'] + spark = SparkSession.builder.remote(spark_config['url']).getOrCreate() + actual_row_count = spark.sql("SELECT COUNT(*) FROM s1.t1").collect()[0][0] + print(f"actual_row_count is {actual_row_count}") + assert actual_row_count==expect_row_count + + +if __name__ == "__main__": + config = configparser.ConfigParser() + config.read("config.ini") + print({section: dict(config[section]) for section in config.sections()}) + check_spark_table(config) diff --git a/integration_tests/iceberg-cdc/python/config.ini b/integration_tests/iceberg-cdc/python/config.ini new file mode 100644 index 0000000000000..bd95eddc5b80e --- /dev/null +++ b/integration_tests/iceberg-cdc/python/config.ini @@ -0,0 +1,8 @@ +[spark] +url=sc://localhost:15002 + +[risingwave] +db=dev +user=root +host=127.0.0.1 +port=4566 diff --git a/integration_tests/iceberg-cdc/python/init.py b/integration_tests/iceberg-cdc/python/init.py new file mode 100644 index 0000000000000..289fa2f161889 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/init.py @@ -0,0 +1,103 @@ +from pyspark.sql import SparkSession +import configparser +import psycopg2 + + +def init_spark_table(args): + spark_config = args['spark'] + spark = SparkSession.builder.remote(spark_config['url']).getOrCreate() + + init_table_sqls = [ + "CREATE SCHEMA IF NOT EXISTS s1", + "DROP TABLE IF EXISTS s1.t1", + """ + CREATE TABLE s1.t1 + ( + user_id string, + target_id string, + target_type string, + event_timestamp string, + behavior_type string, + parent_target_type string, + parent_target_id string + ) USING iceberg + TBLPROPERTIES ('format-version'='2'); + """, + ] + + for sql in init_table_sqls: + print(f"Executing sql: {sql}") + spark.sql(sql) + + +def init_risingwave_mv(args): + rw_config = args['risingwave'] + sqls = [ + "set streaming_parallelism = 4", + """ + CREATE TABLE user_behaviors ( + user_id VARCHAR, + target_id VARCHAR, + target_type VARCHAR, + event_timestamp VARCHAR, + behavior_type VARCHAR, + parent_target_type VARCHAR, + parent_target_id VARCHAR, + PRIMARY KEY(user_id, target_id, event_timestamp) + ) with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'mydb', + table.name = 'user_behaviors', + server.id = '1' + ); + """, + # f""" + # CREATE SINK s1 + # AS SELECT * FROM user_behaviors + # WITH ( + # connector='iceberg', + # type='upsert', + # primary_key = 'user_id, target_id, event_timestamp', + # catalog.type = 'storage', + # s3.endpoint = 'http://minio-0:9301', + # s3.access.key = 'hummockadmin', + # s3.secret.key = 'hummockadmin', + # database.name='demo', + # table.name='s1.t1',warehouse.path = 's3://hummock001/icebergdata/demo',s3.region = 'us-east-1' + # ); + # """ + f""" + CREATE SINK s1 + AS SELECT * FROM user_behaviors + WITH ( + connector='iceberg', + type='upsert', + primary_key = 'user_id, target_id, event_timestamp', + catalog.type = 'rest', + catalog.uri = 'http://rest:8181', + s3.endpoint = 'http://minio-0:9301', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + database.name='demo', + table.name='s1.t1',warehouse.path = 's3://icebergdata/demo/s1/t1',s3.region = 'us-east-1' + ); + """ + ] + with psycopg2.connect(database=rw_config['db'], user=rw_config['user'], host=rw_config['host'], + port=rw_config['port']) as conn: + with conn.cursor() as cursor: + for sql in sqls: + print(f"Executing sql {sql}") + cursor.execute(sql) + + +if __name__ == "__main__": + config = configparser.ConfigParser() + config.read("config.ini") + print({section: dict(config[section]) for section in config.sections()}) + init_spark_table(config) + init_risingwave_mv(config) diff --git a/integration_tests/iceberg-cdc/python/pyproject.toml b/integration_tests/iceberg-cdc/python/pyproject.toml new file mode 100644 index 0000000000000..4c7bce1165796 --- /dev/null +++ b/integration_tests/iceberg-cdc/python/pyproject.toml @@ -0,0 +1,16 @@ +[tool.poetry] +name = "icelake-integration-tests" +version = "0.0.9" +description = "" +authors = ["Renjie Liu "] +readme = "README.md" +packages = [{include = "icelake_integration_tests"}] + +[tool.poetry.dependencies] +python = "^3.11" +pyspark = { version = "3.4.1", extras = ["sql", "connect"] } +psycopg2-binary = "^2.9" + +[build-system] +requires = ["poetry-core"] +build-backend = "poetry.core.masonry.api" diff --git a/integration_tests/iceberg-cdc/run_test.sh b/integration_tests/iceberg-cdc/run_test.sh new file mode 100755 index 0000000000000..2d8b691bc7284 --- /dev/null +++ b/integration_tests/iceberg-cdc/run_test.sh @@ -0,0 +1,19 @@ +#!/bin/bash + +# Start test environment. +docker-compose up -d --wait + +# To avoid exiting by unhealth, set it after start environment. +set -ex + +# Generate data +docker build -t iceberg-cdc-datagen ../datagen +timeout 20 docker run --network=iceberg-cdc_default iceberg-cdc-datagen /datagen --mode clickstream --qps 1 mysql --user mysqluser --password mysqlpw --host mysql --port 3306 --db mydb & + +cd python +poetry update --quiet +# Init source, mv, and sink. +poetry run python init.py +# Wait for sink to be finished. +sleep 40; +poetry run python check.py diff --git a/integration_tests/iceberg-cdc/spark/.gitignore b/integration_tests/iceberg-cdc/spark/.gitignore new file mode 100644 index 0000000000000..51dcf07222856 --- /dev/null +++ b/integration_tests/iceberg-cdc/spark/.gitignore @@ -0,0 +1,3 @@ +derby.log +metastore_db +.ivy \ No newline at end of file diff --git a/integration_tests/iceberg-cdc/spark/spark-connect-server.sh b/integration_tests/iceberg-cdc/spark/spark-connect-server.sh new file mode 100755 index 0000000000000..7c1cd64f1a2f2 --- /dev/null +++ b/integration_tests/iceberg-cdc/spark/spark-connect-server.sh @@ -0,0 +1,21 @@ +#!/bin/bash + +set -ex + +JARS=$(find /opt/spark/deps -type f -name "*.jar" | tr '\n' ':') + +/opt/spark/sbin/start-connect-server.sh \ + --master local[3] \ + --driver-class-path $JARS \ + --conf spark.driver.bindAddress=0.0.0.0 \ + --conf spark.sql.catalog.demo=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions \ + --conf spark.sql.catalog.demo.catalog-impl=org.apache.iceberg.rest.RESTCatalog \ + --conf spark.sql.catalog.demo.uri=http://rest:8181 \ + --conf spark.sql.catalog.demo.s3.endpoint=http://minio-0:9301 \ + --conf spark.sql.catalog.demo.s3.path.style.access=true \ + --conf spark.sql.catalog.demo.s3.access.key=hummockadmin \ + --conf spark.sql.catalog.demo.s3.secret.key=hummockadmin \ + --conf spark.sql.defaultCatalog=demo + +tail -f /opt/spark/logs/spark*.out diff --git a/integration_tests/scripts/run_demos.py b/integration_tests/scripts/run_demos.py index 28623f7ddc4a7..da2519e18db44 100644 --- a/integration_tests/scripts/run_demos.py +++ b/integration_tests/scripts/run_demos.py @@ -42,6 +42,13 @@ def run_demo(demo: str, format: str, wait_time = 40): run_sql_file(sql_file, demo_dir) sleep(10) +def iceberg_cdc_demo(): + demo = "iceberg-cdc" + file_dir = dirname(abspath(__file__)) + project_dir = dirname(file_dir) + demo_dir = os.path.join(project_dir, demo) + print("Running demo: iceberg-cdc") + subprocess.run(["bash","./run_test.sh"], cwd=demo_dir, check=True) def run_iceberg_demo(): demo = "iceberg-sink" @@ -149,5 +156,7 @@ def run_clickhouse_demo(): run_iceberg_demo() elif args.case == "clickhouse-sink": run_clickhouse_demo() +elif args.case == "iceberg-cdc": + iceberg_cdc_demo() else: run_demo(args.case, args.format) From 2d428b153e85973faf8f849cd95e8aaa0690a624 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Wed, 25 Oct 2023 14:03:30 +0800 Subject: [PATCH 5/5] feat(sink): Optimizing ClickHouse Upsert (#12259) --- integration_tests/clickhouse-sink/README.md | 2 + src/connector/src/sink/clickhouse.rs | 410 ++++++++++++-------- 2 files changed, 249 insertions(+), 163 deletions(-) diff --git a/integration_tests/clickhouse-sink/README.md b/integration_tests/clickhouse-sink/README.md index 607621faefeae..a383f3fba5ee4 100644 --- a/integration_tests/clickhouse-sink/README.md +++ b/integration_tests/clickhouse-sink/README.md @@ -23,6 +23,8 @@ docker compose exec clickhouse-server bash /opt/clickhouse/clickhouse-sql/run-sq - create_mv.sql - create_sink.sql +We only support `upsert` with clickhouse' `CollapsingMergeTree` and `VersionedCollapsingMergeTree` + 4. Execute a simple query: ```sh diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index f4fdf9b761f38..fb06baf42920c 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -11,18 +11,17 @@ // 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 core::fmt::Debug; use std::collections::{HashMap, HashSet}; use std::time::Duration; use anyhow::anyhow; -use clickhouse::{Client, Client as ClickHouseClient, Row as ClickHouseRow}; +use clickhouse::{Client as ClickHouseClient, Row as ClickHouseRow}; use itertools::Itertools; -use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::catalog::Schema; use risingwave_common::row::Row; -use risingwave_common::types::{DataType, ScalarRefImpl, Serial}; +use risingwave_common::types::{DataType, Decimal, ScalarRefImpl, Serial}; use serde::ser::{SerializeSeq, SerializeStruct}; use serde::Serialize; use serde_derive::Deserialize; @@ -38,6 +37,10 @@ use crate::sink::{ Result, Sink, SinkError, SinkParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; +const QUERY_ENGINE: &str = + "select distinct ?fields from system.tables where database = ? and table = ?"; +const QUERY_COLUMN: &str = + "select distinct ?fields from system.columns where database = ? and table = ? order by ?"; pub const CLICKHOUSE_SINK: &str = "clickhouse"; const BUFFER_SIZE: usize = 1024; @@ -55,6 +58,75 @@ pub struct ClickHouseCommon { pub table: String, } +#[allow(clippy::enum_variant_names)] +#[derive(Debug)] +enum ClickHouseEngine { + MergeTree, + ReplacingMergeTree, + SummingMergeTree, + AggregatingMergeTree, + CollapsingMergeTree(String), + VersionedCollapsingMergeTree(String), + GraphiteMergeTree, +} +impl ClickHouseEngine { + pub fn is_collapsing_engine(&self) -> bool { + matches!( + self, + ClickHouseEngine::CollapsingMergeTree(_) + | ClickHouseEngine::VersionedCollapsingMergeTree(_) + ) + } + + pub fn get_sign_name(&self) -> Option { + match self { + ClickHouseEngine::CollapsingMergeTree(sign_name) => Some(sign_name.to_string()), + ClickHouseEngine::VersionedCollapsingMergeTree(sign_name) => { + Some(sign_name.to_string()) + } + _ => None, + } + } + + pub fn from_query_engine(engine_name: &ClickhouseQueryEngine) -> Result { + match engine_name.engine.as_str() { + "MergeTree" => Ok(ClickHouseEngine::MergeTree), + "ReplacingMergeTree" => Ok(ClickHouseEngine::ReplacingMergeTree), + "SummingMergeTree" => Ok(ClickHouseEngine::SummingMergeTree), + "AggregatingMergeTree" => Ok(ClickHouseEngine::AggregatingMergeTree), + "VersionedCollapsingMergeTree" => { + let sign_name = engine_name + .create_table_query + .split("VersionedCollapsingMergeTree(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(',') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .to_string(); + Ok(ClickHouseEngine::VersionedCollapsingMergeTree(sign_name)) + } + "CollapsingMergeTree" => { + let sign_name = engine_name + .create_table_query + .split("CollapsingMergeTree(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(')') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .to_string(); + Ok(ClickHouseEngine::CollapsingMergeTree(sign_name)) + } + "GraphiteMergeTree" => Ok(ClickHouseEngine::GraphiteMergeTree), + _ => Err(SinkError::ClickHouse(format!( + "Cannot find clickhouse engine {:?}", + engine_name.engine + ))), + } + } +} + const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(5); impl ClickHouseCommon { @@ -198,9 +270,7 @@ impl ClickHouseSink { } risingwave_common::types::DataType::Float32 => Ok(ck_column.r#type.contains("Float32")), risingwave_common::types::DataType::Float64 => Ok(ck_column.r#type.contains("Float64")), - risingwave_common::types::DataType::Decimal => { - Err(SinkError::ClickHouse("can not support Decimal".to_string())) - } + risingwave_common::types::DataType::Decimal => Ok(ck_column.r#type.contains("Decimal")), risingwave_common::types::DataType::Date => Ok(ck_column.r#type.contains("Date32")), risingwave_common::types::DataType::Varchar => Ok(ck_column.r#type.contains("String")), risingwave_common::types::DataType::Time => Err(SinkError::ClickHouse( @@ -232,7 +302,7 @@ impl ClickHouseSink { Ok(ck_column.r#type.contains("UInt64") | ck_column.r#type.contains("Int64")) } risingwave_common::types::DataType::Int256 => Err(SinkError::ClickHouse( - "clickhouse can not support Interval".to_string(), + "clickhouse can not support Int256".to_string(), )), }; if !is_match? { @@ -264,20 +334,15 @@ impl Sink for ClickHouseSink { // check reachability let client = self.config.common.build_client()?; - let query_column = "select distinct ?fields from system.columns where database = ? and table = ? order by ?".to_string(); - let clickhouse_column = client - .query(&query_column) - .bind(self.config.common.database.clone()) - .bind(self.config.common.table.clone()) - .bind("position") - .fetch_all::() - .await?; - if clickhouse_column.is_empty() { - return Err(SinkError::ClickHouse(format!( - "table {:?}.{:?} is not find in clickhouse", - self.config.common.database, self.config.common.table - ))); + + let (clickhouse_column, clickhouse_engine) = + query_column_engine_from_ck(client, &self.config).await?; + + if !self.is_append_only && !clickhouse_engine.is_collapsing_engine() { + return Err(SinkError::ClickHouse( + "If you want to use upsert, please modify your engine is `VersionedCollapsingMergeTree` or `CollapsingMergeTree` in ClickHouse".to_owned())); } + self.check_column_name_and_type(&clickhouse_column)?; if !self.is_append_only { self.check_pk_match(&clickhouse_column)?; @@ -300,17 +365,20 @@ pub struct ClickHouseSinkWriter { pub config: ClickHouseConfig, schema: Schema, pk_indices: Vec, - client: Client, + client: ClickHouseClient, is_append_only: bool, // Save some features of the clickhouse column type column_correct_vec: Vec, - clickhouse_fields_name: Vec, + rw_fields_name_after_calibration: Vec, + clickhouse_engine: ClickHouseEngine, } #[derive(Debug)] struct ClickHouseSchemaFeature { can_null: bool, // Time accuracy in clickhouse for rw and ck conversions accuracy_time: u8, + + accuracy_decimal: (u8, u8), } impl ClickHouseSinkWriter { @@ -320,25 +388,23 @@ impl ClickHouseSinkWriter { pk_indices: Vec, is_append_only: bool, ) -> Result { - if !is_append_only { - tracing::warn!("Update and delete are not recommended because of their impact on clickhouse performance."); - } let client = config.common.build_client()?; - let query_column = "select distinct ?fields from system.columns where database = ? and table = ? order by position".to_string(); - let clickhouse_column = client - .query(&query_column) - .bind(config.common.database.clone()) - .bind(config.common.table.clone()) - .fetch_all::() - .await?; + + let (clickhouse_column, clickhouse_engine) = + query_column_engine_from_ck(client.clone(), &config).await?; + let column_correct_vec: Result> = clickhouse_column .iter() .map(Self::build_column_correct_vec) .collect(); - let clickhouse_fields_name = build_fields_name_type_from_schema(&schema)? + let mut rw_fields_name_after_calibration = build_fields_name_type_from_schema(&schema)? .iter() .map(|(a, _)| a.clone()) .collect_vec(); + + if let Some(sign) = clickhouse_engine.get_sign_name() { + rw_fields_name_after_calibration.push(sign); + } Ok(Self { config, schema, @@ -346,7 +412,8 @@ impl ClickHouseSinkWriter { client, is_append_only, column_correct_vec: column_correct_vec?, - clickhouse_fields_name, + rw_fields_name_after_calibration, + clickhouse_engine, }) } @@ -368,138 +435,84 @@ impl ClickHouseSinkWriter { } else { 0_u8 }; + let accuracy_decimal = if ck_column.r#type.contains("Decimal(") { + let decimal_all = ck_column + .r#type + .split("Decimal(") + .last() + .ok_or_else(|| SinkError::ClickHouse("must have last".to_string()))? + .split(')') + .next() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .split(", ") + .collect_vec(); + let length = decimal_all + .first() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::ClickHouse(format!("clickhouse sink error {}", e)))?; + + if length > 38 { + return Err(SinkError::ClickHouse( + "RW don't support Decimal256".to_string(), + )); + } + + let scale = decimal_all + .last() + .ok_or_else(|| SinkError::ClickHouse("must have next".to_string()))? + .parse::() + .map_err(|e| SinkError::ClickHouse(format!("clickhouse sink error {}", e)))?; + (length, scale) + } else { + (0_u8, 0_u8) + }; Ok(ClickHouseSchemaFeature { can_null, accuracy_time, + accuracy_decimal, }) } - async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { + async fn write(&mut self, chunk: StreamChunk) -> Result<()> { let mut insert = self.client.insert_with_fields_name( &self.config.common.table, - self.clickhouse_fields_name.clone(), + self.rw_fields_name_after_calibration.clone(), )?; for (op, row) in chunk.rows() { - if op != Op::Insert { - tracing::warn!( - "append only click house sink receive an {:?} which will be ignored.", - op - ); - continue; - } let mut clickhouse_filed_vec = vec![]; for (index, data) in row.iter().enumerate() { clickhouse_filed_vec.extend(ClickHouseFieldWithNull::from_scalar_ref( data, &self.column_correct_vec, index, - true, )?); } - let clickhouse_column = ClickHouseColumn { - row: clickhouse_filed_vec, - }; - insert.write(&clickhouse_column).await?; - } - insert.end().await?; - Ok(()) - } - - async fn upsert(&mut self, chunk: StreamChunk) -> Result<()> { - let get_pk_names_and_data = |row: RowRef<'_>, index: usize| { - let pk_names = self - .schema - .names() - .iter() - .cloned() - .enumerate() - .filter(|(index, _)| self.pk_indices.contains(index)) - .map(|(_, b)| b) - .collect_vec(); - let mut pk_data = vec![]; - for pk_index in &self.pk_indices { - if let ClickHouseFieldWithNull::WithoutSome(v) = - ClickHouseFieldWithNull::from_scalar_ref( - row.datum_at(*pk_index), - &self.column_correct_vec, - index, - false, - )? - .pop() - .unwrap() - { - pk_data.push(v) - } else { - return Err(SinkError::ClickHouse("pk can not be null".to_string())); - } - } - Ok((pk_names, pk_data)) - }; - - for (index, (op, row)) in chunk.rows().enumerate() { match op { - Op::Insert => { - let mut insert = self.client.insert_with_fields_name( - &self.config.common.table, - self.clickhouse_fields_name.clone(), - )?; - let mut clickhouse_filed_vec = vec![]; - for (index, data) in row.iter().enumerate() { - clickhouse_filed_vec.extend(ClickHouseFieldWithNull::from_scalar_ref( - data, - &self.column_correct_vec, - index, - true, - )?); + Op::Insert | Op::UpdateInsert => { + if self.clickhouse_engine.get_sign_name().is_some() { + clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome( + ClickHouseField::Int8(1), + )); } - let clickhouse_column = ClickHouseColumn { - row: clickhouse_filed_vec, - }; - insert.write(&clickhouse_column).await?; - insert.end().await?; } - Op::Delete => { - let (delete_pk_names, delete_pk_data) = get_pk_names_and_data(row, index)?; - self.client - .delete(&self.config.common.table, delete_pk_names) - .delete(delete_pk_data) - .await?; - } - Op::UpdateDelete => continue, - Op::UpdateInsert => { - let (update_pk_names, update_pk_data) = get_pk_names_and_data(row, index)?; - let mut clickhouse_update_filed_vec = vec![]; - for (index, data) in row.iter().enumerate() { - if !self.pk_indices.contains(&index) { - clickhouse_update_filed_vec.extend( - ClickHouseFieldWithNull::from_scalar_ref( - data, - &self.column_correct_vec, - index, - false, - )?, - ); - } + Op::Delete | Op::UpdateDelete => { + if !self.clickhouse_engine.is_collapsing_engine() { + return Err(SinkError::ClickHouse( + "Clickhouse engine don't support upsert".to_string(), + )); } - // Get the names of the columns excluding pk, and use them to update. - let fields_name_update = self - .clickhouse_fields_name - .iter() - .filter(|n| !update_pk_names.contains(n)) - .map(|s| s.to_string()) - .collect_vec(); - - let update = self.client.update( - &self.config.common.table, - update_pk_names, - fields_name_update.clone(), - ); - update - .update_fields(clickhouse_update_filed_vec, update_pk_data) - .await?; + clickhouse_filed_vec.push(ClickHouseFieldWithNull::WithoutSome( + ClickHouseField::Int8(-1), + )) } } + let clickhouse_column = ClickHouseColumn { + row: clickhouse_filed_vec, + }; + insert.write(&clickhouse_column).await?; } + insert.end().await?; Ok(()) } } @@ -510,11 +523,7 @@ impl AsyncTruncateSinkWriter for ClickHouseSinkWriter { chunk: StreamChunk, _add_future: DeliveryFutureManagerAddFuture<'a, Self::DeliveryFuture>, ) -> Result<()> { - if self.is_append_only { - self.append_only(chunk).await - } else { - self.upsert(chunk).await - } + self.write(chunk).await } } @@ -525,6 +534,48 @@ struct SystemColumn { is_in_primary_key: u8, } +#[derive(ClickHouseRow, Deserialize)] +struct ClickhouseQueryEngine { + name: String, + engine: String, + create_table_query: String, +} + +async fn query_column_engine_from_ck( + client: ClickHouseClient, + config: &ClickHouseConfig, +) -> Result<(Vec, ClickHouseEngine)> { + let query_engine = QUERY_ENGINE; + let query_column = QUERY_COLUMN; + + let clickhouse_engine = client + .query(query_engine) + .bind(config.common.database.clone()) + .bind(config.common.table.clone()) + .fetch_all::() + .await?; + let mut clickhouse_column = client + .query(query_column) + .bind(config.common.database.clone()) + .bind(config.common.table.clone()) + .bind("position") + .fetch_all::() + .await?; + if clickhouse_engine.is_empty() || clickhouse_column.is_empty() { + return Err(SinkError::ClickHouse(format!( + "table {:?}.{:?} is not find in clickhouse", + config.common.database, config.common.table + ))); + } + + let clickhouse_engine = ClickHouseEngine::from_query_engine(clickhouse_engine.get(0).unwrap())?; + + if let Some(sign) = &clickhouse_engine.get_sign_name() { + clickhouse_column.retain(|a| sign.ne(&a.name)) + } + Ok((clickhouse_column, clickhouse_engine)) +} + /// Serialize this structure to simulate the `struct` call clickhouse interface #[derive(ClickHouseRow, Debug)] struct ClickHouseColumn { @@ -543,6 +594,26 @@ enum ClickHouseField { String(String), Bool(bool), List(Vec), + Int8(i8), + Decimal(ClickHouseDecimal), +} +#[derive(Debug)] +enum ClickHouseDecimal { + Decimal32(i32), + Decimal64(i64), + Decimal128(i128), +} +impl Serialize for ClickHouseDecimal { + fn serialize(&self, serializer: S) -> std::result::Result + where + S: serde::Serializer, + { + match self { + ClickHouseDecimal::Decimal32(v) => serializer.serialize_i32(*v), + ClickHouseDecimal::Decimal64(v) => serializer.serialize_i64(*v), + ClickHouseDecimal::Decimal128(v) => serializer.serialize_i128(*v), + } + } } /// Enum that support clickhouse nullable @@ -558,7 +629,6 @@ impl ClickHouseFieldWithNull { data: Option>, clickhouse_schema_feature_vec: &Vec, clickhouse_schema_feature_index: usize, - is_insert: bool, ) -> Result> { let clickhouse_schema_feature = clickhouse_schema_feature_vec .get(clickhouse_schema_feature_index) @@ -586,8 +656,29 @@ impl ClickHouseFieldWithNull { ScalarRefImpl::Float64(v) => ClickHouseField::Float64(v.into_inner()), ScalarRefImpl::Utf8(v) => ClickHouseField::String(v.to_string()), ScalarRefImpl::Bool(v) => ClickHouseField::Bool(v), - ScalarRefImpl::Decimal(_) => { - return Err(SinkError::ClickHouse("can not support Decimal".to_string())) + ScalarRefImpl::Decimal(d) => { + if let Decimal::Normalized(d) = d { + let scale = + clickhouse_schema_feature.accuracy_decimal.1 as i32 - d.scale() as i32; + + let scale = if scale < 0 { + d.mantissa() / 10_i128.pow(scale.unsigned_abs()) + } else { + d.mantissa() * 10_i128.pow(scale as u32) + }; + + if clickhouse_schema_feature.accuracy_decimal.0 <= 9 { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal32(scale as i32)) + } else if clickhouse_schema_feature.accuracy_decimal.0 <= 18 { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal64(scale as i64)) + } else { + ClickHouseField::Decimal(ClickHouseDecimal::Decimal128(scale)) + } + } else { + return Err(SinkError::ClickHouse( + "clickhouse can not support Decimal NAN,-INF and INF".to_string(), + )); + } } ScalarRefImpl::Interval(_) => { return Err(SinkError::ClickHouse( @@ -604,14 +695,9 @@ impl ClickHouseFieldWithNull { )) } ScalarRefImpl::Timestamp(v) => { - if is_insert { - let time = v.get_timestamp_nanos() - / 10_i32.pow((9 - clickhouse_schema_feature.accuracy_time).into()) as i64; - ClickHouseField::Int64(time) - } else { - let time = v.truncate_micros().to_string(); - ClickHouseField::String(time) - } + let time = v.get_timestamp_nanos() + / 10_i32.pow((9 - clickhouse_schema_feature.accuracy_time).into()) as i64; + ClickHouseField::Int64(time) } ScalarRefImpl::Timestamptz(_) => { return Err(SinkError::ClickHouse( @@ -630,7 +716,6 @@ impl ClickHouseFieldWithNull { field, clickhouse_schema_feature_vec, clickhouse_schema_feature_index + index, - is_insert, )?; struct_vec.push(ClickHouseFieldWithNull::WithoutSome(ClickHouseField::List( a, @@ -645,7 +730,6 @@ impl ClickHouseFieldWithNull { i, clickhouse_schema_feature_vec, clickhouse_schema_feature_index, - is_insert, )?) } return Ok(vec![ClickHouseFieldWithNull::WithoutSome( @@ -658,9 +742,7 @@ impl ClickHouseFieldWithNull { )) } }; - // Insert needs to be serialized with `Some`, update doesn't need to be serialized with - // `Some` - let data = if is_insert && clickhouse_schema_feature.can_null { + let data = if clickhouse_schema_feature.can_null { vec![ClickHouseFieldWithNull::WithSome(data)] } else { vec![ClickHouseFieldWithNull::WithoutSome(data)] @@ -690,6 +772,8 @@ impl Serialize for ClickHouseField { } s.end() } + ClickHouseField::Decimal(v) => v.serialize(serializer), + ClickHouseField::Int8(v) => serializer.serialize_i8(*v), } } }