From 9da062781b394e2a4f0e1679d1ae8a06e39a3709 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Tue, 9 Jul 2024 14:48:38 +0800 Subject: [PATCH 01/20] feat: compact_noop_update_after_outer_join (#17568) --- src/common/src/array/stream_chunk.rs | 4 ++ src/stream/src/executor/hash_join.rs | 75 +++++++++++++++++++++++-- src/stream/src/executor/join/builder.rs | 62 +++++++++++++++++++- 3 files changed, 132 insertions(+), 9 deletions(-) diff --git a/src/common/src/array/stream_chunk.rs b/src/common/src/array/stream_chunk.rs index 917d484458518..cfee990561aa1 100644 --- a/src/common/src/array/stream_chunk.rs +++ b/src/common/src/array/stream_chunk.rs @@ -525,6 +525,10 @@ impl OpRowMutRef<'_> { } impl StreamChunkMut { + pub fn capacity(&self) -> usize { + self.vis.len() + } + pub fn vis(&self, i: usize) -> bool { self.vis.is_set(i) } diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 0bf825138d92a..66e91cf2ecf2a 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -269,8 +269,10 @@ impl HashJoinExecutor StreamExecutorResult<()> { + let (mut tx_l, mut tx_r, mut hash_join) = + create_classical_executor::<{ JoinType::FullOuter }>(false, false, None).await; + + // push the init barrier for left and right + tx_l.push_barrier(test_epoch(1), false); + tx_r.push_barrier(test_epoch(1), false); + hash_join.next_unwrap_ready_barrier()?; + + tx_l.push_chunk(StreamChunk::from_pretty( + " I I + + 1 1 + ", + )); + let chunk = hash_join.next_unwrap_ready_chunk()?; + assert_eq!( + chunk, + StreamChunk::from_pretty( + " I I I I + + 1 1 . ." + ) + ); + + tx_r.push_chunk(StreamChunk::from_pretty( + " I I + + 1 1 + ", + )); + let chunk = hash_join.next_unwrap_ready_chunk()?; + + assert_eq!( + chunk, + StreamChunk::from_pretty( + " I I I I + U- 1 1 . . + U+ 1 1 1 1" + ) + ); + + tx_l.push_chunk(StreamChunk::from_pretty( + " I I + - 1 1 + + 1 2 + ", + )); + let chunk = hash_join.next_unwrap_ready_chunk()?; + let chunk = chunk.compact(); + assert_eq!( + chunk, + StreamChunk::from_pretty( + " I I I I + - 1 1 1 1 + + 1 2 1 1 + " + ) + ); + + Ok(()) + } + #[tokio::test] async fn test_streaming_hash_full_outer_join_with_nonequi_condition() -> StreamExecutorResult<()> { @@ -3093,8 +3156,8 @@ mod tests { chunk, StreamChunk::from_pretty( " I I I I - U- 2 5 . . - U+ 2 5 2 6 + U- 2 5 . . + U+ 2 5 2 6 + . . 4 8 + . . 3 4" /* regression test (#2420): 3 4 should be forwarded only once * despite matching on eq join on 2 diff --git a/src/stream/src/executor/join/builder.rs b/src/stream/src/executor/join/builder.rs index 72208aa45ded8..c39e385fdee96 100644 --- a/src/stream/src/executor/join/builder.rs +++ b/src/stream/src/executor/join/builder.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::array::stream_chunk::StreamChunkMut; use risingwave_common::array::stream_chunk_builder::StreamChunkBuilder; use risingwave_common::array::{Op, RowRef, StreamChunk}; use risingwave_common::row::{OwnedRow, Row}; @@ -156,6 +157,48 @@ impl JoinChunkBuilder } } + pub fn post_process(c: StreamChunk) -> StreamChunk { + let mut c = StreamChunkMut::from(c); + + // NOTE(st1page): remove the pattern `UpdateDel(k, old), UpdateIns(k, NULL), UpdateDel(k, NULL), UpdateIns(k, new)` + // to avoid this issue + let mut i = 2; + while i < c.capacity() { + if c.op(i - 1) == Op::UpdateInsert + && c.op(i) == Op::UpdateDelete + && c.row_ref(i) == c.row_ref(i - 1) + { + if c.op(i - 2) == Op::UpdateDelete && c.op(i + 1) == Op::UpdateInsert { + c.set_op(i - 2, Op::Delete); + c.set_vis(i - 1, false); + c.set_vis(i, false); + c.set_op(i + 1, Op::Insert); + i += 3; + } else { + debug_assert!( + false, + "unexpected Op sequences {:?}, {:?}, {:?}, {:?}", + c.op(i - 2), + c.op(i - 1), + c.op(i), + c.op(i + 1) + ); + warn!( + "unexpected Op sequences {:?}, {:?}, {:?}, {:?}", + c.op(i - 2), + c.op(i - 1), + c.op(i), + c.op(i + 1) + ); + i += 1; + } + } else { + i += 1; + } + } + c.into() + } + pub fn with_match_on_insert( &mut self, row: &RowRef<'_>, @@ -166,6 +209,7 @@ impl JoinChunkBuilder if matched_row.is_zero_degree() && only_forward_matched_side(T, SIDE) { self.stream_chunk_builder .append_row_matched(Op::Delete, &matched_row.row) + .map(Self::post_process) } else { None } @@ -174,6 +218,7 @@ impl JoinChunkBuilder if matched_row.is_zero_degree() && only_forward_matched_side(T, SIDE) { self.stream_chunk_builder .append_row_matched(Op::Insert, &matched_row.row) + .map(Self::post_process) } else { None } @@ -191,10 +236,12 @@ impl JoinChunkBuilder } self.stream_chunk_builder .append_row(Op::UpdateInsert, row, &matched_row.row) + .map(Self::post_process) // Inner sides } else { self.stream_chunk_builder .append_row(Op::Insert, row, &matched_row.row) + .map(Self::post_process) } } @@ -208,6 +255,7 @@ impl JoinChunkBuilder if matched_row.is_zero_degree() && only_forward_matched_side(T, SIDE) { self.stream_chunk_builder .append_row_matched(Op::Insert, &matched_row.row) + .map(Self::post_process) } else { None } @@ -216,6 +264,7 @@ impl JoinChunkBuilder if matched_row.is_zero_degree() && only_forward_matched_side(T, SIDE) { self.stream_chunk_builder .append_row_matched(Op::Delete, &matched_row.row) + .map(Self::post_process) } else { None } @@ -232,6 +281,8 @@ impl JoinChunkBuilder } self.stream_chunk_builder .append_row_matched(Op::UpdateInsert, &matched_row.row) + .map(|c: StreamChunk| Self::post_process(c)) + // Inner sides } else { // concat with the matched_row and append the new @@ -241,6 +292,7 @@ impl JoinChunkBuilder // the assumption for U+ after U-. self.stream_chunk_builder .append_row(Op::Delete, row, &matched_row.row) + .map(Self::post_process) } } @@ -252,7 +304,9 @@ impl JoinChunkBuilder ) -> Option { // if it's a semi join and the side needs to be maintained. if is_semi(T) && forward_exactly_once(T, SIDE) { - self.stream_chunk_builder.append_row_update(op, row) + self.stream_chunk_builder + .append_row_update(op, row) + .map(Self::post_process) } else { None } @@ -262,7 +316,9 @@ impl JoinChunkBuilder pub fn forward_if_not_matched(&mut self, op: Op, row: RowRef<'_>) -> Option { // if it's outer join or anti join and the side needs to be maintained. if (is_anti(T) && forward_exactly_once(T, SIDE)) || is_outer_side(T, SIDE) { - self.stream_chunk_builder.append_row_update(op, row) + self.stream_chunk_builder + .append_row_update(op, row) + .map(Self::post_process) } else { None } @@ -270,6 +326,6 @@ impl JoinChunkBuilder #[inline] pub fn take(&mut self) -> Option { - self.stream_chunk_builder.take() + self.stream_chunk_builder.take().map(Self::post_process) } } From 8805830552ec8b09a4308f1635b8b478295afe2e Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 9 Jul 2024 15:23:24 +0800 Subject: [PATCH 02/20] refactor: graceful shutdown on compactor node (#17575) Signed-off-by: Bugen Zhao --- src/cmd/src/lib.rs | 3 +- src/cmd_all/src/standalone.rs | 3 +- src/storage/compactor/src/lib.rs | 19 +-- src/storage/compactor/src/server.rs | 131 ++++++++---------- .../src/compaction_test_runner.rs | 3 +- src/tests/simulation/src/cluster.rs | 7 +- 6 files changed, 76 insertions(+), 90 deletions(-) diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 0e711458d5196..7d21fc44b4e57 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -55,8 +55,7 @@ pub fn frontend(opts: FrontendOpts) -> ! { pub fn compactor(opts: CompactorOpts) -> ! { init_risingwave_logger(LoggerSettings::from_opts(&opts)); - // TODO(shutdown): pass the shutdown token - main_okk(|_| risingwave_compactor::start(opts)); + main_okk(|shutdown| risingwave_compactor::start(opts, shutdown)); } pub fn ctl(opts: CtlOpts) -> ! { diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 791921fc07d69..4f8c208c89aa3 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -230,8 +230,9 @@ pub async fn standalone( } if let Some(opts) = compactor_opts { tracing::info!("starting compactor-node thread with cli args: {:?}", opts); + let shutdown = shutdown.clone(); let _compactor_handle = - tokio::spawn(async move { risingwave_compactor::start(opts).await }); + tokio::spawn(async move { risingwave_compactor::start(opts, shutdown).await }); } // wait for log messages to be flushed diff --git a/src/storage/compactor/src/lib.rs b/src/storage/compactor/src/lib.rs index d9fbe5189f43f..22e70ac759aed 100644 --- a/src/storage/compactor/src/lib.rs +++ b/src/storage/compactor/src/lib.rs @@ -24,6 +24,7 @@ use risingwave_common::config::{ AsyncStackTraceOption, CompactorMode, MetricLevel, OverrideConfig, }; use risingwave_common::util::meta_addr::MetaAddressStrategy; +use risingwave_common::util::tokio_util::sync::CancellationToken; use crate::server::{compactor_serve, shared_compactor_serve}; @@ -108,7 +109,10 @@ impl risingwave_common::opts::Opts for CompactorOpts { use std::future::Future; use std::pin::Pin; -pub fn start(opts: CompactorOpts) -> Pin + Send>> { +pub fn start( + opts: CompactorOpts, + shutdown: CancellationToken, +) -> Pin + Send>> { // WARNING: don't change the function signature. Making it `async fn` will cause // slow compile in release mode. match opts.compactor_mode { @@ -118,11 +122,7 @@ pub fn start(opts: CompactorOpts) -> Pin + Send>> { let listen_addr = opts.listen_addr.parse().unwrap(); - let (join_handle, _shutdown_sender) = shared_compactor_serve(listen_addr, opts).await; - - tracing::info!("Server listening at {}", listen_addr); - - join_handle.await.unwrap(); + shared_compactor_serve(listen_addr, opts, shutdown).await; }), None | Some(CompactorMode::Dedicated) => Box::pin(async move { tracing::info!("Compactor node options: {:?}", opts); @@ -140,13 +140,8 @@ pub fn start(opts: CompactorOpts) -> Pin + Send>> { .parse() .unwrap(); tracing::info!(" address is {}", advertise_addr); - let (join_handle, observer_join_handle, _shutdown_sender) = - compactor_serve(listen_addr, advertise_addr, opts).await; - - tracing::info!("Server listening at {}", listen_addr); - join_handle.await.unwrap(); - observer_join_handle.abort(); + compactor_serve(listen_addr, advertise_addr, opts, shutdown).await; }), } } diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index 41091e142b838..8d246016858ef 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -26,6 +26,7 @@ use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; +use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::metrics_manager::MetricsManager; @@ -52,8 +53,6 @@ use risingwave_storage::monitor::{ }; use risingwave_storage::opts::StorageOpts; use tokio::sync::mpsc; -use tokio::sync::oneshot::Sender; -use tokio::task::JoinHandle; use tonic::transport::Endpoint; use tracing::info; @@ -174,11 +173,14 @@ pub async fn prepare_start_parameters( } /// Fetches and runs compaction tasks. +/// +/// Returns when the `shutdown` token is triggered. pub async fn compactor_serve( listen_addr: SocketAddr, advertise_addr: HostAddr, opts: CompactorOpts, -) -> (JoinHandle<()>, JoinHandle<()>, Sender<()>) { + shutdown: CancellationToken, +) { let config = load_config(&opts.config_path, &opts); info!("Starting compactor node",); info!("> config: {:?}", config); @@ -200,7 +202,6 @@ pub async fn compactor_serve( .unwrap(); info!("Assigned compactor id {}", meta_client.worker_id()); - meta_client.activate(&advertise_addr).await.unwrap(); let hummock_metrics = Arc::new(GLOBAL_HUMMOCK_METRICS.clone()); @@ -234,7 +235,7 @@ pub async fn compactor_serve( // use half of limit because any memory which would hold in meta-cache will be allocate by // limited at first. - let observer_join_handle = observer_manager.start().await; + let _observer_join_handle = observer_manager.start().await; let sstable_object_id_manager = Arc::new(SstableObjectIdManager::new( hummock_meta_client.clone(), @@ -259,6 +260,8 @@ pub async fn compactor_serve( task_progress_manager: Default::default(), await_tree_reg: await_tree_reg.clone(), }; + + // TODO(shutdown): don't collect sub-tasks as there's no need to gracefully shutdown them. let mut sub_tasks = vec![ MetaClient::start_heartbeat_loop( meta_client.clone(), @@ -287,50 +290,42 @@ pub async fn compactor_serve( let compactor_srv = CompactorServiceImpl::default(); let monitor_srv = MonitorServiceImpl::new(await_tree_reg); - let (shutdown_send, mut shutdown_recv) = tokio::sync::oneshot::channel(); - let join_handle = tokio::spawn(async move { - tonic::transport::Server::builder() - .add_service(CompactorServiceServer::new(compactor_srv)) - .add_service(MonitorServiceServer::new(monitor_srv)) - .monitored_serve_with_shutdown( - listen_addr, - "grpc-compactor-node-service", - TcpConfig { - tcp_nodelay: true, - keepalive_duration: None, - }, - async move { - tokio::select! { - _ = tokio::signal::ctrl_c() => {}, - _ = &mut shutdown_recv => { - for (join_handle, shutdown_sender) in sub_tasks { - if shutdown_sender.send(()).is_err() { - tracing::warn!("Failed to send shutdown"); - continue; - } - if join_handle.await.is_err() { - tracing::warn!("Failed to join shutdown"); - } - } - }, - } - }, - ) - .await - }); + let server = tonic::transport::Server::builder() + .add_service(CompactorServiceServer::new(compactor_srv)) + .add_service(MonitorServiceServer::new(monitor_srv)) + .monitored_serve_with_shutdown( + listen_addr, + "grpc-compactor-node-service", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + shutdown.clone().cancelled_owned(), + ); + let _server_handle = tokio::spawn(server); // Boot metrics service. if config.server.metrics_level > MetricLevel::Disabled { MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone()); } - (join_handle, observer_join_handle, shutdown_send) + // All set, let the meta service know we're ready. + meta_client.activate(&advertise_addr).await.unwrap(); + + // Wait for the shutdown signal. + shutdown.cancelled().await; + // Run shutdown logic. + meta_client.try_unregister().await; } +/// Fetches and runs compaction tasks under shared mode. +/// +/// Returns when the `shutdown` token is triggered. pub async fn shared_compactor_serve( listen_addr: SocketAddr, opts: CompactorOpts, -) -> (JoinHandle<()>, Sender<()>) { + shutdown: CancellationToken, +) { let config = load_config(&opts.config_path, &opts); info!("Starting shared compactor node",); info!("> config: {:?}", config); @@ -373,7 +368,6 @@ pub async fn shared_compactor_serve( // Run a background heap profiler heap_profiler.start(); - let (shutdown_send, mut shutdown_recv) = tokio::sync::oneshot::channel(); let compaction_executor = Arc::new(CompactionExecutor::new( opts.compaction_worker_threads_number, )); @@ -387,44 +381,35 @@ pub async fn shared_compactor_serve( task_progress_manager: Default::default(), await_tree_reg, }; - let join_handle = tokio::spawn(async move { - tonic::transport::Server::builder() - .add_service(CompactorServiceServer::new(compactor_srv)) - .add_service(MonitorServiceServer::new(monitor_srv)) - .monitored_serve_with_shutdown( - listen_addr, - "grpc-compactor-node-service", - TcpConfig { - tcp_nodelay: true, - keepalive_duration: None, - }, - async move { - let (join_handle, shutdown_sender) = - risingwave_storage::hummock::compactor::start_shared_compactor( - grpc_proxy_client, - receiver, - compactor_context, - ); - tokio::select! { - _ = tokio::signal::ctrl_c() => {}, - _ = &mut shutdown_recv => { - if shutdown_sender.send(()).is_err() { - tracing::warn!("Failed to send shutdown"); - } - if join_handle.await.is_err() { - tracing::warn!("Failed to join shutdown"); - } - }, - } - }, - ) - .await - }); + + risingwave_storage::hummock::compactor::start_shared_compactor( + grpc_proxy_client, + receiver, + compactor_context, + ); + + let server = tonic::transport::Server::builder() + .add_service(CompactorServiceServer::new(compactor_srv)) + .add_service(MonitorServiceServer::new(monitor_srv)) + .monitored_serve_with_shutdown( + listen_addr, + "grpc-compactor-node-service", + TcpConfig { + tcp_nodelay: true, + keepalive_duration: None, + }, + shutdown.clone().cancelled_owned(), + ); + + let _server_handle = tokio::spawn(server); // Boot metrics service. if config.server.metrics_level > MetricLevel::Disabled { MetricsManager::boot_metrics_service(opts.prometheus_listener_addr.clone()); } - (join_handle, shutdown_send) + // Wait for the shutdown signal. + shutdown.cancelled().await; + + // TODO(shutdown): shall we notify the proxy that we are shutting down? } diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index f294d8fb6fe5a..195715e130877 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -30,6 +30,7 @@ use risingwave_common::config::{ }; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, FIRST_VERSION_ID}; @@ -172,7 +173,7 @@ async fn start_compactor_node( "--config-path", &config_path, ]); - risingwave_compactor::start(opts).await + risingwave_compactor::start(opts, CancellationToken::new() /* dummy */).await } pub fn start_compactor_thread( diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index e9a3ff6a212f5..c733288757ba2 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -530,7 +530,12 @@ impl Cluster { .create_node() .name(format!("compactor-{i}")) .ip([192, 168, 4, i as u8].into()) - .init(move || risingwave_compactor::start(opts.clone())) + .init(move || { + risingwave_compactor::start( + opts.clone(), + CancellationToken::new(), // dummy + ) + }) .build(); } From 9dabb6e5c158ab87603dfa31ec513f5e56811283 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 9 Jul 2024 15:39:22 +0800 Subject: [PATCH 03/20] refactor(storage): remove global epoch in uploader (#17500) --- .../event_handler/hummock_event_handler.rs | 5 +- .../src/hummock/event_handler/uploader.rs | 284 ++++++++++-------- 2 files changed, 161 insertions(+), 128 deletions(-) diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index e9d2426862986..b126974c7c085 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -469,9 +469,9 @@ impl HummockEventHandler { table_ids: HashSet, ) { debug!( - "awaiting for epoch to be synced: {}, max_synced_epoch: {}", new_sync_epoch, - self.uploader.max_synced_epoch() + ?table_ids, + "awaiting for epoch to be synced", ); self.uploader .start_sync_epoch(new_sync_epoch, sync_result_sender, table_ids); @@ -481,7 +481,6 @@ impl HummockEventHandler { info!( prev_epoch, max_committed_epoch = self.uploader.max_committed_epoch(), - max_synced_epoch = self.uploader.max_synced_epoch(), "handle clear event" ); diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 453d861330059..55761f1216183 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -461,6 +461,8 @@ struct SyncDataBuilder { spilled_data: SpilledData, table_watermarks: HashMap, + + table_ids_to_ack: HashSet, } impl SyncDataBuilder { @@ -718,7 +720,7 @@ impl LocalInstanceUnsyncData { assert!(latest_epoch_data.is_empty()); assert!(!latest_epoch_data.has_spilled); if cfg!(debug_assertions) { - panic!("sync epoch exceeds latest epoch, and the current instance should have be archived"); + panic!("sync epoch exceeds latest epoch, and the current instance should have been archived"); } warn!( instance_id = self.instance_id, @@ -751,14 +753,19 @@ struct TableUnsyncData { WatermarkDirection, BTreeMap, BitmapBuilder)>, )>, + // newer epoch at the front + syncing_epochs: VecDeque, + max_synced_epoch: Option, } impl TableUnsyncData { - fn new(table_id: TableId) -> Self { + fn new(table_id: TableId, committed_epoch: Option) -> Self { Self { table_id, instance_data: Default::default(), table_watermarks: None, + syncing_epochs: Default::default(), + max_synced_epoch: committed_epoch, } } @@ -772,6 +779,10 @@ impl TableUnsyncData { impl Iterator)>, )>, ) { + if let Some(prev_epoch) = self.max_sync_epoch() { + assert_gt!(epoch, prev_epoch) + } + self.syncing_epochs.push_front(epoch); ( self.instance_data .iter_mut() @@ -790,6 +801,31 @@ impl TableUnsyncData { ) } + fn ack_synced(&mut self, sync_epoch: HummockEpoch) { + let min_sync_epoch = self.syncing_epochs.pop_back().expect("should exist"); + assert_eq!(sync_epoch, min_sync_epoch); + self.max_synced_epoch = Some(sync_epoch); + } + + fn ack_committed(&mut self, committed_epoch: HummockEpoch) { + let synced_epoch_advanced = { + if let Some(max_synced_epoch) = self.max_synced_epoch + && max_synced_epoch >= committed_epoch + { + false + } else { + true + } + }; + if synced_epoch_advanced { + self.max_synced_epoch = Some(committed_epoch); + if let Some(min_syncing_epoch) = self.syncing_epochs.back() { + assert_gt!(*min_syncing_epoch, committed_epoch); + } + self.assert_after_epoch(committed_epoch); + } + } + fn assert_after_epoch(&self, epoch: HummockEpoch) { self.instance_data .values() @@ -800,6 +836,17 @@ impl TableUnsyncData { assert_gt!(*oldest_epoch, epoch); } } + + fn max_sync_epoch(&self) -> Option { + self.syncing_epochs + .front() + .cloned() + .or(self.max_synced_epoch) + } + + fn is_empty(&self) -> bool { + self.instance_data.is_empty() && self.syncing_epochs.is_empty() + } } #[derive(Default)] @@ -820,15 +867,27 @@ impl UnsyncData { table_id: TableId, instance_id: LocalInstanceId, init_epoch: HummockEpoch, + context: &UploaderContext, ) { debug!( table_id = table_id.table_id, instance_id, init_epoch, "init epoch" ); - let table_data = self - .table_data - .entry(table_id) - .or_insert_with(|| TableUnsyncData::new(table_id)); + let table_data = self.table_data.entry(table_id).or_insert_with(|| { + TableUnsyncData::new( + table_id, + context + .pinned_version + .version() + .state_table_info + .info() + .get(&table_id) + .map(|info| info.committed_epoch), + ) + }); + if let Some(max_prev_epoch) = table_data.max_sync_epoch() { + assert_gt!(init_epoch, max_prev_epoch); + } assert!(table_data .instance_data .insert( @@ -890,7 +949,7 @@ impl UnsyncData { debug!(instance_id, "destroy instance"); let table_data = self.table_data.get_mut(&table_id).expect("should exist"); assert!(table_data.instance_data.remove(&instance_id).is_some()); - if table_data.instance_data.is_empty() { + if table_data.is_empty() { self.table_data.remove(&table_id); } } @@ -900,7 +959,7 @@ impl UnsyncData { &mut self, epoch: HummockEpoch, context: &UploaderContext, - table_ids: HashSet, + table_ids: &HashSet, ) -> SyncDataBuilder { let sync_epoch_data = take_before_epoch(&mut self.epoch_data, epoch); @@ -910,11 +969,13 @@ impl UnsyncData { } let mut flush_payload = HashMap::new(); + let mut table_ids_to_ack = HashSet::new(); for (table_id, table_data) in &mut self.table_data { if !table_ids.contains(table_id) { table_data.assert_after_epoch(epoch); continue; } + table_ids_to_ack.insert(*table_id); let (unflushed_payload, table_watermarks) = table_data.sync(epoch); for (instance_id, payload) in unflushed_payload { if !payload.is_empty() { @@ -926,6 +987,7 @@ impl UnsyncData { } } sync_data.flush(context, flush_payload); + sync_data.table_ids_to_ack = table_ids_to_ack; sync_data } @@ -941,6 +1003,9 @@ impl UnsyncData { struct SyncingData { sync_epoch: HummockEpoch, + table_ids: HashSet, + /// Subset of `table_ids` that has existing instance + table_ids_to_ack: HashSet, // task of newer data at the front uploading_tasks: VecDeque, // newer data at the front @@ -1020,7 +1085,7 @@ enum UploaderState { /// Data have 3 sequential stages: unsync (inside each local instance, data can be unsealed, sealed), syncing, synced. /// /// The 3 stages are divided by 2 marginal epochs: `max_syncing_epoch`, -/// `max_synced_epoch`. Epochs satisfy the following inequality. +/// `max_synced_epoch` in each `TableUnSyncData`. Epochs satisfy the following inequality. /// /// (epochs of `synced_data`) <= `max_synced_epoch` < (epochs of `syncing_data`) <= /// `max_syncing_epoch` < (epochs of `unsync_data`) @@ -1028,11 +1093,6 @@ enum UploaderState { /// Data are mostly stored in `VecDeque`, and the order stored in the `VecDeque` indicates the data /// order. Data at the front represents ***newer*** data. pub struct HummockUploader { - /// The maximum epoch that has started syncing - max_syncing_epoch: HummockEpoch, - /// The maximum epoch that has been synced - max_synced_epoch: HummockEpoch, - state: UploaderState, context: UploaderContext, @@ -1046,14 +1106,8 @@ impl HummockUploader { buffer_tracker: BufferTracker, config: &StorageOpts, ) -> Self { - let initial_epoch = pinned_version.version().max_committed_epoch; Self { - max_syncing_epoch: initial_epoch, - max_synced_epoch: initial_epoch, - state: UploaderState::Working(UploaderData { - unsync_data: Default::default(), - syncing_data: Default::default(), - }), + state: UploaderState::Working(UploaderData::default()), context: UploaderContext::new( pinned_version, spawn_upload_task, @@ -1068,10 +1122,6 @@ impl HummockUploader { &self.context.buffer_tracker } - pub(super) fn max_synced_epoch(&self) -> HummockEpoch { - self.max_synced_epoch - } - pub(super) fn max_committed_epoch(&self) -> HummockEpoch { self.context.pinned_version.max_committed_epoch() } @@ -1097,9 +1147,8 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(init_epoch, self.max_syncing_epoch); data.unsync_data - .init_instance(table_id, instance_id, init_epoch); + .init_instance(table_id, instance_id, init_epoch, &self.context); } pub(super) fn local_seal_epoch( @@ -1111,7 +1160,6 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - assert_gt!(next_epoch, self.max_syncing_epoch); data.unsync_data .local_seal_epoch(instance_id, next_epoch, opts); } @@ -1136,17 +1184,9 @@ impl HummockUploader { return; } }; - debug!("start sync epoch: {}", epoch); - assert!( - epoch > self.max_syncing_epoch, - "the epoch {} has started syncing already: {}", - epoch, - self.max_syncing_epoch - ); + debug!(epoch, ?table_ids, "start sync epoch"); - self.max_syncing_epoch = epoch; - - let sync_data = data.unsync_data.sync(epoch, &self.context, table_ids); + let sync_data = data.unsync_data.sync(epoch, &self.context, &table_ids); let SyncDataBuilder { spilled_data: @@ -1155,11 +1195,14 @@ impl HummockUploader { uploaded_data, }, table_watermarks, - .. + table_ids_to_ack, + epochs: _, } = sync_data; data.syncing_data.push_front(SyncingData { sync_epoch: epoch, + table_ids, + table_ids_to_ack, uploading_tasks, uploaded: uploaded_data, table_watermarks, @@ -1172,53 +1215,20 @@ impl HummockUploader { .set(data.syncing_data.len() as _); } - fn set_max_synced_epoch( - max_synced_epoch: &mut HummockEpoch, - max_syncing_epoch: HummockEpoch, - epoch: HummockEpoch, - ) { - assert!( - epoch <= max_syncing_epoch, - "epoch {} that has been synced has not started syncing yet. previous max syncing epoch {}", - epoch, - max_syncing_epoch - ); - assert!( - epoch > *max_synced_epoch, - "epoch {} has been synced. previous max synced epoch: {}", - epoch, - max_synced_epoch - ); - *max_synced_epoch = epoch; - } - pub(crate) fn update_pinned_version(&mut self, pinned_version: PinnedVersion) { assert_ge!( pinned_version.max_committed_epoch(), self.context.pinned_version.max_committed_epoch() ); - let max_committed_epoch = pinned_version.max_committed_epoch(); - self.context.pinned_version = pinned_version; - if self.max_synced_epoch < max_committed_epoch { - self.max_synced_epoch = max_committed_epoch; - } - if self.max_syncing_epoch < max_committed_epoch { - self.max_syncing_epoch = max_committed_epoch; - if let UploaderState::Working(data) = &self.state { - for instance_data in data - .unsync_data - .table_data - .values() - .flat_map(|data| data.instance_data.values()) - { - if let Some(oldest_epoch) = instance_data.sealed_data.back() { - assert_gt!(oldest_epoch.epoch, max_committed_epoch); - } else if let Some(current_epoch) = &instance_data.current_epoch_data { - assert_gt!(current_epoch.epoch, max_committed_epoch); - } + if let UploaderState::Working(data) = &mut self.state { + // TODO: may only `ack_committed` on table whose `committed_epoch` is changed. + for (table_id, info) in pinned_version.version().state_table_info.info() { + if let Some(table_data) = data.unsync_data.table_data.get_mut(table_id) { + table_data.ack_committed(info.committed_epoch); } } } + self.context.pinned_version = pinned_version; } pub(crate) fn may_flush(&mut self) -> bool { @@ -1257,16 +1267,11 @@ impl HummockUploader { } pub(crate) fn clear(&mut self) { - let max_committed_epoch = self.context.pinned_version.max_committed_epoch(); - self.max_synced_epoch = max_committed_epoch; - self.max_syncing_epoch = max_committed_epoch; if let UploaderState::Working(data) = replace( &mut self.state, UploaderState::Working(UploaderData::default()), ) { - data.abort(|| { - HummockError::other(format!("uploader is reset to {}", max_committed_epoch)) - }); + data.abort(|| HummockError::other("uploader is reset")); } self.context.stats.uploader_syncing_epoch_count.set(0); @@ -1287,7 +1292,6 @@ impl UploaderData { &mut self, cx: &mut Context<'_>, context: &UploaderContext, - mut set_max_synced_epoch: impl FnMut(u64), ) -> Poll, ErrState>>> { while let Some(syncing_data) = self.syncing_data.back_mut() { let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { @@ -1330,6 +1334,8 @@ impl UploaderData { let syncing_data = self.syncing_data.pop_back().expect("non-empty"); let SyncingData { sync_epoch, + table_ids: _table_ids, + table_ids_to_ack, uploading_tasks, uploaded, table_watermarks, @@ -1340,7 +1346,14 @@ impl UploaderData { .stats .uploader_syncing_epoch_count .set(self.syncing_data.len() as _); - set_max_synced_epoch(sync_epoch); + for table_id in table_ids_to_ack { + if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { + table_data.ack_synced(sync_epoch); + if table_data.is_empty() { + self.unsync_data.table_data.remove(&table_id); + } + } + } send_sync_result( sync_result_sender, Ok(SyncedData { @@ -1382,17 +1395,7 @@ impl HummockUploader { return Poll::Pending; }; - if let Some(result) = - ready!( - data.poll_syncing_task(cx, &self.context, |new_synced_epoch| { - Self::set_max_synced_epoch( - &mut self.max_synced_epoch, - self.max_syncing_epoch, - new_synced_epoch, - ) - }) - ) - { + if let Some(result) = ready!(data.poll_syncing_task(cx, &self.context)) { match result { Ok(data) => { return Poll::Ready(data); @@ -1442,10 +1445,11 @@ pub(crate) mod tests { use risingwave_common::catalog::TableId; use risingwave_common::must_match; use risingwave_common::util::epoch::{test_epoch, EpochExt}; + use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; - use risingwave_pb::hummock::{KeyRange, SstableInfo}; + use risingwave_pb::hummock::{KeyRange, SstableInfo, StateTableInfoDelta}; use spin::Mutex; use tokio::spawn; use tokio::sync::mpsc::unbounded_channel; @@ -1455,8 +1459,9 @@ pub(crate) mod tests { use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::uploader::uploader_imm::UploaderImm; use crate::hummock::event_handler::uploader::{ - get_payload_imm_ids, HummockUploader, SyncedData, UploadTaskInfo, UploadTaskOutput, - UploadTaskPayload, UploaderContext, UploaderData, UploaderState, UploadingTask, + get_payload_imm_ids, HummockUploader, SyncedData, TableUnsyncData, UploadTaskInfo, + UploadTaskOutput, UploadTaskPayload, UploaderContext, UploaderData, UploaderState, + UploadingTask, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -1481,12 +1486,39 @@ pub(crate) mod tests { fn data(&self) -> &UploaderData { must_match!(&self.state, UploaderState::Working(data) => data) } + + fn table_data(&self) -> &TableUnsyncData { + self.data() + .unsync_data + .table_data + .get(&TEST_TABLE_ID) + .expect("should exist") + } + + fn test_max_syncing_epoch(&self) -> HummockEpoch { + self.table_data().max_sync_epoch().unwrap() + } + + fn test_max_synced_epoch(&self) -> HummockEpoch { + self.table_data().max_synced_epoch.unwrap() + } } fn test_hummock_version(epoch: HummockEpoch) -> HummockVersion { let mut version = HummockVersion::default(); version.id = epoch; version.max_committed_epoch = epoch; + version.state_table_info.apply_delta( + &HashMap::from_iter([( + TEST_TABLE_ID, + StateTableInfoDelta { + committed_epoch: epoch, + safe_epoch: epoch, + compaction_group_id: StaticCompactionGroupId::StateDefault as _, + }, + )]), + &HashSet::new(), + ); version } @@ -1713,7 +1745,7 @@ pub(crate) mod tests { let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1 as HummockEpoch, uploader.max_syncing_epoch); + assert_eq!(epoch1 as HummockEpoch, uploader.test_max_syncing_epoch()); assert_eq!(1, uploader.data().syncing_data.len()); let syncing_data = uploader.data().syncing_data.front().unwrap(); assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); @@ -1752,7 +1784,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context @@ -1768,8 +1800,10 @@ pub(crate) mod tests { let epoch1 = INITIAL_EPOCH.next_epoch(); let (sync_tx, sync_rx) = oneshot::channel(); + uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch1); + uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch1); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1, uploader.max_syncing_epoch); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); assert_uploader_pending(&mut uploader).await; @@ -1779,7 +1813,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context .pinned_version @@ -1802,7 +1836,7 @@ pub(crate) mod tests { let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch1, uploader.max_syncing_epoch); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); assert_uploader_pending(&mut uploader).await; @@ -1812,7 +1846,7 @@ pub(crate) mod tests { } _ => unreachable!(), }; - assert_eq!(epoch1, uploader.max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); let new_pinned_version = uploader .context .pinned_version @@ -1826,11 +1860,9 @@ pub(crate) mod tests { async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); - assert!( - poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context, |_| unreachable!())) - .await - .is_none() - ); + assert!(poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context)) + .await + .is_none()); assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); } @@ -1849,28 +1881,30 @@ pub(crate) mod tests { let version3 = initial_pinned_version.new_pin_version(test_hummock_version(epoch3)); let version4 = initial_pinned_version.new_pin_version(test_hummock_version(epoch4)); let version5 = initial_pinned_version.new_pin_version(test_hummock_version(epoch5)); - uploader.update_pinned_version(version1); - assert_eq!(epoch1, uploader.max_synced_epoch); - assert_eq!(epoch1, uploader.max_syncing_epoch); uploader.init_instance(TEST_LOCAL_INSTANCE_ID, TEST_TABLE_ID, epoch6); + + uploader.update_pinned_version(version1); + assert_eq!(epoch1, uploader.test_max_synced_epoch()); + assert_eq!(epoch1, uploader.test_max_syncing_epoch()); + let imm = gen_imm(epoch6).await; uploader.add_imm(TEST_LOCAL_INSTANCE_ID, imm.clone()); uploader.update_pinned_version(version2); - assert_eq!(epoch2, uploader.max_synced_epoch); - assert_eq!(epoch2, uploader.max_syncing_epoch); + assert_eq!(epoch2, uploader.test_max_synced_epoch()); + assert_eq!(epoch2, uploader.test_max_syncing_epoch()); uploader.local_seal_epoch_for_test(TEST_LOCAL_INSTANCE_ID, epoch6); uploader.update_pinned_version(version3); - assert_eq!(epoch3, uploader.max_synced_epoch); - assert_eq!(epoch3, uploader.max_syncing_epoch); + assert_eq!(epoch3, uploader.test_max_synced_epoch()); + assert_eq!(epoch3, uploader.test_max_syncing_epoch()); let (sync_tx, sync_rx) = oneshot::channel(); uploader.start_sync_epoch(epoch6, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); uploader.update_pinned_version(version4); - assert_eq!(epoch4, uploader.max_synced_epoch); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch4, uploader.test_max_synced_epoch()); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); let sst = uploader.next_uploaded_sst().await; assert_eq!(&get_imm_ids([&imm]), sst.imm_ids()); @@ -1885,8 +1919,8 @@ pub(crate) mod tests { } uploader.update_pinned_version(version5); - assert_eq!(epoch6, uploader.max_synced_epoch); - assert_eq!(epoch6, uploader.max_syncing_epoch); + assert_eq!(epoch6, uploader.test_max_synced_epoch()); + assert_eq!(epoch6, uploader.test_max_syncing_epoch()); } fn prepare_uploader_order_test( @@ -2134,7 +2168,7 @@ pub(crate) mod tests { } else { unreachable!("should be sync finish"); } - assert_eq!(epoch2, uploader.max_synced_epoch); + assert_eq!(epoch2, uploader.test_max_synced_epoch()); // current uploader state: // unsealed: epoch4: imm: imm4 @@ -2188,7 +2222,7 @@ pub(crate) mod tests { } else { unreachable!("should be sync finish"); } - assert_eq!(epoch4, uploader.max_synced_epoch); + assert_eq!(epoch4, uploader.test_max_synced_epoch()); // current uploader state: // unsealed: empty From b9ceb8d208c2d906d45af2a5926456d41629e67f Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 9 Jul 2024 17:20:06 +0800 Subject: [PATCH 04/20] feat(storage): decouple upload task finish order from epoch (#17460) --- .../{uploader.rs => uploader/mod.rs} | 633 +++++++++--------- .../event_handler/uploader/task_manager.rs | 187 ++++++ .../src/hummock/store/hummock_storage.rs | 14 +- 3 files changed, 494 insertions(+), 340 deletions(-) rename src/storage/src/hummock/event_handler/{uploader.rs => uploader/mod.rs} (83%) create mode 100644 src/storage/src/hummock/event_handler/uploader/task_manager.rs diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs similarity index 83% rename from src/storage/src/hummock/event_handler/uploader.rs rename to src/storage/src/hummock/event_handler/uploader/mod.rs index 55761f1216183..06f0c3aff77a4 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod task_manager; + use std::cmp::Ordering; use std::collections::btree_map::Entry; -use std::collections::{BTreeMap, HashMap, HashSet, VecDeque}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; use std::fmt::{Debug, Display, Formatter}; use std::future::{poll_fn, Future}; use std::mem::{replace, swap, take}; -use std::pin::Pin; use std::sync::Arc; use std::task::{ready, Context, Poll}; @@ -35,6 +36,7 @@ use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, VnodeWatermark, WatermarkDirection, }; use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, LocalSstableInfo}; +use task_manager::{TaskManager, UploadingTaskStatus}; use thiserror_ext::AsReport; use tokio::sync::oneshot; use tokio::task::JoinHandle; @@ -161,9 +163,13 @@ mod uploader_imm { } } +#[derive(PartialEq, Eq, Hash, PartialOrd, Ord, Copy, Clone, Debug)] +struct UploadingTaskId(usize); + /// A wrapper for a uploading task that compacts and uploads the imm payload. Task context are /// stored so that when the task fails, it can be re-tried. struct UploadingTask { + task_id: UploadingTaskId, // newer data at the front input: UploadTaskInput, join_handle: JoinHandle>, @@ -219,7 +225,7 @@ impl UploadingTask { .collect() } - fn new(input: UploadTaskInput, context: &UploaderContext) -> Self { + fn new(task_id: UploadingTaskId, input: UploadTaskInput, context: &UploaderContext) -> Self { assert!(!input.is_empty()); let mut epochs = input .iter() @@ -254,6 +260,7 @@ impl UploadingTask { let join_handle = (context.spawn_upload_task)(payload, task_info.clone()); context.stats.uploader_uploading_task_count.inc(); Self { + task_id, input, join_handle, task_info, @@ -324,81 +331,6 @@ impl UploadingTask { } } -impl Future for UploadingTask { - type Output = HummockResult>; - - fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll { - self.poll_result(cx) - } -} - -#[derive(Default, Debug)] -/// Manage the spilled data. Task and uploaded data at the front is newer data. Task data are -/// always newer than uploaded data. Task holding oldest data is always collected first. -struct SpilledData { - // ordered spilling tasks. Task at the back is spilling older data. - uploading_tasks: VecDeque, - // ordered spilled data. Data at the back is older. - uploaded_data: VecDeque>, -} - -impl SpilledData { - fn add_task(&mut self, task: UploadingTask) { - self.uploading_tasks.push_front(task); - } - - /// Poll the successful spill of the oldest uploading task. Return `Poll::Ready(None)` is there - /// is no uploading task - fn poll_success_spill( - &mut self, - cx: &mut Context<'_>, - ) -> Poll>> { - // only poll the oldest uploading task if there is any - if let Some(task) = self.uploading_tasks.back_mut() { - let staging_sstable_info = ready!(task.poll_ok_with_retry(cx)); - self.uploaded_data.push_front(staging_sstable_info.clone()); - self.uploading_tasks.pop_back(); - Poll::Ready(Some(staging_sstable_info)) - } else { - Poll::Ready(None) - } - } - - fn abort(self) { - for task in self.uploading_tasks { - task.join_handle.abort(); - } - } -} - -#[derive(Default, Debug)] -struct EpochData { - spilled_data: SpilledData, -} - -impl EpochData { - fn flush( - &mut self, - context: &UploaderContext, - imms: HashMap>, - ) -> usize { - if !imms.is_empty() { - let task = UploadingTask::new(imms, context); - context.stats.spill_task_counts_from_unsealed.inc(); - context - .stats - .spill_task_size_from_unsealed - .inc_by(task.task_info.task_size as u64); - info!("Spill unsealed data. Task: {}", task.get_task_info()); - let size = task.task_info.task_size; - self.spilled_data.add_task(task); - size - } else { - 0 - } - } -} - impl TableUnsyncData { fn add_table_watermarks( &mut self, @@ -453,72 +385,9 @@ impl TableUnsyncData { } } -#[derive(Default)] -struct SyncDataBuilder { - // newer epochs come first - epochs: VecDeque, - - spilled_data: SpilledData, - - table_watermarks: HashMap, - - table_ids_to_ack: HashSet, -} - -impl SyncDataBuilder { - /// Add the data of a new epoch. - /// - /// Note: it may happen that, for example, currently we hold `imms` and `spilled_data` of epoch - /// 3, and after we add the spilled data of epoch 4, both `imms` and `spilled_data` hold data - /// of both epoch 3 and 4, which seems breaking the rules that data in `imms` are - /// always newer than data in `spilled_data`, because epoch 3 data of `imms` - /// seems older than epoch 4 data of `spilled_data`. However, if this happens, the epoch 3 - /// data of `imms` must not overlap with the epoch 4 data of `spilled_data`. The explanation is - /// as followed: - /// - /// First, unsync data has 3 stages, from earlier to later, imms, uploading task, and - /// uploaded. When we try to spill unsync data, we first pick the imms of older epoch until - /// the imms of older epoch are all picked. When we try to poll the uploading tasks of unsync - /// data, we first poll the task of older epoch, until there is no uploading task in older - /// epoch. Therefore, we can reach that, if two data are in the same stage, but - /// different epochs, data in the older epoch will always enter the next stage earlier than data - /// in the newer epoch. - /// - /// Second, we have an assumption that, if a key has been written in a newer epoch, e.g. epoch4, - /// it will no longer be written in an older epoch, e.g. epoch3, and then, if two data of the - /// same key are at the imm stage, the data of older epoch must appear earlier than the data - /// of newer epoch. - /// - /// Based on the two points above, we can reach that, if two data of a same key appear in - /// different epochs, the data of older epoch will not appear at a later stage than the data - /// of newer epoch. Therefore, we can safely merge the data of each stage when we seal an epoch. - fn add_new_epoch(&mut self, epoch: HummockEpoch, mut unseal_epoch_data: EpochData) { - if let Some(prev_max_epoch) = self.epochs.front() { - assert!( - epoch > *prev_max_epoch, - "epoch {} to seal not greater than prev max epoch {}", - epoch, - prev_max_epoch - ); - } - - self.epochs.push_front(epoch); - // for each local instance, earlier data must be spilled at earlier epoch. Therefore, since we add spill data from old epoch - // to new epoch, - unseal_epoch_data - .spilled_data - .uploading_tasks - .append(&mut self.spilled_data.uploading_tasks); - unseal_epoch_data - .spilled_data - .uploaded_data - .append(&mut self.spilled_data.uploaded_data); - self.spilled_data.uploading_tasks = unseal_epoch_data.spilled_data.uploading_tasks; - self.spilled_data.uploaded_data = unseal_epoch_data.spilled_data.uploaded_data; - } - +impl UploaderData { fn add_table_watermarks( - &mut self, + all_table_watermarks: &mut HashMap, table_id: TableId, direction: WatermarkDirection, watermarks: impl Iterator)>, @@ -540,19 +409,11 @@ impl SyncDataBuilder { } } if let Some(table_watermarks) = table_watermarks { - assert!(self - .table_watermarks + assert!(all_table_watermarks .insert(table_id, table_watermarks) .is_none()); } } - - fn flush(&mut self, context: &UploaderContext, payload: UploadTaskInput) { - if !payload.is_empty() { - let task = UploadingTask::new(payload, context); - self.spilled_data.add_task(task); - } - } } struct LocalInstanceEpochData { @@ -753,6 +614,7 @@ struct TableUnsyncData { WatermarkDirection, BTreeMap, BitmapBuilder)>, )>, + spill_tasks: BTreeMap>, // newer epoch at the front syncing_epochs: VecDeque, max_synced_epoch: Option, @@ -764,6 +626,7 @@ impl TableUnsyncData { table_id, instance_data: Default::default(), table_watermarks: None, + spill_tasks: Default::default(), syncing_epochs: Default::default(), max_synced_epoch: committed_epoch, } @@ -778,6 +641,7 @@ impl TableUnsyncData { WatermarkDirection, impl Iterator)>, )>, + impl Iterator, ) { if let Some(prev_epoch) = self.max_sync_epoch() { assert_gt!(epoch, prev_epoch) @@ -798,6 +662,9 @@ impl TableUnsyncData { .map(|(epoch, (watermarks, _))| (epoch, watermarks)), ) }), + take_before_epoch(&mut self.spill_tasks, epoch) + .into_values() + .flat_map(|tasks| tasks.into_iter()), ) } @@ -858,7 +725,9 @@ struct UnsyncData { table_data: HashMap, // An index as a mapping from instance id to its table id instance_table_id: HashMap, - epoch_data: BTreeMap, + // TODO: this is only used in spill to get existing epochs and can be removed + // when we support spill not based on epoch + epochs: BTreeMap, } impl UnsyncData { @@ -899,7 +768,7 @@ impl UnsyncData { .instance_table_id .insert(instance_id, table_id) .is_none()); - self.epoch_data.entry(init_epoch).or_default(); + self.epochs.insert(init_epoch, ()); } fn instance_data( @@ -938,59 +807,123 @@ impl UnsyncData { .get_mut(&instance_id) .expect("should exist"); let epoch = instance_data.local_seal_epoch(next_epoch); - self.epoch_data.entry(next_epoch).or_default(); + self.epochs.insert(next_epoch, ()); if let Some((direction, table_watermarks)) = opts.table_watermarks { table_data.add_table_watermarks(epoch, table_watermarks, direction); } } - fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) { + fn may_destroy_instance(&mut self, instance_id: LocalInstanceId) -> Option { if let Some(table_id) = self.instance_table_id.remove(&instance_id) { debug!(instance_id, "destroy instance"); let table_data = self.table_data.get_mut(&table_id).expect("should exist"); assert!(table_data.instance_data.remove(&instance_id).is_some()); if table_data.is_empty() { - self.table_data.remove(&table_id); + Some(self.table_data.remove(&table_id).expect("should exist")) + } else { + None } + } else { + None } } +} +impl UploaderData { fn sync( &mut self, epoch: HummockEpoch, context: &UploaderContext, - table_ids: &HashSet, - ) -> SyncDataBuilder { - let sync_epoch_data = take_before_epoch(&mut self.epoch_data, epoch); + table_ids: HashSet, + sync_result_sender: oneshot::Sender>, + ) { + // clean old epochs + let _epochs = take_before_epoch(&mut self.unsync_data.epochs, epoch); - let mut sync_data = SyncDataBuilder::default(); - for (epoch, epoch_data) in sync_epoch_data { - sync_data.add_new_epoch(epoch, epoch_data); - } + let mut all_table_watermarks = HashMap::new(); + let mut uploading_tasks = HashSet::new(); + let mut spilled_tasks = BTreeSet::new(); let mut flush_payload = HashMap::new(); let mut table_ids_to_ack = HashSet::new(); - for (table_id, table_data) in &mut self.table_data { + for (table_id, table_data) in &mut self.unsync_data.table_data { if !table_ids.contains(table_id) { table_data.assert_after_epoch(epoch); continue; } table_ids_to_ack.insert(*table_id); - let (unflushed_payload, table_watermarks) = table_data.sync(epoch); + let (unflushed_payload, table_watermarks, task_ids) = table_data.sync(epoch); for (instance_id, payload) in unflushed_payload { if !payload.is_empty() { flush_payload.insert(instance_id, payload); } } if let Some((direction, watermarks)) = table_watermarks { - sync_data.add_table_watermarks(*table_id, direction, watermarks); + Self::add_table_watermarks( + &mut all_table_watermarks, + *table_id, + direction, + watermarks, + ); } + for task_id in task_ids { + if self.spilled_data.contains_key(&task_id) { + spilled_tasks.insert(task_id); + } else { + uploading_tasks.insert(task_id); + } + } + } + + let sync_id = { + let sync_id = self.next_sync_id; + self.next_sync_id += 1; + SyncId(sync_id) + }; + + if let Some(extra_flush_task_id) = self.task_manager.sync( + context, + sync_id, + flush_payload, + uploading_tasks.iter().cloned(), + &table_ids, + ) { + uploading_tasks.insert(extra_flush_task_id); } - sync_data.flush(context, flush_payload); - sync_data.table_ids_to_ack = table_ids_to_ack; - sync_data + + // iter from large task_id to small one so that newer data at the front + let uploaded = spilled_tasks + .iter() + .rev() + .map(|task_id| { + let (sst, spill_table_ids) = + self.spilled_data.remove(task_id).expect("should exist"); + assert!( + spill_table_ids.is_subset(&table_ids), + "spill_table_ids: {spill_table_ids:?}, table_ids: {table_ids:?}" + ); + sst + }) + .collect(); + + self.syncing_data.insert( + sync_id, + SyncingData { + sync_epoch: epoch, + table_ids, + table_ids_to_ack, + remaining_uploading_tasks: uploading_tasks, + uploaded, + table_watermarks: all_table_watermarks, + sync_result_sender, + }, + ); + + self.check_upload_task_consistency(); } +} +impl UnsyncData { fn ack_flushed(&mut self, sstable_info: &StagingSstableInfo) { for (instance_id, imm_ids) in sstable_info.imm_ids() { if let Some(instance_data) = self.instance_data(*instance_id) { @@ -1006,8 +939,7 @@ struct SyncingData { table_ids: HashSet, /// Subset of `table_ids` that has existing instance table_ids_to_ack: HashSet, - // task of newer data at the front - uploading_tasks: VecDeque, + remaining_uploading_tasks: HashSet, // newer data at the front uploaded: VecDeque>, table_watermarks: HashMap, @@ -1046,25 +978,24 @@ impl UploaderContext { } } +#[derive(PartialEq, Eq, Hash, PartialOrd, Ord, Copy, Clone, Debug)] +struct SyncId(usize); + #[derive(Default)] struct UploaderData { unsync_data: UnsyncData, - /// Data that has started syncing but not synced yet. `epoch` satisfies - /// `max_synced_epoch < epoch <= max_syncing_epoch`. - /// Newer epoch at the front - syncing_data: VecDeque, + syncing_data: BTreeMap, + + task_manager: TaskManager, + spilled_data: HashMap, HashSet)>, + next_sync_id: usize, } impl UploaderData { fn abort(self, err: impl Fn() -> HummockError) { - for (_, epoch_data) in self.unsync_data.epoch_data { - epoch_data.spilled_data.abort(); - } - for syncing_data in self.syncing_data { - for task in syncing_data.uploading_tasks { - task.join_handle.abort(); - } + self.task_manager.abort(); + for syncing_data in self.syncing_data.into_values() { send_sync_result(syncing_data.sync_result_sender, Err(err())); } } @@ -1186,28 +1117,9 @@ impl HummockUploader { }; debug!(epoch, ?table_ids, "start sync epoch"); - let sync_data = data.unsync_data.sync(epoch, &self.context, &table_ids); + data.sync(epoch, &self.context, table_ids, sync_result_sender); - let SyncDataBuilder { - spilled_data: - SpilledData { - uploading_tasks, - uploaded_data, - }, - table_watermarks, - table_ids_to_ack, - epochs: _, - } = sync_data; - - data.syncing_data.push_front(SyncingData { - sync_epoch: epoch, - table_ids, - table_ids_to_ack, - uploading_tasks, - uploaded: uploaded_data, - table_watermarks, - sync_result_sender, - }); + data.may_notify_sync_task(&self.context); self.context .stats @@ -1238,7 +1150,7 @@ impl HummockUploader { if self.context.buffer_tracker.need_flush() { let mut curr_batch_flush_size = 0; // iterate from older epoch to newer epoch - for (epoch, epoch_data) in &mut data.unsync_data.epoch_data { + for epoch in &mut data.unsync_data.epochs.keys() { if !self .context .buffer_tracker @@ -1246,20 +1158,35 @@ impl HummockUploader { { break; } + let mut spilled_table_ids = HashSet::new(); let mut payload = HashMap::new(); - for (instance_id, instance_data) in data - .unsync_data - .table_data - .values_mut() - .flat_map(|data| data.instance_data.iter_mut()) - { - let instance_payload = instance_data.spill(*epoch); - if !instance_payload.is_empty() { - payload.insert(*instance_id, instance_payload); + for (table_id, table_data) in &mut data.unsync_data.table_data { + for (instance_id, instance_data) in &mut table_data.instance_data { + let instance_payload = instance_data.spill(*epoch); + if !instance_payload.is_empty() { + payload.insert(*instance_id, instance_payload); + spilled_table_ids.insert(*table_id); + } } } - curr_batch_flush_size += epoch_data.flush(&self.context, payload); + if !payload.is_empty() { + let (task_id, task_size, spilled_table_ids) = + data.task_manager + .spill(&self.context, spilled_table_ids, payload); + for table_id in spilled_table_ids { + data.unsync_data + .table_data + .get_mut(table_id) + .expect("should exist") + .spill_tasks + .entry(*epoch) + .or_default() + .push_front(task_id); + } + curr_batch_flush_size += task_size; + } } + data.check_upload_task_consistency(); curr_batch_flush_size > 0 } else { false @@ -1281,108 +1208,123 @@ impl HummockUploader { let UploaderState::Working(data) = &mut self.state else { return; }; - data.unsync_data.may_destroy_instance(instance_id); + if let Some(removed_table_data) = data.unsync_data.may_destroy_instance(instance_id) { + data.task_manager.remove_table_spill_tasks( + removed_table_data.table_id, + removed_table_data + .spill_tasks + .into_values() + .flat_map(|task_ids| task_ids.into_iter()) + .filter(|task_id| { + if let Some((_, table_ids)) = data.spilled_data.get_mut(task_id) { + assert!(table_ids.remove(&removed_table_data.table_id)); + if table_ids.is_empty() { + data.spilled_data.remove(task_id); + } + false + } else { + true + } + }), + ) + } + data.check_upload_task_consistency(); } } impl UploaderData { - /// Poll the syncing task of the syncing data of the oldest epoch. Return `Poll::Ready(None)` if - /// there is no syncing data. - fn poll_syncing_task( - &mut self, - cx: &mut Context<'_>, - context: &UploaderContext, - ) -> Poll, ErrState>>> { - while let Some(syncing_data) = self.syncing_data.back_mut() { - let sstable_info = if let Some(task) = syncing_data.uploading_tasks.back_mut() { - let result = ready!(task.poll_result(cx)); - let _task = syncing_data.uploading_tasks.pop_back().expect("non-empty"); - let sstable_info = match result { - Ok(sstable_info) => sstable_info, - Err(e) => { - let SyncingData { - sync_epoch, - uploading_tasks, - sync_result_sender, - .. - } = self.syncing_data.pop_back().expect("non-empty"); - for task in uploading_tasks { - task.join_handle.abort(); - } - send_sync_result( - sync_result_sender, - Err(HummockError::other(format!( - "failed sync task: {:?}", - e.as_report() - ))), - ); - - return Poll::Ready(Some(Err(ErrState { - failed_epoch: sync_epoch, - reason: format!("{:?}", e.as_report()), - }))); + fn may_notify_sync_task(&mut self, context: &UploaderContext) { + while let Some((_, syncing_data)) = self.syncing_data.first_key_value() + && syncing_data.remaining_uploading_tasks.is_empty() + { + let (_, syncing_data) = self.syncing_data.pop_first().expect("non-empty"); + let SyncingData { + sync_epoch, + table_ids: _table_ids, + table_ids_to_ack, + remaining_uploading_tasks: _, + uploaded, + table_watermarks, + sync_result_sender, + } = syncing_data; + context + .stats + .uploader_syncing_epoch_count + .set(self.syncing_data.len() as _); + + for table_id in table_ids_to_ack { + if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { + table_data.ack_synced(sync_epoch); + if table_data.is_empty() { + self.unsync_data.table_data.remove(&table_id); } - }; - syncing_data.uploaded.push_front(sstable_info.clone()); - self.unsync_data.ack_flushed(&sstable_info); - Some(sstable_info) - } else { - None - }; + } + } - if syncing_data.uploading_tasks.is_empty() { - let syncing_data = self.syncing_data.pop_back().expect("non-empty"); - let SyncingData { - sync_epoch, - table_ids: _table_ids, - table_ids_to_ack, - uploading_tasks, - uploaded, + send_sync_result( + sync_result_sender, + Ok(SyncedData { + uploaded_ssts: uploaded, table_watermarks, - sync_result_sender, - } = syncing_data; - assert!(uploading_tasks.is_empty()); - context - .stats - .uploader_syncing_epoch_count - .set(self.syncing_data.len() as _); - for table_id in table_ids_to_ack { - if let Some(table_data) = self.unsync_data.table_data.get_mut(&table_id) { - table_data.ack_synced(sync_epoch); - if table_data.is_empty() { - self.unsync_data.table_data.remove(&table_id); - } - } + }), + ) + } + } + + fn check_upload_task_consistency(&self) { + #[cfg(debug_assertions)] + { + let mut spill_task_table_id_from_data: HashMap<_, HashSet<_>> = HashMap::new(); + for table_data in self.unsync_data.table_data.values() { + for task_id in table_data + .spill_tasks + .iter() + .flat_map(|(_, tasks)| tasks.iter()) + { + assert!(spill_task_table_id_from_data + .entry(*task_id) + .or_default() + .insert(table_data.table_id)); } - send_sync_result( - sync_result_sender, - Ok(SyncedData { - uploaded_ssts: uploaded, - table_watermarks, - }), - ) } + let syncing_task_id_from_data: HashMap<_, HashSet<_>> = self + .syncing_data + .iter() + .filter_map(|(sync_id, data)| { + if data.remaining_uploading_tasks.is_empty() { + None + } else { + Some((*sync_id, data.remaining_uploading_tasks.clone())) + } + }) + .collect(); - if let Some(sstable_info) = sstable_info { - return Poll::Ready(Some(Ok(sstable_info))); + let mut spill_task_table_id_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); + for (task_id, (_, table_ids)) in &self.spilled_data { + spill_task_table_id_from_manager.insert(*task_id, table_ids.clone()); } - } - Poll::Ready(None) - } - - /// Poll the success of the oldest spilled task of unsync spill data. Return `Poll::Ready(None)` if - /// there is no spilling task. - fn poll_spill_task(&mut self, cx: &mut Context<'_>) -> Poll>> { - // iterator from older epoch to new epoch so that the spill task are finished in epoch order - for epoch_data in self.unsync_data.epoch_data.values_mut() { - // if None, there is no spilling task. Search for the unsync data of the next epoch in - // the next iteration. - if let Some(sstable_info) = ready!(epoch_data.spilled_data.poll_success_spill(cx)) { - self.unsync_data.ack_flushed(&sstable_info); - return Poll::Ready(Some(sstable_info)); + let mut syncing_task_from_manager: HashMap<_, HashSet<_>> = HashMap::new(); + for (task_id, status) in self.task_manager.tasks() { + match status { + UploadingTaskStatus::Spilling(table_ids) => { + assert!(spill_task_table_id_from_manager + .insert(task_id, table_ids.clone()) + .is_none()); + } + UploadingTaskStatus::Sync(sync_id) => { + assert!(syncing_task_from_manager + .entry(*sync_id) + .or_default() + .insert(task_id)); + } + } } + assert_eq!( + spill_task_table_id_from_data, + spill_task_table_id_from_manager + ); + assert_eq!(syncing_task_id_from_data, syncing_task_from_manager); } - Poll::Ready(None) } } @@ -1395,34 +1337,57 @@ impl HummockUploader { return Poll::Pending; }; - if let Some(result) = ready!(data.poll_syncing_task(cx, &self.context)) { + if let Some((task_id, status, result)) = ready!(data.task_manager.poll_task_result(cx)) + { match result { - Ok(data) => { - return Poll::Ready(data); + Ok(sst) => { + data.unsync_data.ack_flushed(&sst); + match status { + UploadingTaskStatus::Sync(sync_id) => { + let syncing_data = + data.syncing_data.get_mut(&sync_id).expect("should exist"); + syncing_data.uploaded.push_front(sst.clone()); + assert!(syncing_data.remaining_uploading_tasks.remove(&task_id)); + data.may_notify_sync_task(&self.context); + } + UploadingTaskStatus::Spilling(table_ids) => { + data.spilled_data.insert(task_id, (sst.clone(), table_ids)); + } + } + data.check_upload_task_consistency(); + Poll::Ready(sst) } - Err(e) => { - let failed_epoch = e.failed_epoch; + Err((sync_id, e)) => { + let syncing_data = + data.syncing_data.remove(&sync_id).expect("should exist"); + let failed_epoch = syncing_data.sync_epoch; let data = must_match!(replace( &mut self.state, - UploaderState::Err(e), + UploaderState::Err(ErrState { + failed_epoch, + reason: e.as_report().to_string(), + }), ), UploaderState::Working(data) => data); + let _ = syncing_data + .sync_result_sender + .send(Err(HummockError::other(format!( + "failed to sync: {:?}", + e.as_report() + )))); + data.abort(|| { HummockError::other(format!( "previous epoch {} failed to sync", failed_epoch )) }); - return Poll::Pending; + Poll::Pending } } + } else { + Poll::Pending } - - if let Some(sstable_info) = ready!(data.poll_spill_task(cx)) { - return Poll::Ready(sstable_info); - } - - Poll::Pending }) } } @@ -1432,9 +1397,10 @@ pub(crate) mod tests { use std::collections::{HashMap, HashSet, VecDeque}; use std::future::{poll_fn, Future}; use std::ops::Deref; + use std::pin::pin; use std::sync::atomic::AtomicUsize; - use std::sync::atomic::Ordering::SeqCst; - use std::sync::Arc; + use std::sync::atomic::Ordering::{Relaxed, SeqCst}; + use std::sync::{Arc, LazyLock}; use std::task::Poll; use bytes::Bytes; @@ -1461,7 +1427,7 @@ pub(crate) mod tests { use crate::hummock::event_handler::uploader::{ get_payload_imm_ids, HummockUploader, SyncedData, TableUnsyncData, UploadTaskInfo, UploadTaskOutput, UploadTaskPayload, UploaderContext, UploaderData, UploaderState, - UploadingTask, + UploadingTask, UploadingTaskId, }; use crate::hummock::event_handler::{LocalInstanceId, TEST_LOCAL_INSTANCE_ID}; use crate::hummock::local_version::pinned_version::PinnedVersion; @@ -1640,7 +1606,12 @@ pub(crate) mod tests { TEST_LOCAL_INSTANCE_ID, imms.into_iter().map(UploaderImm::for_test).collect_vec(), )]); - Self::new(input, context) + static NEXT_TASK_ID: LazyLock = LazyLock::new(|| AtomicUsize::new(0)); + Self::new( + UploadingTaskId(NEXT_TASK_ID.fetch_add(1, Relaxed)), + input, + context, + ) } } @@ -1670,11 +1641,11 @@ pub(crate) mod tests { let imm = gen_imm(INITIAL_EPOCH).await; let imm_size = imm.size(); let imm_ids = get_imm_ids(vec![&imm]); - let task = UploadingTask::from_vec(vec![imm], &uploader_context); + let mut task = UploadingTask::from_vec(vec![imm], &uploader_context); assert_eq!(imm_size, task.task_info.task_size); assert_eq!(imm_ids, task.task_info.imm_ids); assert_eq!(vec![INITIAL_EPOCH], task.task_info.epochs); - let output = task.await.unwrap(); + let output = poll_fn(|cx| task.poll_result(cx)).await.unwrap(); assert_eq!( output.sstable_infos(), &dummy_success_upload_output().new_value_ssts @@ -1685,8 +1656,8 @@ pub(crate) mod tests { let uploader_context = test_uploader_context(dummy_fail_upload_future); let imm = gen_imm(INITIAL_EPOCH).await; - let task = UploadingTask::from_vec(vec![imm], &uploader_context); - let _ = task.await.unwrap_err(); + let mut task = UploadingTask::from_vec(vec![imm], &uploader_context); + let _ = poll_fn(|cx| task.poll_result(cx)).await.unwrap_err(); } #[tokio::test] @@ -1747,10 +1718,10 @@ pub(crate) mod tests { uploader.start_sync_epoch(epoch1, sync_tx, HashSet::from_iter([TEST_TABLE_ID])); assert_eq!(epoch1 as HummockEpoch, uploader.test_max_syncing_epoch()); assert_eq!(1, uploader.data().syncing_data.len()); - let syncing_data = uploader.data().syncing_data.front().unwrap(); + let (_, syncing_data) = uploader.data().syncing_data.first_key_value().unwrap(); assert_eq!(epoch1 as HummockEpoch, syncing_data.sync_epoch); assert!(syncing_data.uploaded.is_empty()); - assert!(!syncing_data.uploading_tasks.is_empty()); + assert!(!syncing_data.remaining_uploading_tasks.is_empty()); let staging_sst = uploader.next_uploaded_sst().await; assert_eq!(&vec![epoch1], staging_sst.epochs()); @@ -1859,11 +1830,9 @@ pub(crate) mod tests { #[tokio::test] async fn test_uploader_poll_empty() { let mut uploader = test_uploader(dummy_success_upload_future); - let data = must_match!(&mut uploader.state, UploaderState::Working(data) => data); - assert!(poll_fn(|cx| data.poll_syncing_task(cx, &uploader.context)) - .await - .is_none()); - assert!(poll_fn(|cx| data.poll_spill_task(cx)).await.is_none()); + let fut = uploader.next_uploaded_sst(); + let mut fut = pin!(fut); + assert!(poll_fn(|cx| Poll::Ready(fut.as_mut().poll(cx).is_pending())).await); } #[tokio::test] diff --git a/src/storage/src/hummock/event_handler/uploader/task_manager.rs b/src/storage/src/hummock/event_handler/uploader/task_manager.rs new file mode 100644 index 0000000000000..2347be1ed57eb --- /dev/null +++ b/src/storage/src/hummock/event_handler/uploader/task_manager.rs @@ -0,0 +1,187 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use super::*; + +#[derive(Debug)] +pub(super) enum UploadingTaskStatus { + Spilling(HashSet), + Sync(SyncId), +} + +#[derive(Debug)] +struct TaskEntry { + task: UploadingTask, + status: UploadingTaskStatus, +} + +#[derive(Default, Debug)] +pub(super) struct TaskManager { + tasks: HashMap, + // newer task at the front + task_order: VecDeque, + next_task_id: usize, +} + +impl TaskManager { + fn add_task( + &mut self, + task: UploadingTask, + status: UploadingTaskStatus, + ) -> &UploadingTaskStatus { + let task_id = task.task_id; + self.task_order.push_front(task.task_id); + assert!(self + .tasks + .insert(task.task_id, TaskEntry { task, status }) + .is_none()); + &self.tasks.get(&task_id).expect("should exist").status + } + + fn poll_task( + &mut self, + cx: &mut Context<'_>, + task_id: UploadingTaskId, + ) -> Poll, (SyncId, HummockError)>> { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + let result = match &entry.status { + UploadingTaskStatus::Spilling(_) => { + let sst = ready!(entry.task.poll_ok_with_retry(cx)); + Ok(sst) + } + UploadingTaskStatus::Sync(sync_id) => { + let result = ready!(entry.task.poll_result(cx)); + result.map_err(|e| (*sync_id, e)) + } + }; + Poll::Ready(result) + } + + fn get_next_task_id(&mut self) -> UploadingTaskId { + let task_id = self.next_task_id; + self.next_task_id += 1; + UploadingTaskId(task_id) + } + + #[expect(clippy::type_complexity)] + pub(super) fn poll_task_result( + &mut self, + cx: &mut Context<'_>, + ) -> Poll< + Option<( + UploadingTaskId, + UploadingTaskStatus, + Result, (SyncId, HummockError)>, + )>, + > { + if let Some(task_id) = self.task_order.back() { + let task_id = *task_id; + let result = ready!(self.poll_task(cx, task_id)); + self.task_order.pop_back(); + let entry = self.tasks.remove(&task_id).expect("should exist"); + + Poll::Ready(Some((task_id, entry.status, result))) + } else { + Poll::Ready(None) + } + } + + pub(super) fn abort(self) { + for task in self.tasks.into_values() { + task.task.join_handle.abort(); + } + } + + pub(super) fn spill( + &mut self, + context: &UploaderContext, + table_ids: HashSet, + imms: HashMap>, + ) -> (UploadingTaskId, usize, &HashSet) { + assert!(!imms.is_empty()); + let task = UploadingTask::new(self.get_next_task_id(), imms, context); + context.stats.spill_task_counts_from_unsealed.inc(); + context + .stats + .spill_task_size_from_unsealed + .inc_by(task.task_info.task_size as u64); + info!("Spill data. Task: {}", task.get_task_info()); + let size = task.task_info.task_size; + let id = task.task_id; + let status = self.add_task(task, UploadingTaskStatus::Spilling(table_ids)); + ( + id, + size, + must_match!(status, UploadingTaskStatus::Spilling(table_ids) => table_ids), + ) + } + + pub(super) fn remove_table_spill_tasks( + &mut self, + table_id: TableId, + task_ids: impl Iterator, + ) { + for task_id in task_ids { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + let empty = must_match!(&mut entry.status, UploadingTaskStatus::Spilling(table_ids) => { + assert!(table_ids.remove(&table_id)); + table_ids.is_empty() + }); + if empty { + let task = self.tasks.remove(&task_id).expect("should exist").task; + task.join_handle.abort(); + } + } + } + + pub(super) fn sync( + &mut self, + context: &UploaderContext, + sync_id: SyncId, + unflushed_payload: UploadTaskInput, + spill_task_ids: impl Iterator, + sync_table_ids: &HashSet, + ) -> Option { + let task = if unflushed_payload.is_empty() { + None + } else { + Some(UploadingTask::new( + self.get_next_task_id(), + unflushed_payload, + context, + )) + }; + + for task_id in spill_task_ids { + let entry = self.tasks.get_mut(&task_id).expect("should exist"); + must_match!(&entry.status, UploadingTaskStatus::Spilling(table_ids) => { + assert!(table_ids.is_subset(sync_table_ids), "spill table_ids: {table_ids:?}, sync_table_ids: {sync_table_ids:?}"); + }); + entry.status = UploadingTaskStatus::Sync(sync_id); + } + + task.map(|task| { + let id = task.task_id; + self.add_task(task, UploadingTaskStatus::Sync(sync_id)); + id + }) + } + + #[cfg(debug_assertions)] + pub(super) fn tasks(&self) -> impl Iterator { + self.tasks + .iter() + .map(|(task_id, entry)| (*task_id, &entry.status)) + } +} diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 11aa643c3659a..7b28588c5054a 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -565,16 +565,14 @@ impl StateStore for HummockStorage { fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture { let (tx, rx) = oneshot::channel(); - self.hummock_event_sender - .send(HummockEvent::SyncEpoch { - new_sync_epoch: epoch, - sync_result_sender: tx, - table_ids, - }) - .expect("should send success"); + let _ = self.hummock_event_sender.send(HummockEvent::SyncEpoch { + new_sync_epoch: epoch, + sync_result_sender: tx, + table_ids, + }); rx.map(|recv_result| { Ok(recv_result - .expect("should wait success")? + .map_err(|_| HummockError::other("failed to receive sync result"))?? .into_sync_result()) }) } From b3e1faddf07c1c653f611883c62ae5b6d9d5ca93 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 9 Jul 2024 19:40:00 +0800 Subject: [PATCH 05/20] feat(source): support Avro Union type (#17485) Signed-off-by: xxchan --- e2e_test/commands/sr_register | 27 ++ .../source_inline/kafka/avro/alter_source.slt | 6 +- e2e_test/source_inline/kafka/avro/union.slt | 175 +++++++ src/common/src/types/mod.rs | 2 + src/connector/codec/src/decoder/avro/mod.rs | 446 +++++++++++++++++- .../codec/src/decoder/avro/schema.rs | 141 +++++- src/connector/codec/src/decoder/mod.rs | 23 +- .../codec/tests/integration_tests/avro.rs | 353 +++++++++++++- .../src/parser/debezium/avro_parser.rs | 42 +- src/connector/src/parser/unified/json.rs | 2 +- 10 files changed, 1161 insertions(+), 56 deletions(-) create mode 100755 e2e_test/commands/sr_register create mode 100644 e2e_test/source_inline/kafka/avro/union.slt diff --git a/e2e_test/commands/sr_register b/e2e_test/commands/sr_register new file mode 100755 index 0000000000000..57dc65e50610d --- /dev/null +++ b/e2e_test/commands/sr_register @@ -0,0 +1,27 @@ +#!/usr/bin/env bash + +set -euo pipefail + +# Register a schema to schema registry +# +# Usage: sr_register +# +# https://docs.confluent.io/platform/current/schema-registry/develop/api.html#post--subjects-(string-%20subject)-versions + +# Validate arguments +if [[ $# -ne 2 ]]; then + echo "Usage: sr_register " + exit 1 +fi + +subject="$1" +schema="$2" + + +if [[ -z $subject || -z $schema ]]; then + echo "Error: Arguments cannot be empty" + exit 1 +fi + +echo "$schema" | jq '{"schema": tojson}' \ +| curl -X POST -H 'content-type:application/vnd.schemaregistry.v1+json' -d @- "${RISEDEV_SCHEMA_REGISTRY_URL}/subjects/${subject}/versions" diff --git a/e2e_test/source_inline/kafka/avro/alter_source.slt b/e2e_test/source_inline/kafka/avro/alter_source.slt index 446fc6196d32b..57677af57cd92 100644 --- a/e2e_test/source_inline/kafka/avro/alter_source.slt +++ b/e2e_test/source_inline/kafka/avro/alter_source.slt @@ -12,8 +12,7 @@ system ok rpk topic create 'avro_alter_source_test' system ok -echo '{"type":"record","name":"Root","fields":[{"name":"foo","type":"string"}]}' | jq '{"schema": tojson}' \ -| curl -X POST -H 'content-type:application/json' -d @- "${RISEDEV_SCHEMA_REGISTRY_URL}/subjects/avro_alter_source_test-value/versions" +sr_register avro_alter_source_test-value '{"type":"record","name":"Root","fields":[{"name":"foo","type":"string"}]}' statement ok create source s @@ -27,8 +26,7 @@ FORMAT PLAIN ENCODE AVRO ( # create a new version of schema and produce a message system ok -echo '{"type":"record","name":"Root","fields":[{"name":"bar","type":"int","default":0},{"name":"foo","type":"string"}]}' | jq '{"schema": tojson}' \ -| curl -X POST -H 'content-type:application/json' -d @- "${RISEDEV_SCHEMA_REGISTRY_URL}/subjects/avro_alter_source_test-value/versions" +sr_register avro_alter_source_test-value '{"type":"record","name":"Root","fields":[{"name":"bar","type":"int","default":0},{"name":"foo","type":"string"}]}' system ok echo '{"foo":"ABC", "bar":1}' | rpk topic produce --schema-id=topic avro_alter_source_test diff --git a/e2e_test/source_inline/kafka/avro/union.slt b/e2e_test/source_inline/kafka/avro/union.slt new file mode 100644 index 0000000000000..44e1db659d120 --- /dev/null +++ b/e2e_test/source_inline/kafka/avro/union.slt @@ -0,0 +1,175 @@ +control substitution on + +system ok +rpk topic delete 'avro-union' || true; \ +(rpk sr subject delete 'avro-union-value' && rpk sr subject delete 'avro-union-value' --permanent) || true; +rpk topic create avro-union + +system ok +sr_register avro-union-value ' +{ + "type": "record", + "name": "Root", + "fields": [ + { + "name": "unionType", + "type": ["int", "string"] + }, + { + "name": "unionTypeComplex", + "type": [ + "null", + {"type": "record", "name": "Email","fields": [{"name":"inner","type":"string"}]}, + {"type": "record", "name": "Fax","fields": [{"name":"inner","type":"int"}]}, + {"type": "record", "name": "Sms","fields": [{"name":"inner","type":"int"}]} + ] + }, + { + "name": "enumField", + "type": ["null", "int", { + "type": "enum", + "name": "myEnum", + "namespace": "my.namespace", + "symbols": ["A", "B", "C", "D"] + }], + "default": null + } + ] +} +' + +system ok +cat<: ScalarBounds> + 'a + Copy { macro_rules! scalar_impl_enum { ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { /// `ScalarImpl` embeds all possible scalars in the evaluation framework. + /// + /// See `for_all_variants` for the definition. #[derive(Debug, Clone, PartialEq, Eq, EstimateSize)] pub enum ScalarImpl { $( $variant_name($scalar) ),* diff --git a/src/connector/codec/src/decoder/avro/mod.rs b/src/connector/codec/src/decoder/avro/mod.rs index cdd9aea416c8f..2cd6cf5ac77c3 100644 --- a/src/connector/codec/src/decoder/avro/mod.rs +++ b/src/connector/codec/src/decoder/avro/mod.rs @@ -15,7 +15,7 @@ mod schema; use std::sync::LazyLock; -use apache_avro::schema::{DecimalSchema, RecordSchema}; +use apache_avro::schema::{DecimalSchema, RecordSchema, UnionSchema}; use apache_avro::types::{Value, ValueKind}; use apache_avro::{Decimal as AvroDecimal, Schema}; use chrono::Datelike; @@ -33,6 +33,7 @@ use risingwave_common::util::iter_util::ZipEqFast; pub use self::schema::{avro_schema_to_column_descs, MapHandling, ResolvedAvroSchema}; use super::utils::extract_decimal; use super::{bail_uncategorized, uncategorized, Access, AccessError, AccessResult}; +use crate::decoder::avro::schema::avro_schema_to_struct_field_name; #[derive(Clone)] /// Options for parsing an `AvroValue` into Datum, with an optional avro schema. @@ -107,6 +108,54 @@ impl<'a> AvroParseOptions<'a> { let v: ScalarImpl = match (type_expected, value) { (_, Value::Null) => return Ok(DatumCow::NULL), + // ---- Union (with >=2 non null variants), and nullable Union ([null, record]) ----- + (DataType::Struct(struct_type_info), Value::Union(variant, v)) => { + let Some(Schema::Union(u)) = self.schema else { + // XXX: Is this branch actually unreachable? (if self.schema is correctly used) + return Err(create_error()); + }; + + if let Some(inner) = get_nullable_union_inner(u) { + // nullable Union ([null, record]) + return Self { + schema: Some(inner), + relax_numeric: self.relax_numeric, + } + .convert_to_datum(v, type_expected); + } + let variant_schema = &u.variants()[*variant as usize]; + + if matches!(variant_schema, &Schema::Null) { + return Ok(DatumCow::NULL); + } + + // Here we compare the field name, instead of using the variant idx to find the field idx. + // The latter approach might also work, but might be more error-prone. + // We will need to get the index of the "null" variant, and then re-map the variant index to the field index. + // XXX: probably we can unwrap here (if self.schema is correctly used) + let expected_field_name = avro_schema_to_struct_field_name(variant_schema)?; + + let mut fields = Vec::with_capacity(struct_type_info.len()); + for (field_name, field_type) in struct_type_info + .names() + .zip_eq_fast(struct_type_info.types()) + { + if field_name == expected_field_name { + let datum = Self { + schema: Some(variant_schema), + relax_numeric: self.relax_numeric, + } + .convert_to_datum(v, field_type)? + .to_owned_datum(); + + fields.push(datum) + } else { + fields.push(None) + } + } + StructValue::new(fields).into() + } + // nullable Union ([null, T]) (_, Value::Union(_, v)) => { let schema = self.extract_inner_schema(None); return Self { @@ -290,6 +339,12 @@ impl Access for AvroAccess<'_> { let mut value = self.value; let mut options: AvroParseOptions<'_> = self.options.clone(); + debug_assert!( + path.len() == 1 + || (path.len() == 2 && matches!(path[0], "before" | "after" | "source")), + "unexpected path access: {:?}", + path + ); let mut i = 0; while i < path.len() { let key = path[i]; @@ -299,6 +354,29 @@ impl Access for AvroAccess<'_> { }; match value { Value::Union(_, v) => { + // The debezium "before" field is a nullable union. + // "fields": [ + // { + // "name": "before", + // "type": [ + // "null", + // { + // "type": "record", + // "name": "Value", + // "fields": [...], + // } + // ], + // "default": null + // }, + // { + // "name": "after", + // "type": [ + // "null", + // "Value" + // ], + // "default": null + // }, + // ...] value = v; options.schema = options.extract_inner_schema(None); continue; @@ -338,18 +416,30 @@ pub(crate) fn avro_decimal_to_rust_decimal( )) } -pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { +/// If the union schema is `[null, T]` or `[T, null]`, returns `Some(T)`; otherwise returns `None`. +fn get_nullable_union_inner(union_schema: &UnionSchema) -> Option<&'_ Schema> { + let variants = union_schema.variants(); + // Note: `[null, null] is invalid`, we don't need to worry about that. + if variants.len() == 2 && variants.contains(&Schema::Null) { + let inner_schema = variants + .iter() + .find(|s| !matches!(s, &&Schema::Null)) + .unwrap(); + Some(inner_schema) + } else { + None + } +} + +pub fn avro_schema_skip_nullable_union(schema: &Schema) -> anyhow::Result<&Schema> { match schema { - Schema::Union(union_schema) => { - let inner_schema = union_schema - .variants() - .iter() - .find(|s| !matches!(s, &&Schema::Null)) - .ok_or_else(|| { - anyhow::format_err!("illegal avro record schema {:?}", union_schema) - })?; - Ok(inner_schema) - } + Schema::Union(union_schema) => match get_nullable_union_inner(union_schema) { + Some(s) => Ok(s), + None => Err(anyhow::format_err!( + "illegal avro union schema, expected [null, T], got {:?}", + union_schema + )), + }, other => Ok(other), } } @@ -372,7 +462,9 @@ pub fn avro_extract_field_schema<'a>( Ok(&field.schema) } Schema::Array(schema) => Ok(schema), - Schema::Union(_) => avro_schema_skip_union(schema), + // Only nullable union should be handled here. + // We will not extract inner schema for real union (and it's not extractable). + Schema::Union(_) => avro_schema_skip_nullable_union(schema), Schema::Map(schema) => Ok(schema), _ => bail!("avro schema does not have inner item, schema: {:?}", schema), } @@ -476,11 +568,337 @@ pub(crate) fn avro_to_jsonb(avro: &Value, builder: &mut jsonbb::Builder) -> Acce mod tests { use std::str::FromStr; - use apache_avro::Decimal as AvroDecimal; + use apache_avro::{from_avro_datum, Decimal as AvroDecimal}; + use expect_test::expect; use risingwave_common::types::{Datum, Decimal}; use super::*; + /// Test the behavior of the Rust Avro lib for handling union with logical type. + #[test] + fn test_avro_lib_union() { + // duplicate types + let s = Schema::parse_str(r#"["null", "null"]"#); + expect![[r#" + Err( + Unions cannot contain duplicate types, + ) + "#]] + .assert_debug_eq(&s); + let s = Schema::parse_str(r#"["int", "int"]"#); + expect![[r#" + Err( + Unions cannot contain duplicate types, + ) + "#]] + .assert_debug_eq(&s); + // multiple map/array are considered as the same type, regardless of the element type! + let s = Schema::parse_str( + r#"[ +"null", +{ + "type": "map", + "values" : "long", + "default": {} +}, +{ + "type": "map", + "values" : "int", + "default": {} +} +] +"#, + ); + expect![[r#" + Err( + Unions cannot contain duplicate types, + ) + "#]] + .assert_debug_eq(&s); + let s = Schema::parse_str( + r#"[ +"null", +{ + "type": "array", + "items" : "long", + "default": {} +}, +{ + "type": "array", + "items" : "int", + "default": {} +} +] +"#, + ); + expect![[r#" + Err( + Unions cannot contain duplicate types, + ) + "#]] + .assert_debug_eq(&s); + // multiple named types + let s = Schema::parse_str( + r#"[ +"null", +{"type":"fixed","name":"a","size":16}, +{"type":"fixed","name":"b","size":32} +] +"#, + ); + expect![[r#" + Ok( + Union( + UnionSchema { + schemas: [ + Null, + Fixed( + FixedSchema { + name: Name { + name: "a", + namespace: None, + }, + aliases: None, + doc: None, + size: 16, + attributes: {}, + }, + ), + Fixed( + FixedSchema { + name: Name { + name: "b", + namespace: None, + }, + aliases: None, + doc: None, + size: 32, + attributes: {}, + }, + ), + ], + variant_index: { + Null: 0, + }, + }, + ), + ) + "#]] + .assert_debug_eq(&s); + + // union in union + let s = Schema::parse_str(r#"["int", ["null", "int"]]"#); + expect![[r#" + Err( + Unions may not directly contain a union, + ) + "#]] + .assert_debug_eq(&s); + + // logical type + let s = Schema::parse_str(r#"["null", {"type":"string","logicalType":"uuid"}]"#).unwrap(); + expect![[r#" + Union( + UnionSchema { + schemas: [ + Null, + Uuid, + ], + variant_index: { + Null: 0, + Uuid: 1, + }, + }, + ) + "#]] + .assert_debug_eq(&s); + // Note: Java Avro lib rejects this (logical type unions with its physical type) + let s = Schema::parse_str(r#"["string", {"type":"string","logicalType":"uuid"}]"#).unwrap(); + expect![[r#" + Union( + UnionSchema { + schemas: [ + String, + Uuid, + ], + variant_index: { + String: 0, + Uuid: 1, + }, + }, + ) + "#]] + .assert_debug_eq(&s); + // Note: Java Avro lib rejects this (logical type unions with its physical type) + let s = Schema::parse_str(r#"["int", {"type":"int", "logicalType": "date"}]"#).unwrap(); + expect![[r#" + Union( + UnionSchema { + schemas: [ + Int, + Date, + ], + variant_index: { + Int: 0, + Date: 1, + }, + }, + ) + "#]] + .assert_debug_eq(&s); + // Note: Java Avro lib allows this (2 decimal with different "name") + let s = Schema::parse_str( + r#"[ +{"type":"fixed","name":"Decimal128","size":16,"logicalType":"decimal","precision":38,"scale":2}, +{"type":"fixed","name":"Decimal256","size":32,"logicalType":"decimal","precision":50,"scale":2} +]"#, + ); + expect![[r#" + Err( + Unions cannot contain duplicate types, + ) + "#]] + .assert_debug_eq(&s); + } + + #[test] + fn test_avro_lib_union_record_bug() { + // multiple named types (record) + let s = Schema::parse_str( + r#" + { + "type": "record", + "name": "Root", + "fields": [ + { + "name": "unionTypeComplex", + "type": [ + "null", + {"type": "record", "name": "Email","fields": [{"name":"inner","type":"string"}]}, + {"type": "record", "name": "Fax","fields": [{"name":"inner","type":"int"}]}, + {"type": "record", "name": "Sms","fields": [{"name":"inner","type":"int"}]} + ] + } + ] + } + "#, + ) + .unwrap(); + + let bytes = hex::decode("060c").unwrap(); + // Correct should be variant 3 (Sms) + let correct_value = from_avro_datum(&s, &mut bytes.as_slice(), None); + expect![[r#" + Ok( + Record( + [ + ( + "unionTypeComplex", + Union( + 3, + Record( + [ + ( + "inner", + Int( + 6, + ), + ), + ], + ), + ), + ), + ], + ), + ) + "#]] + .assert_debug_eq(&correct_value); + // Bug: We got variant 2 (Fax) here, if we pass the reader schema. + let wrong_value = from_avro_datum(&s, &mut bytes.as_slice(), Some(&s)); + expect![[r#" + Ok( + Record( + [ + ( + "unionTypeComplex", + Union( + 2, + Record( + [ + ( + "inner", + Int( + 6, + ), + ), + ], + ), + ), + ), + ], + ), + ) + "#]] + .assert_debug_eq(&wrong_value); + + // The bug below can explain what happened. + // The two records below are actually incompatible: https://avro.apache.org/docs/1.11.1/specification/_print/#schema-resolution + // > both schemas are records with the _same (unqualified) name_ + // In from_avro_datum, it first reads the value with the writer schema, and then + // it just uses the reader schema to interpret the value. + // The value doesn't have record "name" information. So it wrongly passed the conversion. + // The correct way is that we need to use both the writer and reader schema in the second step to interpret the value. + + let s = Schema::parse_str( + r#" + { + "type": "record", + "name": "Root", + "fields": [ + { + "name": "a", + "type": "int" + } + ] + } + "#, + ) + .unwrap(); + let s2 = Schema::parse_str( + r#" +{ + "type": "record", + "name": "Root222", + "fields": [ + { + "name": "a", + "type": "int" + } + ] +} + "#, + ) + .unwrap(); + + let bytes = hex::decode("0c").unwrap(); + let value = from_avro_datum(&s, &mut bytes.as_slice(), Some(&s2)); + expect![[r#" + Ok( + Record( + [ + ( + "a", + Int( + 6, + ), + ), + ], + ), + ) + "#]] + .assert_debug_eq(&value); + } + #[test] fn test_convert_decimal() { // 280 diff --git a/src/connector/codec/src/decoder/avro/schema.rs b/src/connector/codec/src/decoder/avro/schema.rs index fe96495d089ea..324b7fd426a56 100644 --- a/src/connector/codec/src/decoder/avro/schema.rs +++ b/src/connector/codec/src/decoder/avro/schema.rs @@ -14,14 +14,18 @@ use std::sync::{Arc, LazyLock}; +use anyhow::Context; use apache_avro::schema::{DecimalSchema, RecordSchema, ResolvedSchema, Schema}; use apache_avro::AvroResult; use itertools::Itertools; -use risingwave_common::bail; +use risingwave_common::error::NotImplemented; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{DataType, Decimal}; +use risingwave_common::{bail, bail_not_implemented}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; +use super::get_nullable_union_inner; + /// Avro schema with `Ref` inlined. The newtype is used to indicate whether the schema is resolved. /// /// TODO: Actually most of the place should use resolved schema, but currently they just happen to work (Some edge cases are not met yet). @@ -198,20 +202,46 @@ fn avro_type_mapping( DataType::List(Box::new(item_type)) } Schema::Union(union_schema) => { - // We only support using union to represent nullable fields, not general unions. - let variants = union_schema.variants(); - if variants.len() != 2 || !variants.contains(&Schema::Null) { - bail!( - "unsupported Avro type, only unions like [null, T] is supported: {:?}", - schema - ); - } - let nested_schema = variants - .iter() - .find_or_first(|s| !matches!(s, Schema::Null)) - .unwrap(); + // Note: Unions may not immediately contain other unions. So a `null` must represent a top-level null. + // e.g., ["null", ["null", "string"]] is not allowed + + // Note: Unions may not contain more than one schema with the same type, except for the named types record, fixed and enum. + // https://avro.apache.org/docs/1.11.1/specification/_print/#unions + debug_assert!( + union_schema + .variants() + .iter() + .map(Schema::canonical_form) // Schema doesn't implement Eq, but only PartialEq. + .duplicates() + .next() + .is_none(), + "Union contains duplicate types: {union_schema:?}", + ); + match get_nullable_union_inner(union_schema) { + Some(inner) => avro_type_mapping(inner, map_handling)?, + None => { + // Convert the union to a struct, each field of the struct represents a variant of the union. + // Refer to https://github.com/risingwavelabs/risingwave/issues/16273#issuecomment-2179761345 to see why it's not perfect. + // Note: Avro union's variant tag is type name, not field name (unlike Rust enum, or Protobuf oneof). + + // XXX: do we need to introduce union.handling.mode? + let (fields, field_names) = union_schema + .variants() + .iter() + // null will mean the whole struct is null + .filter(|variant| !matches!(variant, &&Schema::Null)) + .map(|variant| { + avro_type_mapping(variant, map_handling).and_then(|t| { + let name = avro_schema_to_struct_field_name(variant)?; + Ok((t, name)) + }) + }) + .process_results(|it| it.unzip::<_, _, Vec<_>, Vec<_>>()) + .context("failed to convert Avro union to struct")?; - avro_type_mapping(nested_schema, map_handling)? + DataType::new_struct(fields, field_names) + } + } } Schema::Ref { name } => { if name.name == DBZ_VARIABLE_SCALE_DECIMAL_NAME @@ -219,7 +249,7 @@ fn avro_type_mapping( { DataType::Decimal } else { - bail!("unsupported Avro type: {:?}", schema); + bail_not_implemented!("Avro type: {:?}", schema); } } Schema::Map(value_schema) => { @@ -229,20 +259,25 @@ fn avro_type_mapping( if supported_avro_to_json_type(value_schema) { DataType::Jsonb } else { - bail!( - "unsupported Avro type, cannot convert map to jsonb: {:?}", + bail_not_implemented!( + issue = 16963, + "Avro map type to jsonb: {:?}", schema - ) + ); } } None => { + // We require it to be specified, because we don't want to have a bad default behavior. + // But perhaps changing the default behavior won't be a breaking change, + // because it affects only on creation time, what the result ColumnDesc will be, and the ColumnDesc will be persisted. + // This is unlike timestamp.handing.mode, which affects parser's behavior on the runtime. bail!("`map.handling.mode` not specified in ENCODE AVRO (...). Currently supported modes: `jsonb`") } } } Schema::Uuid => DataType::Varchar, Schema::Null | Schema::Fixed(_) => { - bail!("unsupported Avro type: {:?}", schema) + bail_not_implemented!("Avro type: {:?}", schema); } }; @@ -280,3 +315,71 @@ fn supported_avro_to_json_type(schema: &Schema) -> bool { | Schema::Union(_) => false, } } + +/// The field name when converting Avro union type to RisingWave struct type. +pub(super) fn avro_schema_to_struct_field_name(schema: &Schema) -> Result { + Ok(match schema { + Schema::Null => unreachable!(), + Schema::Union(_) => unreachable!(), + // Primitive types + Schema::Boolean => "boolean".to_string(), + Schema::Int => "int".to_string(), + Schema::Long => "long".to_string(), + Schema::Float => "float".to_string(), + Schema::Double => "double".to_string(), + Schema::Bytes => "bytes".to_string(), + Schema::String => "string".to_string(), + // Unnamed Complex types + Schema::Array(_) => "array".to_string(), + Schema::Map(_) => "map".to_string(), + // Named Complex types + Schema::Enum(_) | Schema::Ref { name: _ } | Schema::Fixed(_) | Schema::Record(_) => { + // schema.name().unwrap().fullname(None) + // See test_avro_lib_union_record_bug + // https://github.com/risingwavelabs/risingwave/issues/17632 + bail_not_implemented!(issue=17632, "Avro named type used in Union type: {:?}", schema) + + } + + // Logical types are currently banned. See https://github.com/risingwavelabs/risingwave/issues/17616 + +/* + Schema::Uuid => "uuid".to_string(), + // Decimal is the most tricky. https://avro.apache.org/docs/1.11.1/specification/_print/#decimal + // - A decimal logical type annotates Avro bytes _or_ fixed types. + // - It has attributes `precision` and `scale`. + // "For the purposes of schema resolution, two schemas that are decimal logical types match if their scales and precisions match." + // - When the physical type is fixed, it's a named type. And a schema containing 2 decimals is possible: + // [ + // {"type":"fixed","name":"Decimal128","size":16,"logicalType":"decimal","precision":38,"scale":2}, + // {"type":"fixed","name":"Decimal256","size":32,"logicalType":"decimal","precision":50,"scale":2} + // ] + // In this case (a logical type's physical type is a named type), perhaps we should use the physical type's `name`. + Schema::Decimal(_) => "decimal".to_string(), + Schema::Date => "date".to_string(), + // Note: in Avro, the name style is "time-millis", etc. + // But in RisingWave (Postgres), it will require users to use quotes, i.e., + // select (struct)."time-millis", (struct).time_millies from t; + // The latter might be more user-friendly. + Schema::TimeMillis => "time_millis".to_string(), + Schema::TimeMicros => "time_micros".to_string(), + Schema::TimestampMillis => "timestamp_millis".to_string(), + Schema::TimestampMicros => "timestamp_micros".to_string(), + Schema::LocalTimestampMillis => "local_timestamp_millis".to_string(), + Schema::LocalTimestampMicros => "local_timestamp_micros".to_string(), + Schema::Duration => "duration".to_string(), +*/ + Schema::Uuid + | Schema::Decimal(_) + | Schema::Date + | Schema::TimeMillis + | Schema::TimeMicros + | Schema::TimestampMillis + | Schema::TimestampMicros + | Schema::LocalTimestampMillis + | Schema::LocalTimestampMicros + | Schema::Duration => { + bail_not_implemented!(issue=17616, "Avro logicalType used in Union type: {:?}", schema) + } + }) +} diff --git a/src/connector/codec/src/decoder/mod.rs b/src/connector/codec/src/decoder/mod.rs index c7e04ab210a6e..cd7fe14ab74ea 100644 --- a/src/connector/codec/src/decoder/mod.rs +++ b/src/connector/codec/src/decoder/mod.rs @@ -16,6 +16,7 @@ pub mod avro; pub mod json; pub mod utils; +use risingwave_common::error::NotImplemented; use risingwave_common::types::{DataType, Datum, DatumCow, ToOwnedDatum}; use thiserror::Error; use thiserror_ext::Macro; @@ -40,23 +41,41 @@ pub enum AccessError { /// Errors that are not categorized into variants above. #[error("{message}")] Uncategorized { message: String }, + + #[error(transparent)] + NotImplemented(#[from] NotImplemented), } pub type AccessResult = std::result::Result; /// Access to a field in the data structure. Created by `AccessBuilder`. +/// +/// It's the `ENCODE ...` part in `FORMAT ... ENCODE ...` pub trait Access { /// Accesses `path` in the data structure (*parsed* Avro/JSON/Protobuf data), /// and then converts it to RisingWave `Datum`. + /// /// `type_expected` might or might not be used during the conversion depending on the implementation. /// /// # Path /// - /// We usually expect the data is a record (struct), and `path` represents field path. + /// We usually expect the data (`Access` instance) is a record (struct), and `path` represents field path. /// The data (or part of the data) represents the whole row (`Vec`), /// and we use different `path` to access one column at a time. /// - /// e.g., for Avro, we access `["col_name"]`; for Debezium Avro, we access `["before", "col_name"]`. + /// TODO: the meaning of `path` is a little confusing and maybe over-abstracted. + /// `access` does not need to serve arbitrarily deep `path` access, but just "top-level" access. + /// The API creates an illusion that arbitrary access is supported, but it's not. + /// Perhapts we should separate out another trait like `ToDatum`, + /// which only does type mapping, without caring about the path. And `path` itself is only an `enum` instead of `&[&str]`. + /// + /// What `path` to access is decided by the CDC layer, i.e., the `FORMAT ...` part (`ChangeEvent`). + /// e.g., + /// - `DebeziumChangeEvent` accesses `["before", "col_name"]` for value, + /// `["source", "db"]`, `["source", "table"]` etc. for additional columns' values, + /// `["op"]` for op type. + /// - `MaxwellChangeEvent` accesses `["data", "col_name"]` for value, `["type"]` for op type. + /// - In the simplest case, for `FORMAT PLAIN/UPSERT` (`KvEvent`), they just access `["col_name"]` for value, and op type is derived. /// /// # Returns /// diff --git a/src/connector/codec/tests/integration_tests/avro.rs b/src/connector/codec/tests/integration_tests/avro.rs index fab143b2bf9e7..11421c151d7a5 100644 --- a/src/connector/codec/tests/integration_tests/avro.rs +++ b/src/connector/codec/tests/integration_tests/avro.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context; use apache_avro::from_avro_datum; use risingwave_connector_codec::decoder::avro::{ avro_schema_to_column_descs, AvroAccess, AvroParseOptions, MapHandling, ResolvedAvroSchema, }; use risingwave_connector_codec::decoder::Access; use risingwave_connector_codec::AvroSchema; +use thiserror_ext::AsReport; use crate::utils::*; @@ -44,6 +46,24 @@ struct Config { data_encoding: TestDataEncoding, } +fn avro_schema_str_to_risingwave_schema( + avro_schema: &str, + config: &Config, +) -> anyhow::Result<(ResolvedAvroSchema, Vec)> { + // manually implement some logic in AvroParserConfig::map_to_columns + let avro_schema = AvroSchema::parse_str(avro_schema).context("failed to parse Avro schema")?; + let resolved_schema = + ResolvedAvroSchema::create(avro_schema.into()).context("failed to resolve Avro schema")?; + + let rw_schema = + avro_schema_to_column_descs(&resolved_schema.resolved_schema, config.map_handling) + .context("failed to convert Avro schema to RisingWave schema")? + .iter() + .map(ColumnDesc::from) + .collect_vec(); + Ok((resolved_schema, rw_schema)) +} + /// Data driven testing for converting Avro Schema to RisingWave Schema, and then converting Avro data into RisingWave data. /// /// The expected results can be automatically updated. To run and update the tests: @@ -79,17 +99,15 @@ fn check( expected_risingwave_schema: expect_test::Expect, expected_risingwave_data: expect_test::Expect, ) { - // manually implement some logic in AvroParserConfig::map_to_columns - let avro_schema = AvroSchema::parse_str(avro_schema).expect("failed to parse Avro schema"); - let resolved_schema = - ResolvedAvroSchema::create(avro_schema.into()).expect("failed to resolve Avro schema"); - - let rw_schema = - avro_schema_to_column_descs(&resolved_schema.resolved_schema, config.map_handling) - .expect("failed to convert Avro schema to RisingWave schema") - .iter() - .map(ColumnDesc::from) - .collect_vec(); + let (resolved_schema, rw_schema) = + match avro_schema_str_to_risingwave_schema(avro_schema, &config) { + Ok(res) => res, + Err(e) => { + expected_risingwave_schema.assert_eq(&format!("{}", e.as_report())); + expected_risingwave_data.assert_eq(""); + return; + } + }; expected_risingwave_schema.assert_eq(&format!( "{:#?}", rw_schema.iter().map(ColumnDescTestDisplay).collect_vec() @@ -554,3 +572,316 @@ fn test_1() { Owned(Float64(OrderedFloat(NaN)))"#]], ); } + +#[test] +fn test_union() { + // A basic test + check( + r#" +{ + "type": "record", + "name": "Root", + "fields": [ + { + "name": "unionType", + "type": ["int", "string"] + }, + { + "name": "unionTypeComplex", + "type": [ + "null", + {"type": "record", "name": "Email","fields": [{"name":"inner","type":"string"}]}, + {"type": "record", "name": "Fax","fields": [{"name":"inner","type":"int"}]}, + {"type": "record", "name": "Sms","fields": [{"name":"inner","type":"int"}]} + ] + }, + { + "name": "nullableString", + "type": ["null", "string"] + } + ] +} + "#, + &[ + // { + // "unionType": {"int": 114514}, + // "unionTypeComplex": {"Sms": {"inner":6}}, + // "nullableString": null + // } + "00a4fd0d060c00", + // { + // "unionType": {"int": 114514}, + // "unionTypeComplex": {"Fax": {"inner":6}}, + // "nullableString": null + // } + "00a4fd0d040c00", + // { + // "unionType": {"string": "oops"}, + // "unionTypeComplex": null, + // "nullableString": {"string": "hello"} + // } + "02086f6f707300020a68656c6c6f", + // { + // "unionType": {"string": "oops"}, + // "unionTypeComplex": {"Email": {"inner":"a@b.c"}}, + // "nullableString": null + // } + "02086f6f7073020a6140622e6300", + ], + Config { + map_handling: None, + data_encoding: TestDataEncoding::HexBinary, + }, + // FIXME: why the struct type doesn't have field_descs? https://github.com/risingwavelabs/risingwave/issues/17128 + expect![[r#" + failed to convert Avro schema to RisingWave schema: failed to convert Avro union to struct: Feature is not yet implemented: Avro named type used in Union type: Record(RecordSchema { name: Name { name: "Email", namespace: None }, aliases: None, doc: None, fields: [RecordField { name: "inner", doc: None, aliases: None, default: None, schema: String, order: Ascending, position: 0, custom_attributes: {} }], lookup: {"inner": 0}, attributes: {} }) + Tracking issue: https://github.com/risingwavelabs/risingwave/issues/17632"#]], + expect![""], + ); + + // logicalType is currently rejected + // https://github.com/risingwavelabs/risingwave/issues/17616 + check( + r#" +{ +"type": "record", +"name": "Root", +"fields": [ + { + "name": "unionLogical", + "type": ["int", {"type":"int", "logicalType": "date"}] + } +] +} + "#, + &[], + Config { + map_handling: None, + data_encoding: TestDataEncoding::HexBinary, + }, + expect![[r#" + failed to convert Avro schema to RisingWave schema: failed to convert Avro union to struct: Feature is not yet implemented: Avro logicalType used in Union type: Date + Tracking issue: https://github.com/risingwavelabs/risingwave/issues/17616"#]], + expect![""], + ); + + // test named type. Consider namespace. + // https://avro.apache.org/docs/1.11.1/specification/_print/#names + // List of things to take care: + // - Record fields and enum symbols DO NOT have namespace. + // - If the name specified contains a dot, then it is assumed to be a fullname, and any namespace also specified is IGNORED. + // - If a name doesn't have its own namespace, it will look for its most tightly enclosing named schema. + check( + r#" +{ + "type": "record", + "name": "Root", + "namespace": "RootNamespace", + "fields": [ + { + "name": "littleFieldToMakeNestingLooksBetter", + "type": ["null","int"], "default": null + }, + { + "name": "recordField", + "type": ["null", "int", { + "type": "record", + "name": "my.name.spaced.record", + "namespace": "when.name.contains.dot.namespace.is.ignored", + "fields": [ + {"name": "hello", "type": {"type": "int", "default": 1}}, + {"name": "world", "type": {"type": "double", "default": 1}} + ] + }], + "default": null + }, + { + "name": "enumField", + "type": ["null", "int", { + "type": "enum", + "name": "myEnum", + "namespace": "my.namespace", + "symbols": ["A", "B", "C", "D"] + }], + "default": null + }, + { + "name": "anotherEnumFieldUsingRootNamespace", + "type": ["null", "int", { + "type": "enum", + "name": "myEnum", + "symbols": ["A", "B", "C", "D"] + }], + "default": null + } + ] +} +"#, + &[ + // { + // "enumField":{"my.namespace.myEnum":"A"}, + // "anotherEnumFieldUsingRootNamespace":{"RootNamespace.myEnum": "D"} + // } + "000004000406", + ], + Config { + map_handling: None, + data_encoding: TestDataEncoding::HexBinary, + }, + expect![[r#" + failed to convert Avro schema to RisingWave schema: failed to convert Avro union to struct: Feature is not yet implemented: Avro named type used in Union type: Record(RecordSchema { name: Name { name: "record", namespace: Some("my.name.spaced") }, aliases: None, doc: None, fields: [RecordField { name: "hello", doc: None, aliases: None, default: None, schema: Int, order: Ascending, position: 0, custom_attributes: {} }, RecordField { name: "world", doc: None, aliases: None, default: None, schema: Double, order: Ascending, position: 1, custom_attributes: {} }], lookup: {"hello": 0, "world": 1}, attributes: {} }) + Tracking issue: https://github.com/risingwavelabs/risingwave/issues/17632"#]], + expect![""], + ); + + // This is provided by a user https://github.com/risingwavelabs/risingwave/issues/16273#issuecomment-2051480710 + check( + r#" +{ + "namespace": "com.abc.efg.mqtt", + "name": "also.DataMessage", + "type": "record", + "fields": [ + { + "name": "metrics", + "type": { + "type": "array", + "items": { + "name": "also_data_metric", + "type": "record", + "fields": [ + { + "name": "id", + "type": "string" + }, + { + "name": "name", + "type": "string" + }, + { + "name": "norm_name", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "uom", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "data", + "type": { + "type": "array", + "items": { + "name": "dataItem", + "type": "record", + "fields": [ + { + "name": "ts", + "type": "string", + "doc": "Timestamp of the metric." + }, + { + "name": "value", + "type": [ + "null", + "boolean", + "double", + "string" + ], + "doc": "Value of the metric." + } + ] + } + }, + "doc": "The data message" + } + ], + "doc": "A metric object" + } + }, + "doc": "A list of metrics." + } + ] +} + "#, + &[ + // { + // "metrics": [ + // {"id":"foo", "name":"a", "data": [] } + // ] + // } + "0206666f6f026100000000", + // { + // "metrics": [ + // {"id":"foo", "name":"a", "norm_name": null, "uom": {"string":"c"}, "data": [{"ts":"1", "value":null}, {"ts":"2", "value": {"boolean": true }}] } + // ] + // } + "0206666f6f02610002026304023100023202010000", + ], + Config { + map_handling: None, + data_encoding: TestDataEncoding::HexBinary, + }, + expect![[r#" + [ + metrics(#1): List( + Struct { + id: Varchar, + name: Varchar, + norm_name: Varchar, + uom: Varchar, + data: List( + Struct { + ts: Varchar, + value: Struct { + boolean: Boolean, + double: Float64, + string: Varchar, + }, + }, + ), + }, + ), + ]"#]], + expect![[r#" + Owned([ + StructValue( + Utf8("foo"), + Utf8("a"), + null, + null, + [], + ), + ]) + ---- + Owned([ + StructValue( + Utf8("foo"), + Utf8("a"), + null, + Utf8("c"), + [ + StructValue( + Utf8("1"), + null, + ), + StructValue( + Utf8("2"), + StructValue( + Bool(true), + null, + null, + ), + ), + ], + ), + ])"#]], + ); +} diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index 1a40b87c9d498..04f80ebba1ca1 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -19,8 +19,8 @@ use apache_avro::types::Value; use apache_avro::{from_avro_datum, Schema}; use risingwave_common::try_match_expand; use risingwave_connector_codec::decoder::avro::{ - avro_extract_field_schema, avro_schema_skip_union, avro_schema_to_column_descs, AvroAccess, - AvroParseOptions, ResolvedAvroSchema, + avro_extract_field_schema, avro_schema_skip_nullable_union, avro_schema_to_column_descs, + AvroAccess, AvroParseOptions, ResolvedAvroSchema, }; use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; use risingwave_pb::plan_common::ColumnDesc; @@ -125,8 +125,40 @@ impl DebeziumAvroParserConfig { } pub fn map_to_columns(&self) -> ConnectorResult> { + // Refer to debezium_avro_msg_schema.avsc for how the schema looks like: + + // "fields": [ + // { + // "name": "before", + // "type": [ + // "null", + // { + // "type": "record", + // "name": "Value", + // "fields": [...], + // } + // ], + // "default": null + // }, + // { + // "name": "after", + // "type": [ + // "null", + // "Value" + // ], + // "default": null + // }, + // ...] + + // Other fields are: + // - source: describes the source metadata for the event + // - op + // - ts_ms + // - transaction + // See + avro_schema_to_column_descs( - avro_schema_skip_union(avro_extract_field_schema( + avro_schema_skip_nullable_union(avro_extract_field_schema( // FIXME: use resolved schema here. // Currently it works because "after" refers to a subtree in "before", // but in theory, inside "before" there could also be a reference. @@ -227,7 +259,7 @@ mod tests { let outer_schema = get_outer_schema(); let expected_inner_schema = Schema::parse_str(inner_shema_str).unwrap(); - let extracted_inner_schema = avro_schema_skip_union( + let extracted_inner_schema = avro_schema_skip_nullable_union( avro_extract_field_schema(&outer_schema, Some("before")).unwrap(), ) .unwrap(); @@ -318,7 +350,7 @@ mod tests { fn test_map_to_columns() { let outer_schema = get_outer_schema(); let columns = avro_schema_to_column_descs( - avro_schema_skip_union( + avro_schema_skip_nullable_union( avro_extract_field_schema(&outer_schema, Some("before")).unwrap(), ) .unwrap(), diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index e4a229bb61b98..ca709e2eebc73 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -534,7 +534,7 @@ impl JsonParseOptions { (DataType::Struct(struct_type_info), ValueType::Object) => { // Collecting into a Result> doesn't reserve the capacity in advance, so we `Vec::with_capacity` instead. // https://github.com/rust-lang/rust/issues/48994 - let mut fields = Vec::with_capacity(struct_type_info.types().len()); + let mut fields = Vec::with_capacity(struct_type_info.len()); for (field_name, field_type) in struct_type_info .names() .zip_eq_fast(struct_type_info.types()) From c9a6a1fdde4b78ccea71d27a8432753ed697df55 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 9 Jul 2024 20:15:01 +0800 Subject: [PATCH 06/20] refactor(common): remove dead code and simplify (#17585) Signed-off-by: xxchan --- src/common/common_service/src/lib.rs | 12 +- .../common_service/src/observer_manager.rs | 49 +--- src/common/metrics/src/monitor/connection.rs | 22 -- src/common/metrics/src/monitor/mod.rs | 71 +----- src/common/metrics/src/monitor/my_stats.rs | 221 ------------------ src/compute/src/observer/observer_manager.rs | 6 +- src/compute/src/server.rs | 6 +- src/frontend/src/observer/observer_manager.rs | 6 +- src/frontend/src/session.rs | 3 +- src/meta/node/src/server.rs | 5 +- src/meta/src/hummock/manager/mod.rs | 2 +- src/object_store/src/object/s3.rs | 2 +- src/rpc_client/src/compute_client.rs | 2 +- src/rpc_client/src/connector_client.rs | 2 +- src/rpc_client/src/stream_client.rs | 2 +- .../compactor_observer/observer_manager.rs | 6 +- src/storage/compactor/src/server.rs | 5 +- .../src/bin/replay/replay_impl.rs | 2 +- .../hummock_test/src/compactor_tests.rs | 2 +- .../src/mock_notification_client.rs | 2 +- src/storage/hummock_test/src/test_utils.rs | 2 +- src/storage/src/hummock/observer_manager.rs | 6 +- .../src/hummock/store/hummock_storage.rs | 2 +- src/storage/src/store_impl.rs | 2 +- 24 files changed, 53 insertions(+), 387 deletions(-) delete mode 100644 src/common/metrics/src/monitor/my_stats.rs diff --git a/src/common/common_service/src/lib.rs b/src/common/common_service/src/lib.rs index c09c84012819b..2cf9a56e076f3 100644 --- a/src/common/common_service/src/lib.rs +++ b/src/common/common_service/src/lib.rs @@ -18,9 +18,13 @@ #![feature(impl_trait_in_assoc_type)] #![feature(error_generic_member_access)] -pub mod metrics_manager; -pub mod observer_manager; +mod metrics_manager; +mod observer_manager; +mod tracing; pub use metrics_manager::MetricsManager; - -pub mod tracing; +pub use observer_manager::{ + Channel, NotificationClient, ObserverError, ObserverManager, ObserverState, + RpcNotificationClient, +}; +pub use tracing::TracingExtractLayer; diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index e760a0e16866c..bf7e457be8b1c 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -22,42 +22,6 @@ use thiserror_ext::AsReport; use tokio::task::JoinHandle; use tonic::{Status, Streaming}; -pub trait SubscribeTypeEnum { - fn subscribe_type() -> SubscribeType; -} - -pub struct SubscribeFrontend {} - -impl SubscribeTypeEnum for SubscribeFrontend { - fn subscribe_type() -> SubscribeType { - SubscribeType::Frontend - } -} - -pub struct SubscribeHummock {} - -impl SubscribeTypeEnum for SubscribeHummock { - fn subscribe_type() -> SubscribeType { - SubscribeType::Hummock - } -} - -pub struct SubscribeCompactor {} - -impl SubscribeTypeEnum for SubscribeCompactor { - fn subscribe_type() -> SubscribeType { - SubscribeType::Compactor - } -} - -pub struct SubscribeCompute {} - -impl SubscribeTypeEnum for SubscribeCompute { - fn subscribe_type() -> SubscribeType { - SubscribeType::Compute - } -} - /// `ObserverManager` is used to update data based on notification from meta. /// Call `start` to spawn a new asynchronous task /// We can write the notification logic by implementing `ObserverNodeImpl`. @@ -68,7 +32,7 @@ pub struct ObserverManager { } pub trait ObserverState: Send + 'static { - type SubscribeType: SubscribeTypeEnum; + fn subscribe_type() -> SubscribeType; /// modify data after receiving notification from meta fn handle_notification(&mut self, resp: SubscribeResponse); @@ -109,10 +73,7 @@ where S: ObserverState, { pub async fn new(client: T, observer_states: S) -> Self { - let rx = client - .subscribe(S::SubscribeType::subscribe_type()) - .await - .unwrap(); + let rx = client.subscribe(S::subscribe_type()).await.unwrap(); Self { rx, client, @@ -214,11 +175,7 @@ where /// `re_subscribe` is used to re-subscribe to the meta's notification. async fn re_subscribe(&mut self) { loop { - match self - .client - .subscribe(S::SubscribeType::subscribe_type()) - .await - { + match self.client.subscribe(S::subscribe_type()).await { Ok(rx) => { tracing::debug!("re-subscribe success"); self.rx = rx; diff --git a/src/common/metrics/src/monitor/connection.rs b/src/common/metrics/src/monitor/connection.rs index 295fb6399ba4b..e5774a3f16d7d 100644 --- a/src/common/metrics/src/monitor/connection.rs +++ b/src/common/metrics/src/monitor/connection.rs @@ -587,28 +587,6 @@ impl tonic::transport::server::Router { } } -#[cfg(not(madsim))] -pub fn monitored_tcp_incoming( - listen_addr: std::net::SocketAddr, - connection_type: impl Into, - config: TcpConfig, -) -> Result< - MonitoredConnection, - Box, -> { - let incoming = tonic::transport::server::TcpIncoming::new( - listen_addr, - config.tcp_nodelay, - config.keepalive_duration, - )?; - Ok(MonitoredConnection::new( - incoming, - MonitorNewConnectionImpl { - connection_type: connection_type.into(), - }, - )) -} - #[derive(Clone)] pub struct MonitorNewConnectionImpl { connection_type: String, diff --git a/src/common/metrics/src/monitor/mod.rs b/src/common/metrics/src/monitor/mod.rs index 10b5c966e636a..316cac9ea907c 100644 --- a/src/common/metrics/src/monitor/mod.rs +++ b/src/common/metrics/src/monitor/mod.rs @@ -12,20 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -pub mod connection; -pub mod my_stats; -pub mod process; -pub mod rwlock; +pub use connection::{monitor_connector, EndpointExt, RouterExt, TcpConfig}; +pub use rwlock::MonitoredRwLock; -use std::sync::LazyLock; +mod connection; +mod process; +mod rwlock; -use prometheus::core::{ - AtomicI64, AtomicU64, Collector, GenericCounter, GenericCounterVec, GenericGauge, Metric, -}; -use prometheus::{Histogram, HistogramVec, Registry}; +use std::sync::LazyLock; -use crate::monitor::my_stats::MyHistogram; -use crate::monitor::process::monitor_process; +use prometheus::Registry; #[cfg(target_os = "linux")] static PAGESIZE: std::sync::LazyLock = @@ -35,59 +31,8 @@ static PAGESIZE: std::sync::LazyLock = pub static CLOCK_TICK: std::sync::LazyLock = std::sync::LazyLock::new(|| unsafe { libc::sysconf(libc::_SC_CLK_TCK) as u64 }); -/// Define extension method `print` used in `print_statistics`. -pub trait Print { - fn print(&self); -} - -impl Print for GenericCounter { - fn print(&self) { - let desc = &self.desc()[0].fq_name; - let counter = self.metric().get_counter().get_value() as u64; - println!("{desc} COUNT : {counter}"); - } -} - -impl Print for GenericGauge { - fn print(&self) { - let desc = &self.desc()[0].fq_name; - let counter = self.get(); - println!("{desc} COUNT : {counter}"); - } -} - -impl Print for Histogram { - fn print(&self) { - let desc = &self.desc()[0].fq_name; - - let histogram = MyHistogram::from_prom_hist(self.metric().get_histogram()); - let p50 = histogram.get_percentile(50.0); - let p95 = histogram.get_percentile(95.0); - let p99 = histogram.get_percentile(99.0); - let p100 = histogram.get_percentile(100.0); - - let sample_count = self.get_sample_count(); - let sample_sum = self.get_sample_sum(); - println!("{desc} P50 : {p50} P95 : {p95} P99 : {p99} P100 : {p100} COUNT : {sample_count} SUM : {sample_sum}"); - } -} - -impl Print for HistogramVec { - fn print(&self) { - let desc = &self.desc()[0].fq_name; - println!("{desc} {:?}", self); - } -} - -impl Print for GenericCounterVec { - fn print(&self) { - let desc = &self.desc()[0].fq_name; - println!("{desc} {:?}", self); - } -} - pub static GLOBAL_METRICS_REGISTRY: LazyLock = LazyLock::new(|| { let registry = Registry::new(); - monitor_process(®istry); + process::monitor_process(®istry); registry }); diff --git a/src/common/metrics/src/monitor/my_stats.rs b/src/common/metrics/src/monitor/my_stats.rs deleted file mode 100644 index 52c71167f2f97..0000000000000 --- a/src/common/metrics/src/monitor/my_stats.rs +++ /dev/null @@ -1,221 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::fmt::{Display, Formatter}; - -use itertools::Itertools; -use prometheus::proto::Histogram; -use rw_iter_util::ZipEqFast; - -#[derive(Clone, Default, Debug)] -pub struct MyHistogram { - pub upper_bound_list: Vec, - pub count_list: Vec, - pub total_count: u64, - pub total_sum: f64, -} - -impl MyHistogram { - pub fn from_prom_hist(histogram: &Histogram) -> MyHistogram { - let mut upper_bound_list = Vec::new(); - let mut count_list = Vec::new(); - - let total_count = histogram.get_sample_count(); - let total_sum = histogram.get_sample_sum(); - - let buckets = histogram.get_bucket(); - for bucket in buckets { - let upper_bound = bucket.get_upper_bound(); - let count = bucket.get_cumulative_count(); - upper_bound_list.push(upper_bound); - count_list.push(count); - } - - MyHistogram { - upper_bound_list, - count_list, - total_count, - total_sum, - } - } - - pub fn from_diff(prev: &MyHistogram, cur: &MyHistogram) -> MyHistogram { - MyHistogram { - upper_bound_list: cur.upper_bound_list.clone(), - count_list: match prev.count_list.is_empty() { - true => cur.count_list.clone(), - false => prev - .count_list - .iter() - .zip_eq_fast(cur.count_list.iter()) - .map(|(&pb, &cb)| cb - pb) - .collect_vec(), - }, - total_sum: cur.total_sum - prev.total_sum, - total_count: cur.total_count - prev.total_count, - } - } - - pub fn get_percentile(&self, p: f64) -> f64 { - let sample_count = self.total_count; - - // empty bucket may appear - if sample_count == 0 { - return 0.0; - } - let threshold = (sample_count as f64 * (p / 100.0_f64)).ceil() as u64; - let mut last_upper_bound = 0.0; - let mut last_count = 0; - for (&upper_bound, &count) in self - .upper_bound_list - .iter() - .zip_eq_fast(self.count_list.iter()) - { - if count >= threshold { - // assume scale linearly within this bucket, - // return a value between last_upper_bound and upper_bound - let right_left_diff = upper_bound - last_upper_bound; - return last_upper_bound - + right_left_diff * (threshold - last_count) as f64 - / (count - last_count) as f64; - } - last_upper_bound = upper_bound; - last_count = count; - } - - 0.0 - } -} - -impl Display for MyHistogram { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - // calculate latencies statistics - let mean = self.total_sum / self.total_count as f64; - let p50 = self.get_percentile(50.0); - let p90 = self.get_percentile(90.0); - let p99 = self.get_percentile(99.0); - let p100 = self.get_percentile(100.0); - - write!( - f, - "latency: - mean: {}, - p50: {}, - p90: {}, - p99: {}, - p100: {};", - mean, p50, p90, p99, p100 - ) - } -} - -#[cfg(test)] -mod tests { - use prometheus::core::Metric; - use prometheus::{histogram_opts, register_histogram_with_registry, Registry}; - - use super::*; - - #[test] - fn test_proc_histogram_basic() { - fn new_simple_histogram(upper_bound: u64) -> MyHistogram { - let registry = Registry::new(); - let buckets = (1..=upper_bound).map(|x| x as f64).collect::>(); - let opts = histogram_opts!("test_histogram", "test_histogram", buckets); - - let histogram = register_histogram_with_registry!(opts, registry).unwrap(); - - for value in 1..=upper_bound { - histogram.observe(value as f64); - } - - MyHistogram::from_prom_hist(histogram.metric().get_histogram()) - } - - let histogram = new_simple_histogram(999); - assert_eq!(histogram.get_percentile(50.0) as u64, 500); - assert_eq!(histogram.get_percentile(90.0) as u64, 900); - assert_eq!(histogram.get_percentile(99.0) as u64, 990); - assert_eq!(histogram.get_percentile(99.9) as u64, 999); - assert_eq!(histogram.get_percentile(100.0) as u64, 999); - - let histogram = new_simple_histogram(1000); - assert_eq!(histogram.get_percentile(50.0) as u64, 500); - assert_eq!(histogram.get_percentile(90.0) as u64, 900); - assert_eq!(histogram.get_percentile(99.0) as u64, 990); - assert_eq!(histogram.get_percentile(99.9) as u64, 1000); - assert_eq!(histogram.get_percentile(100.0) as u64, 1000); - - let histogram = new_simple_histogram(9999); - assert_eq!(histogram.get_percentile(50.0) as u64, 5000); - assert_eq!(histogram.get_percentile(90.0) as u64, 9000); - assert_eq!(histogram.get_percentile(99.0) as u64, 9900); - assert_eq!(histogram.get_percentile(99.9) as u64, 9990); - assert_eq!(histogram.get_percentile(100.0) as u64, 9999); - } - - #[test] - fn test_proc_histogram_uneven_distributed() { - let registry = Registry::new(); - let buckets = vec![ - 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1.0, 2.5, 5.0, 10.0, - ]; - let opts = histogram_opts!("test_histogram", "test_histogram", buckets); - let histogram = register_histogram_with_registry!(opts, registry).unwrap(); - - let mut i = 0.005; - while i < 10.0 { - histogram.observe(i); - i += 0.005; - } - - let histogram = MyHistogram::from_prom_hist(histogram.metric().get_histogram()); - assert_eq!(histogram.get_percentile(50.0), 5.0); - assert_eq!(histogram.get_percentile(90.0), 9.004004004004004); - assert_eq!(histogram.get_percentile(99.0), 9.904904904904905); - assert_eq!(histogram.get_percentile(99.9), 9.994994994994995); - assert_eq!(histogram.get_percentile(100.0), 10.0); - } - - #[test] - fn test_proc_histogram_realistic() { - let registry = Registry::new(); - let buckets = vec![ - 0.005, 0.01, 0.025, 0.05, 0.1, 0.25, 0.5, 1.0, 2.5, 5.0, 10.0, - ]; - let opts = histogram_opts!("test_histogram", "test_histogram", buckets); - let histogram = register_histogram_with_registry!(opts, registry).unwrap(); - - histogram.observe(0.0012); - histogram.observe(0.0013); - histogram.observe(0.003); - - histogram.observe(0.0132); - histogram.observe(0.0143); - histogram.observe(0.0146); - histogram.observe(0.0249); - - histogram.observe(0.99); - - histogram.observe(6.11); - histogram.observe(7.833); - - let histogram = MyHistogram::from_prom_hist(histogram.metric().get_histogram()); - assert_eq!(histogram.get_percentile(50.0), 0.0175); - assert_eq!(histogram.get_percentile(90.0), 7.5); - assert_eq!(histogram.get_percentile(99.0), 10.00); - assert_eq!(histogram.get_percentile(99.9), 10.00); - assert_eq!(histogram.get_percentile(100.0), 10.00); - } -} diff --git a/src/compute/src/observer/observer_manager.rs b/src/compute/src/observer/observer_manager.rs index e4192fb0b6f9c..62e2d699668f4 100644 --- a/src/compute/src/observer/observer_manager.rs +++ b/src/compute/src/observer/observer_manager.rs @@ -13,7 +13,7 @@ // limitations under the License. use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; -use risingwave_common_service::observer_manager::{ObserverState, SubscribeCompute}; +use risingwave_common_service::ObserverState; use risingwave_pb::meta::subscribe_response::Info; use risingwave_pb::meta::SubscribeResponse; @@ -22,7 +22,9 @@ pub struct ComputeObserverNode { } impl ObserverState for ComputeObserverNode { - type SubscribeType = SubscribeCompute; + fn subscribe_type() -> risingwave_pb::meta::SubscribeType { + risingwave_pb::meta::SubscribeType::Compute + } fn handle_notification(&mut self, resp: SubscribeResponse) { let Some(info) = resp.info.as_ref() else { diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3b0466b6cc0d1..d7dcbd5146c31 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -28,7 +28,7 @@ use risingwave_common::config::{ MAX_CONNECTION_WINDOW_SIZE, STREAM_WINDOW_SIZE, }; use risingwave_common::lru::init_global_sequencer_args; -use risingwave_common::monitor::connection::{RouterExt, TcpConfig}; +use risingwave_common::monitor::{RouterExt, TcpConfig}; use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::telemetry::manager::TelemetryManager; @@ -38,9 +38,7 @@ use risingwave_common::util::pretty_bytes::convert; use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; -use risingwave_common_service::metrics_manager::MetricsManager; -use risingwave_common_service::observer_manager::ObserverManager; -use risingwave_common_service::tracing::TracingExtractLayer; +use risingwave_common_service::{MetricsManager, ObserverManager, TracingExtractLayer}; use risingwave_connector::source::monitor::GLOBAL_SOURCE_METRICS; use risingwave_dml::dml_manager::DmlManager; use risingwave_pb::common::WorkerType; diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 38e84d213bf49..169c6bff1b950 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::CatalogVersion; use risingwave_common::hash::WorkerSlotMapping; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; -use risingwave_common_service::observer_manager::{ObserverState, SubscribeFrontend}; +use risingwave_common_service::ObserverState; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::relation::RelationInfo; @@ -50,7 +50,9 @@ pub struct FrontendObserverNode { } impl ObserverState for FrontendObserverNode { - type SubscribeType = SubscribeFrontend; + fn subscribe_type() -> risingwave_pb::meta::SubscribeType { + risingwave_pb::meta::SubscribeType::Frontend + } fn handle_notification(&mut self, resp: SubscribeResponse) { let Some(info) = resp.info.as_ref() else { diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 2148656bee00b..5f0ad6d62b751 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -64,8 +64,7 @@ use risingwave_common::util::resource_util; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; -use risingwave_common_service::observer_manager::ObserverManager; -use risingwave_common_service::MetricsManager; +use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_connector::source::monitor::{SourceMetrics, GLOBAL_SOURCE_METRICS}; use risingwave_pb::common::WorkerType; use risingwave_pb::health::health_server::HealthServer; diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 57ccad6c9b5e6..9098ee1429c82 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -21,13 +21,12 @@ use etcd_client::ConnectOptions; use futures::future::join_all; use otlp_embedded::TraceServiceServer; use regex::Regex; -use risingwave_common::monitor::connection::{RouterExt, TcpConfig}; +use risingwave_common::monitor::{RouterExt, TcpConfig}; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::{report_scarf_enabled, report_to_scarf, telemetry_env_enabled}; -use risingwave_common_service::metrics_manager::MetricsManager; -use risingwave_common_service::tracing::TracingExtractLayer; +use risingwave_common_service::{MetricsManager, TracingExtractLayer}; use risingwave_meta::barrier::StreamRpcManager; use risingwave_meta::controller::catalog::CatalogController; use risingwave_meta::controller::cluster::ClusterController; diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 65eede718c5c4..51d3c6c397ea3 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arc_swap::ArcSwap; use bytes::Bytes; use itertools::Itertools; -use risingwave_common::monitor::rwlock::MonitoredRwLock; +use risingwave_common::monitor::MonitoredRwLock; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 3ed5fc01ba40c..f382a97293fa9 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -53,7 +53,7 @@ use futures::{stream, Stream, StreamExt, TryStreamExt}; use hyper::Body; use itertools::Itertools; use risingwave_common::config::ObjectStoreConfig; -use risingwave_common::monitor::connection::monitor_connector; +use risingwave_common::monitor::monitor_connector; use risingwave_common::range::RangeBoundsExt; use thiserror_ext::AsReport; use tokio::task::JoinHandle; diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index f908bb21aa3a2..4d959ec5209a6 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -18,7 +18,7 @@ use std::time::Duration; use async_trait::async_trait; use futures::StreamExt; use risingwave_common::config::{MAX_CONNECTION_WINDOW_SIZE, STREAM_WINDOW_SIZE}; -use risingwave_common::monitor::connection::{EndpointExt, TcpConfig}; +use risingwave_common::monitor::{EndpointExt, TcpConfig}; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::tracing::TracingContext; use risingwave_pb::batch_plan::{PlanFragment, TaskId, TaskOutputId}; diff --git a/src/rpc_client/src/connector_client.rs b/src/rpc_client/src/connector_client.rs index 30d78290b6d98..c81a74d2fa709 100644 --- a/src/rpc_client/src/connector_client.rs +++ b/src/rpc_client/src/connector_client.rs @@ -19,7 +19,7 @@ use std::time::Duration; use anyhow::{anyhow, Context}; use futures::TryStreamExt; use risingwave_common::config::{MAX_CONNECTION_WINDOW_SIZE, STREAM_WINDOW_SIZE}; -use risingwave_common::monitor::connection::{EndpointExt, TcpConfig}; +use risingwave_common::monitor::{EndpointExt, TcpConfig}; use risingwave_pb::connector_service::connector_service_client::ConnectorServiceClient; use risingwave_pb::connector_service::sink_coordinator_stream_request::{ CommitMetadata, StartCoordinator, diff --git a/src/rpc_client/src/stream_client.rs b/src/rpc_client/src/stream_client.rs index 4710be7085ef6..988931cb207b6 100644 --- a/src/rpc_client/src/stream_client.rs +++ b/src/rpc_client/src/stream_client.rs @@ -19,7 +19,7 @@ use anyhow::anyhow; use async_trait::async_trait; use futures::TryStreamExt; use risingwave_common::config::MAX_CONNECTION_WINDOW_SIZE; -use risingwave_common::monitor::connection::{EndpointExt, TcpConfig}; +use risingwave_common::monitor::{EndpointExt, TcpConfig}; use risingwave_common::util::addr::HostAddr; use risingwave_pb::stream_service::stream_service_client::StreamServiceClient; use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; diff --git a/src/storage/compactor/src/compactor_observer/observer_manager.rs b/src/storage/compactor/src/compactor_observer/observer_manager.rs index 3a37fd6060595..97d78f6c2849f 100644 --- a/src/storage/compactor/src/compactor_observer/observer_manager.rs +++ b/src/storage/compactor/src/compactor_observer/observer_manager.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; -use risingwave_common_service::observer_manager::{ObserverState, SubscribeCompactor}; +use risingwave_common_service::ObserverState; use risingwave_pb::catalog::Table; use risingwave_pb::meta::relation::RelationInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -32,7 +32,9 @@ pub struct CompactorObserverNode { } impl ObserverState for CompactorObserverNode { - type SubscribeType = SubscribeCompactor; + fn subscribe_type() -> risingwave_pb::meta::SubscribeType { + risingwave_pb::meta::SubscribeType::Compactor + } fn handle_notification(&mut self, resp: SubscribeResponse) { let Some(info) = resp.info.as_ref() else { diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index 8d246016858ef..e139bc201cd48 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -19,7 +19,7 @@ use std::time::Duration; use risingwave_common::config::{ extract_storage_memory_config, load_config, AsyncStackTraceOption, MetricLevel, RwConfig, }; -use risingwave_common::monitor::connection::{RouterExt, TcpConfig}; +use risingwave_common::monitor::{RouterExt, TcpConfig}; use risingwave_common::system_param::local_manager::LocalSystemParamsManager; use risingwave_common::system_param::reader::{SystemParamsRead, SystemParamsReader}; use risingwave_common::telemetry::manager::TelemetryManager; @@ -29,8 +29,7 @@ use risingwave_common::util::resource_util::memory::system_memory_available_byte use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; -use risingwave_common_service::metrics_manager::MetricsManager; -use risingwave_common_service::observer_manager::ObserverManager; +use risingwave_common_service::{MetricsManager, ObserverManager}; use risingwave_object_store::object::build_remote_object_store; use risingwave_object_store::object::object_metrics::GLOBAL_OBJECT_STORE_METRICS; use risingwave_pb::common::WorkerType; diff --git a/src/storage/hummock_test/src/bin/replay/replay_impl.rs b/src/storage/hummock_test/src/bin/replay/replay_impl.rs index d8a2a2f0c24bd..fe9e5874e328e 100644 --- a/src/storage/hummock_test/src/bin/replay/replay_impl.rs +++ b/src/storage/hummock_test/src/bin/replay/replay_impl.rs @@ -19,7 +19,7 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::catalog::TableId; use risingwave_common::util::addr::HostAddr; -use risingwave_common_service::observer_manager::{Channel, NotificationClient, ObserverError}; +use risingwave_common_service::{Channel, NotificationClient, ObserverError}; use risingwave_hummock_sdk::key::TableKey; use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_trace::{ diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index e3c19f54e4340..96f237704abf5 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -28,7 +28,7 @@ pub(crate) mod tests { use risingwave_common::constants::hummock::CompactionFilterFlag; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, Epoch, EpochExt}; - use risingwave_common_service::observer_manager::NotificationClient; + use risingwave_common_service::NotificationClient; use risingwave_hummock_sdk::can_concat; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{ diff --git a/src/storage/hummock_test/src/mock_notification_client.rs b/src/storage/hummock_test/src/mock_notification_client.rs index 68c75d57937a2..906781a7725b2 100644 --- a/src/storage/hummock_test/src/mock_notification_client.rs +++ b/src/storage/hummock_test/src/mock_notification_client.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_common::util::addr::HostAddr; -use risingwave_common_service::observer_manager::{Channel, NotificationClient, ObserverError}; +use risingwave_common_service::{Channel, NotificationClient, ObserverError}; use risingwave_meta::hummock::{HummockManager, HummockManagerRef}; use risingwave_meta::manager::{MessageStatus, MetaSrvEnv, NotificationManagerRef, WorkerKey}; use risingwave_pb::backup_service::MetaBackupManifestId; diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index 5a2034facdbfa..bf5c4a8dd8d8c 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use itertools::Itertools; use risingwave_common::catalog::TableId; -use risingwave_common_service::observer_manager::ObserverManager; +use risingwave_common_service::ObserverManager; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::TableKey; pub use risingwave_hummock_sdk::key::{gen_key_from_bytes, gen_key_from_str}; diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index 0725424aaca76..a9171005aeaa9 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use std::sync::Arc; -use risingwave_common_service::observer_manager::{ObserverState, SubscribeHummock}; +use risingwave_common_service::ObserverState; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_trace::TraceSpan; use risingwave_pb::catalog::Table; @@ -38,7 +38,9 @@ pub struct HummockObserverNode { } impl ObserverState for HummockObserverNode { - type SubscribeType = SubscribeHummock; + fn subscribe_type() -> risingwave_pb::meta::SubscribeType { + risingwave_pb::meta::SubscribeType::Hummock + } fn handle_notification(&mut self, resp: SubscribeResponse) { let Some(info) = resp.info.as_ref() else { diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 7b28588c5054a..f7794604e5a8b 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -25,7 +25,7 @@ use itertools::Itertools; use more_asserts::assert_gt; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::is_max_epoch; -use risingwave_common_service::observer_manager::{NotificationClient, ObserverManager}; +use risingwave_common_service::{NotificationClient, ObserverManager}; use risingwave_hummock_sdk::key::{ is_empty_key_range, vnode, vnode_range, TableKey, TableKeyRange, }; diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 8dd718324d9a5..20a146ce04f10 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -22,7 +22,7 @@ use foyer::{ DirectFsDeviceOptionsBuilder, HybridCacheBuilder, RateLimitPicker, RuntimeConfigBuilder, }; use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; -use risingwave_common_service::observer_manager::RpcNotificationClient; +use risingwave_common_service::RpcNotificationClient; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_object_store::object::build_remote_object_store; From 568a2712b190f192ce7076e76c7f972bbbbb5d9c Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Tue, 9 Jul 2024 21:30:55 +0800 Subject: [PATCH 07/20] refactor(sink): remove snowflake sink http client (#17627) --- integration_tests/snowflake-sink/README.md | 16 +- .../snowflake-sink/create_sink.sql | 17 +- .../snowflake-sink/snowflake_prep.sql | 16 +- .../snowflake-sink/upsert/create_sink.sql | 17 +- src/connector/src/sink/mod.rs | 1 - src/connector/src/sink/snowflake.rs | 154 ++++++------ src/connector/src/sink/snowflake_connector.rs | 225 ------------------ src/connector/with_options_sink.yaml | 38 +-- .../src/object/opendal_engine/opendal_s3.rs | 1 + 9 files changed, 107 insertions(+), 378 deletions(-) delete mode 100644 src/connector/src/sink/snowflake_connector.rs diff --git a/integration_tests/snowflake-sink/README.md b/integration_tests/snowflake-sink/README.md index 8d38921985b5d..de55dc8c95b32 100644 --- a/integration_tests/snowflake-sink/README.md +++ b/integration_tests/snowflake-sink/README.md @@ -18,22 +18,12 @@ note: the required credentials including the following, i.e., - `snowflake.aws_secret_access_key` (a.k.a. the `AWS_SECRET_KEY` in snowflake stage) ### 1.2 Snowflake setup - -users will then need to setup the snowflake, which includes, i.e., -- generate the key-value pair for later authentication -- create a `role` and grant the appropriate permission -- setup the credential for the user (e.g., `RSA_PUBLIC_KEY`), and retrieve the `snowflake.rsa_public_key_fp` which will later be used in risingwave -- create a `table` to store the sink data from risingwave -- create a `stage` to refer the external s3 bucket, which will be used internally by snowflake to load the corresponding data -- create a `pipe` to actual receive loaded data from the pre-defined stage and copy the data to the snowflake table. - -ps. -1. this assumes the users have already created their accounts and the corresponding databases in snowflake. -2. for detailed authentication process, refer to [official authentication guide](https://docs.snowflake.com/en/developer-guide/sql-api/authenticating). -3. for detailed commands, refer to [official reference](https://docs.snowflake.com/en/reference) +You need to have a table ,a stage and a pipe. In the meantime you need to open s3's SQS queue +You can complete the above setup by https://docs.snowflake.com/en/user-guide/data-load-snowpipe-auto-s3 an example for snowflake setup commands could be checked at `snowflake_prep.sql`, this also corresponds to the following example sinking use case. + ## 2. Begin to sink data launch your risingwave cluster, and execute the following sql commands respectively. diff --git a/integration_tests/snowflake-sink/create_sink.sql b/integration_tests/snowflake-sink/create_sink.sql index 0986005d8c717..6ef762b6708ca 100644 --- a/integration_tests/snowflake-sink/create_sink.sql +++ b/integration_tests/snowflake-sink/create_sink.sql @@ -1,18 +1,11 @@ CREATE SINK snowflake_sink FROM ss_mv WITH ( connector = 'snowflake', type = 'append-only', - snowflake.database = 'EXAMPLE_DB', - snowflake.schema = 'EXAMPLE_SCHEMA', - snowflake.pipe = 'EXAMPLE_SNOWFLAKE_PIPE', - snowflake.account_identifier = '-', - snowflake.user = 'XZHSEH', - snowflake.rsa_public_key_fp = 'EXAMPLE_FP', - snowflake.private_key = 'EXAMPLE_PK', - snowflake.s3_bucket = 'EXAMPLE_S3_BUCKET', - snowflake.aws_access_key_id = 'EXAMPLE_AWS_ID', - snowflake.aws_secret_access_key = 'EXAMPLE_SECRET_KEY', - snowflake.aws_region = 'EXAMPLE_REGION', - snowflake.s3_path = 'EXAMPLE_S3_PATH', + s3.bucket_name = 'EXAMPLE_S3_BUCKET', + s3.credentials.access = 'EXAMPLE_AWS_ACCESS', + s3.credentials.secret = 'EXAMPLE_AWS_SECRET', + s3.region_name = 'ap-EXAMPLE_REGION-2', + s3.path = 'EXAMPLE_S3_PATH', -- depends on your mv setup, note that snowflake sink *only* supports -- `append-only` mode at present. force_append_only = 'true' diff --git a/integration_tests/snowflake-sink/snowflake_prep.sql b/integration_tests/snowflake-sink/snowflake_prep.sql index b684a93b2d299..33b88c799b900 100644 --- a/integration_tests/snowflake-sink/snowflake_prep.sql +++ b/integration_tests/snowflake-sink/snowflake_prep.sql @@ -1,20 +1,5 @@ USE EXAMPLE_DB; -ALTER USER xzhseh SET RSA_PUBLIC_KEY='your local rsa public key'; - --- set user permission to account admin level -GRANT ROLE ACCOUNTADMIN TO USER xzhseh; - --- you could either retrieve the fp from desc user's info panel, --- or from the following select stmt. -DESC USER xzhseh; --- also fine, see the documentation for details. -SELECT TRIM( - (SELECT "value" FROM TABLE(RESULT_SCAN(LAST_QUERY_ID())) - WHERE "property" = 'RSA_PUBLIC_KEY_FP'), - 'SHA256:' -); - -- snowflake table, note to keep the same column name(s). CREATE OR REPLACE TABLE example_snowflake_sink_table (user_id INT, target_id VARCHAR, event_timestamp TIMESTAMP_TZ); @@ -28,6 +13,7 @@ CREATE OR REPLACE STAGE example_snowflake_stage CREATE OR REPLACE PIPE example_snowflake_pipe AS COPY INTO example_snowflake_sink_table FROM @example_snowflake_stage MATCH_BY_COLUMN_NAME = CASE_INSENSITIVE; +-- you will get an AWS SQS ARN, please add it to the list of event notifications in s3 -- select from table after sinking from rw SELECT * FROM example_snowflake_sink_table WHERE event_timestamp IS NOT NULL; diff --git a/integration_tests/snowflake-sink/upsert/create_sink.sql b/integration_tests/snowflake-sink/upsert/create_sink.sql index 20f45968439db..fe85d8bec1683 100644 --- a/integration_tests/snowflake-sink/upsert/create_sink.sql +++ b/integration_tests/snowflake-sink/upsert/create_sink.sql @@ -1,16 +1,9 @@ CREATE SINK snowflake_sink FROM ss_mv WITH ( connector = 'snowflake', type = 'append-only', - snowflake.database = 'EXAMPLE_DB', - snowflake.schema = 'EXAMPLE_SCHEMA', - snowflake.pipe = 'EXAMPLE_SNOWFLAKE_PIPE', - snowflake.account_identifier = '-', - snowflake.user = 'XZHSEH', - snowflake.rsa_public_key_fp = 'EXAMPLE_FP', - snowflake.private_key = 'EXAMPLE_PK', - snowflake.s3_bucket = 'EXAMPLE_S3_BUCKET', - snowflake.aws_access_key_id = 'EXAMPLE_AWS_ID', - snowflake.aws_secret_access_key = 'EXAMPLE_SECRET_KEY', - snowflake.aws_region = 'EXAMPLE_REGION', - snowflake.s3_path = 'EXAMPLE_S3_PATH', + s3.bucket_name = 'EXAMPLE_S3_BUCKET', + s3.credentials.access = 'EXAMPLE_AWS_ACCESS', + s3.credentials.secret = 'EXAMPLE_AWS_SECRET', + s3.region_name = 'ap-EXAMPLE_REGION-2', + s3.path = 'EXAMPLE_S3_PATH', ); \ No newline at end of file diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 872e7bbfeaf7c..a1a993803568b 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -38,7 +38,6 @@ pub mod pulsar; pub mod redis; pub mod remote; pub mod snowflake; -pub mod snowflake_connector; pub mod sqlserver; pub mod starrocks; pub mod test_sink; diff --git a/src/connector/src/sink/snowflake.rs b/src/connector/src/sink/snowflake.rs index 0f512933cd0f7..1c9d67352247c 100644 --- a/src/connector/src/sink/snowflake.rs +++ b/src/connector/src/sink/snowflake.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap}; +use std::collections::BTreeMap; use std::fmt::Write; use std::sync::Arc; @@ -22,7 +22,11 @@ use bytes::{Bytes, BytesMut}; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::Schema; -use risingwave_object_store::object::{ObjectStore, OpendalStreamingUploader, StreamingUploader}; +use risingwave_common::config::ObjectStoreConfig; +use risingwave_object_store::object::object_metrics::GLOBAL_OBJECT_STORE_METRICS; +use risingwave_object_store::object::{ + ObjectStore, OpendalObjectStore, OpendalStreamingUploader, StreamingUploader, +}; use serde::Deserialize; use serde_json::Value; use serde_with::serde_as; @@ -32,70 +36,43 @@ use with_options::WithOptions; use super::encoder::{ JsonEncoder, RowEncoder, TimeHandlingMode, TimestampHandlingMode, TimestamptzHandlingMode, }; -use super::snowflake_connector::{generate_s3_file_name, SnowflakeHttpClient, SnowflakeS3Client}; use super::writer::LogSinkerOf; use super::{SinkError, SinkParam}; use crate::sink::writer::SinkWriterExt; use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkWriter, SinkWriterParam}; pub const SNOWFLAKE_SINK: &str = "snowflake"; +const S3_INTERMEDIATE_FILE_NAME: &str = "RW_SNOWFLAKE_S3_SINK_FILE"; -#[derive(Deserialize, Debug, Clone, WithOptions)] +#[derive(Debug, Clone, Deserialize, WithOptions)] pub struct SnowflakeCommon { - /// The snowflake database used for sinking - #[serde(rename = "snowflake.database")] - pub database: String, - - /// The corresponding schema where sink table exists - #[serde(rename = "snowflake.schema")] - pub schema: String, - - /// The created pipe object, will be used as `insertFiles` target - #[serde(rename = "snowflake.pipe")] - pub pipe: String, - - /// The unique, snowflake provided `account_identifier` - /// NOTE: please use the form `-` - /// For detailed guidance, reference: - #[serde(rename = "snowflake.account_identifier")] - pub account_identifier: String, - - /// The user that owns the table to be sinked - /// NOTE: the user should've been granted corresponding *role* - /// reference: - #[serde(rename = "snowflake.user")] - pub user: String, - - /// The public key fingerprint used when generating custom `jwt_token` - /// reference: - #[serde(rename = "snowflake.rsa_public_key_fp")] - pub rsa_public_key_fp: String, - - /// The rsa pem key *without* encryption - #[serde(rename = "snowflake.private_key")] - pub private_key: String, - /// The s3 bucket where intermediate sink files will be stored - #[serde(rename = "snowflake.s3_bucket")] + #[serde(rename = "snowflake.s3_bucket", alias = "s3.bucket_name")] pub s3_bucket: String, /// The optional s3 path to be specified /// the actual file location would be `s3:////` /// if this field is specified by user(s) /// otherwise it would be `s3:///` - #[serde(rename = "snowflake.s3_path")] + #[serde(rename = "snowflake.s3_path", alias = "s3.path")] pub s3_path: Option, /// s3 credentials - #[serde(rename = "snowflake.aws_access_key_id")] + #[serde( + rename = "snowflake.aws_access_key_id", + alias = "s3.credentials.access" + )] pub aws_access_key_id: String, /// s3 credentials - #[serde(rename = "snowflake.aws_secret_access_key")] + #[serde( + rename = "snowflake.aws_secret_access_key", + alias = "s3.credentials.secret" + )] pub aws_secret_access_key: String, /// The s3 region, e.g., us-east-2 - #[serde(rename = "snowflake.aws_region")] + #[serde(rename = "snowflake.aws_region", alias = "s3.region_name")] pub aws_region: String, } @@ -173,8 +150,6 @@ pub struct SnowflakeSinkWriter { pk_indices: Vec, #[expect(dead_code)] is_append_only: bool, - /// the client used to send `insertFiles` post request - http_client: SnowflakeHttpClient, /// the client to insert file to external storage (i.e., s3) s3_client: SnowflakeS3Client, row_encoder: JsonEncoder, @@ -187,7 +162,7 @@ pub struct SnowflakeSinkWriter { /// note: the option here *implicitly* indicates whether we have at /// least call `streaming_upload` once during this epoch, /// which is mainly used to prevent uploading empty data. - streaming_uploader: Option<(OpendalStreamingUploader, String)>, + streaming_uploader: Option, } impl SnowflakeSinkWriter { @@ -197,18 +172,6 @@ impl SnowflakeSinkWriter { pk_indices: Vec, is_append_only: bool, ) -> Result { - let http_client = SnowflakeHttpClient::new( - config.common.account_identifier.clone(), - config.common.user.clone(), - config.common.database.clone(), - config.common.schema.clone(), - config.common.pipe.clone(), - config.common.rsa_public_key_fp.clone(), - config.common.private_key.clone(), - HashMap::new(), - config.common.s3_path.clone(), - ); - let s3_client = SnowflakeS3Client::new( config.common.s3_bucket.clone(), config.common.s3_path.clone(), @@ -222,7 +185,6 @@ impl SnowflakeSinkWriter { schema: schema.clone(), pk_indices, is_append_only, - http_client, s3_client, row_encoder: JsonEncoder::new( schema, @@ -245,7 +207,7 @@ impl SnowflakeSinkWriter { /// and `streaming_upload` being called the first time. /// i.e., lazily initialization of the internal `streaming_uploader`. /// plus, this function is *pure*, the `&mut self` here is to make rustc (and tokio) happy. - async fn new_streaming_uploader(&mut self) -> Result<(OpendalStreamingUploader, String)> { + async fn new_streaming_uploader(&mut self) -> Result { let file_suffix = self.file_suffix(); let path = generate_s3_file_name(self.s3_client.s3_path(), &file_suffix); let uploader = self @@ -260,12 +222,12 @@ impl SnowflakeSinkWriter { ) }) .map_err(SinkError::Snowflake)?; - Ok((uploader, file_suffix)) + Ok(uploader) } /// write data to the current streaming uploader for this epoch. async fn streaming_upload(&mut self, data: Bytes) -> Result<()> { - let (uploader, _) = match self.streaming_uploader.as_mut() { + let uploader = match self.streaming_uploader.as_mut() { Some(s) => s, None => { assert!( @@ -286,18 +248,18 @@ impl SnowflakeSinkWriter { /// finalize streaming upload for this epoch. /// ensure all the data has been properly uploaded to intermediate s3. - async fn finish_streaming_upload(&mut self) -> Result> { + async fn finish_streaming_upload(&mut self) -> Result<()> { let uploader = std::mem::take(&mut self.streaming_uploader); - let Some((uploader, file_suffix)) = uploader else { + let Some(uploader) = uploader else { // there is no data to be uploaded for this epoch - return Ok(None); + return Ok(()); }; uploader .finish() .await .context("failed to finish streaming upload to s3") .map_err(SinkError::Snowflake)?; - Ok(Some(file_suffix)) + Ok(()) } async fn append_only(&mut self, chunk: StreamChunk) -> Result<()> { @@ -344,13 +306,7 @@ impl SnowflakeSinkWriter { // note that after `finish_streaming_upload`, do *not* interact with // `streaming_uploader` until new data comes in at next epoch, // since the ownership has been taken in this method, and `None` will be left. - let Some(file_suffix) = self.finish_streaming_upload().await? else { - // represents there is no data to be uploaded for this epoch - return Ok(()); - }; - // trigger `insertFiles` post request to snowflake - self.http_client.send_request(&file_suffix).await?; - Ok(()) + self.finish_streaming_upload().await } } @@ -384,3 +340,57 @@ impl SinkWriter for SnowflakeSinkWriter { Ok(()) } } + +/// The helper function to generate the *global unique* s3 file name. +pub(crate) fn generate_s3_file_name(s3_path: Option<&str>, suffix: &str) -> String { + match s3_path { + Some(path) => format!("{}/{}_{}", path, S3_INTERMEDIATE_FILE_NAME, suffix), + None => format!("{}_{}", S3_INTERMEDIATE_FILE_NAME, suffix), + } +} + +/// todo: refactor this part after s3 sink is available +pub struct SnowflakeS3Client { + #[expect(dead_code)] + s3_bucket: String, + s3_path: Option, + pub opendal_s3_engine: OpendalObjectStore, +} + +impl SnowflakeS3Client { + pub fn new( + s3_bucket: String, + s3_path: Option, + aws_access_key_id: String, + aws_secret_access_key: String, + aws_region: String, + ) -> Result { + // FIXME: we should use the `ObjectStoreConfig` instead of default + // just use default configuration here for opendal s3 engine + let config = ObjectStoreConfig::default(); + + let metrics = Arc::new(GLOBAL_OBJECT_STORE_METRICS.clone()); + + // create the s3 engine for streaming upload to the intermediate s3 bucket + let opendal_s3_engine = OpendalObjectStore::new_s3_engine_with_credentials( + &s3_bucket, + Arc::new(config), + metrics, + &aws_access_key_id, + &aws_secret_access_key, + &aws_region, + ) + .context("failed to create opendal s3 engine") + .map_err(SinkError::Snowflake)?; + + Ok(Self { + s3_bucket, + s3_path, + opendal_s3_engine, + }) + } + + pub fn s3_path(&self) -> Option<&str> { + self.s3_path.as_deref() + } +} diff --git a/src/connector/src/sink/snowflake_connector.rs b/src/connector/src/sink/snowflake_connector.rs deleted file mode 100644 index bfd2458900294..0000000000000 --- a/src/connector/src/sink/snowflake_connector.rs +++ /dev/null @@ -1,225 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::HashMap; -use std::sync::Arc; -use std::time::{SystemTime, UNIX_EPOCH}; - -use anyhow::{anyhow, Context}; -use jsonwebtoken::{encode, Algorithm, EncodingKey, Header}; -use object_metrics::GLOBAL_OBJECT_STORE_METRICS; -use reqwest::{header, Client, RequestBuilder, StatusCode}; -use risingwave_object_store::object::*; -use serde::{Deserialize, Serialize}; - -use super::doris_starrocks_connector::POOL_IDLE_TIMEOUT; -use super::{Result, SinkError}; - -const SNOWFLAKE_HOST_ADDR: &str = "snowflakecomputing.com"; -const SNOWFLAKE_REQUEST_ID: &str = "RW_SNOWFLAKE_SINK"; -const S3_INTERMEDIATE_FILE_NAME: &str = "RW_SNOWFLAKE_S3_SINK_FILE"; - -/// The helper function to generate the *global unique* s3 file name. -pub(crate) fn generate_s3_file_name(s3_path: Option<&str>, suffix: &str) -> String { - match s3_path { - Some(path) => format!("{}/{}_{}", path, S3_INTERMEDIATE_FILE_NAME, suffix), - None => format!("{}_{}", S3_INTERMEDIATE_FILE_NAME, suffix), - } -} - -/// Claims is used when constructing `jwt_token` -/// with payload specified. -/// reference: -#[derive(Debug, Serialize, Deserialize)] -struct Claims { - iss: String, - sub: String, - iat: usize, - exp: usize, -} - -#[derive(Debug)] -pub struct SnowflakeHttpClient { - url: String, - rsa_public_key_fp: String, - account: String, - user: String, - private_key: String, - #[expect(dead_code)] - header: HashMap, - s3_path: Option, -} - -impl SnowflakeHttpClient { - pub fn new( - account: String, - user: String, - db: String, - schema: String, - pipe: String, - rsa_public_key_fp: String, - private_key: String, - header: HashMap, - s3_path: Option, - ) -> Self { - // todo: ensure if we need user to *explicitly* provide the `request_id` - // currently it seems that this is not important. - // reference to the snowpipe rest api is as below, i.e., - // - let url = format!( - "https://{}.{}/v1/data/pipes/{}.{}.{}/insertFiles?requestId={}", - account.clone(), - SNOWFLAKE_HOST_ADDR, - db, - schema, - pipe, - SNOWFLAKE_REQUEST_ID - ); - - Self { - url, - rsa_public_key_fp, - account, - user, - private_key, - header, - s3_path, - } - } - - /// Generate a 59-minutes valid `jwt_token` for authentication of snowflake side - /// And please note that we will NOT strictly counting the time interval - /// of `jwt_token` expiration. - /// Which essentially means that this method should be called *every time* we want - /// to send `insertFiles` request to snowflake server. - fn generate_jwt_token(&self) -> Result { - let header = Header::new(Algorithm::RS256); - let now = SystemTime::now() - .duration_since(UNIX_EPOCH) - .unwrap() - .as_secs() as usize; - let lifetime = 59 * 60; - - // Ensure the account and username are uppercase - let account = self.account.to_uppercase(); - let user = self.user.to_uppercase(); - - // Construct the fully qualified username - let qualified_username = format!("{}.{}", account, user); - - let claims = Claims { - iss: format!("{}.{}", qualified_username.clone(), self.rsa_public_key_fp), - sub: qualified_username, - iat: now, - exp: now + lifetime, - }; - - let jwt_token = encode( - &header, - &claims, - &EncodingKey::from_rsa_pem(self.private_key.as_ref()) - .context("failed to encode from provided rsa pem key") - .map_err(SinkError::Snowflake)?, - ) - .context("failed to encode jwt_token") - .map_err(SinkError::Snowflake)?; - Ok(jwt_token) - } - - fn build_request_and_client(&self) -> RequestBuilder { - let client = Client::builder() - .pool_idle_timeout(POOL_IDLE_TIMEOUT) - .build() - .unwrap(); - - client.post(&self.url) - } - - /// NOTE: this function should ONLY be called *after* - /// uploading files to remote external staged storage, i.e., AWS S3 - pub async fn send_request(&self, file_suffix: &str) -> Result<()> { - let builder = self.build_request_and_client(); - - // Generate the jwt_token - let jwt_token = self.generate_jwt_token()?; - let builder = builder - .header(header::CONTENT_TYPE, "text/plain") - .header("Authorization", format!("Bearer {}", jwt_token)) - .header( - "X-Snowflake-Authorization-Token-Type".to_string(), - "KEYPAIR_JWT", - ) - .body(generate_s3_file_name(self.s3_path.as_deref(), file_suffix)); - - let response = builder - .send() - .await - .map_err(|err| SinkError::Snowflake(anyhow!(err)))?; - - if response.status() != StatusCode::OK { - return Err(SinkError::Snowflake(anyhow!( - "failed to make http request, error code: {}\ndetailed response: {:#?}", - response.status(), - response, - ))); - } - - Ok(()) - } -} - -/// todo: refactor this part after s3 sink is available -pub struct SnowflakeS3Client { - #[expect(dead_code)] - s3_bucket: String, - s3_path: Option, - pub opendal_s3_engine: OpendalObjectStore, -} - -impl SnowflakeS3Client { - pub fn new( - s3_bucket: String, - s3_path: Option, - aws_access_key_id: String, - aws_secret_access_key: String, - aws_region: String, - ) -> Result { - // FIXME: we should use the `ObjectStoreConfig` instead of default - // just use default configuration here for opendal s3 engine - let config = ObjectStoreConfig::default(); - let metrics = Arc::new(GLOBAL_OBJECT_STORE_METRICS.clone()); - - // create the s3 engine for streaming upload to the intermediate s3 bucket - let opendal_s3_engine = OpendalObjectStore::new_s3_engine_with_credentials( - &s3_bucket, - Arc::new(config), - metrics, - &aws_access_key_id, - &aws_secret_access_key, - &aws_region, - ) - .context("failed to create opendal s3 engine") - .map_err(SinkError::Snowflake)?; - - Ok(Self { - s3_bucket, - s3_path, - opendal_s3_engine, - }) - } - - pub fn s3_path(&self) -> Option<&str> { - self.s3_path.as_deref() - } -} diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index 27362900dd054..653acaadaaaf1 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -749,54 +749,36 @@ RedisConfig: required: true SnowflakeConfig: fields: - - name: snowflake.database - field_type: String - comments: The snowflake database used for sinking - required: true - - name: snowflake.schema - field_type: String - comments: The corresponding schema where sink table exists - required: true - - name: snowflake.pipe - field_type: String - comments: The created pipe object, will be used as `insertFiles` target - required: true - - name: snowflake.account_identifier - field_type: String - comments: 'The unique, snowflake provided `account_identifier` NOTE: please use the form `-` For detailed guidance, reference: ' - required: true - - name: snowflake.user - field_type: String - comments: 'The user that owns the table to be sinked NOTE: the user should''ve been granted corresponding *role* reference: ' - required: true - - name: snowflake.rsa_public_key_fp - field_type: String - comments: 'The public key fingerprint used when generating custom `jwt_token` reference: ' - required: true - - name: snowflake.private_key - field_type: String - comments: The rsa pem key *without* encryption - required: true - name: snowflake.s3_bucket field_type: String comments: The s3 bucket where intermediate sink files will be stored required: true + alias: + - s3.bucket_name - name: snowflake.s3_path field_type: String comments: The optional s3 path to be specified the actual file location would be `s3:////` if this field is specified by user(s) otherwise it would be `s3:///` required: false + alias: + - s3.path - name: snowflake.aws_access_key_id field_type: String comments: s3 credentials required: true + alias: + - s3.credentials.access - name: snowflake.aws_secret_access_key field_type: String comments: s3 credentials required: true + alias: + - s3.credentials.secret - name: snowflake.aws_region field_type: String comments: The s3 region, e.g., us-east-2 required: true + alias: + - s3.region_name SqlServerConfig: fields: - name: sqlserver.host diff --git a/src/object_store/src/object/opendal_engine/opendal_s3.rs b/src/object_store/src/object/opendal_engine/opendal_s3.rs index 5ba90ad93ccba..183496d08673a 100644 --- a/src/object_store/src/object/opendal_engine/opendal_s3.rs +++ b/src/object_store/src/object/opendal_engine/opendal_s3.rs @@ -135,6 +135,7 @@ impl OpendalObjectStore { builder.access_key_id(aws_access_key_id); builder.secret_access_key(aws_secret_access_key); builder.region(aws_region); + builder.disable_config_load(); let http_client = Self::new_http_client(config.as_ref())?; builder.http_client(http_client); From 5aa2d80a98e2a5259b5df2d1cd7bafbc51ba91cc Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 10 Jul 2024 11:26:46 +0800 Subject: [PATCH 08/20] doc(dev-guide): update link in README to the site (#17638) Signed-off-by: xxchan --- README.md | 2 +- docs/dev/src/intro.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index 156b53060313d..e08bb45d47c92 100644 --- a/README.md +++ b/README.md @@ -150,4 +150,4 @@ RisingWave is distributed under the Apache License (Version 2.0). Please refer t ## Contributing -Thanks for your interest in contributing to the project! Please refer to [contribution guidelines](CONTRIBUTING.md) for more information. +Thanks for your interest in contributing to the project! Please refer to [RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/)for more information. diff --git a/docs/dev/src/intro.md b/docs/dev/src/intro.md index c4317c8d71dd3..011ad4a149f83 100644 --- a/docs/dev/src/intro.md +++ b/docs/dev/src/intro.md @@ -1,6 +1,6 @@ # Introduction -This guide is intended to be used by contributors to learn about how to develop RisingWave. The instructions about how to submit code changes are included in [contributing guidelines](./contributing.md). +This guide is intended to be used by contributors to learn about how to develop RisingWave. The instructions about how to submit code changes are included in [contribution guidelines](./contributing.md). If you have questions, you can search for existing discussions or start a new discussion in the [Discussions forum of RisingWave](https://github.com/risingwavelabs/risingwave/discussions), or ask in the RisingWave Community channel on Slack. Please use the [invitation link](https://risingwave.com/slack) to join the channel. From b9ad21b3f0edd8a478d5c199967f01d088704e7f Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 10 Jul 2024 12:12:51 +0800 Subject: [PATCH 09/20] chore(storage): unify opendal version (#17639) --- Cargo.toml | 1 + src/batch/Cargo.toml | 2 +- src/connector/Cargo.toml | 2 +- src/object_store/Cargo.toml | 2 +- 4 files changed, 4 insertions(+), 3 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index 57fda38e8629d..45bf33f25f889 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -143,6 +143,7 @@ arrow-cast-iceberg = { package = "arrow-cast", version = "52" } # After apache/iceberg-rust#411 is merged, we move to the upstream version. iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "0c6e133e6f4655ff9ce4ad57b577dc7f692dd902" } iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "0c6e133e6f4655ff9ce4ad57b577dc7f692dd902" } +opendal = "0.47" arrow-array = "50" arrow-arith = "50" arrow-cast = "50" diff --git a/src/batch/Cargo.toml b/src/batch/Cargo.toml index c3042346aff85..ec7091ea882c4 100644 --- a/src/batch/Cargo.toml +++ b/src/batch/Cargo.toml @@ -32,7 +32,7 @@ hytra = "0.1.2" iceberg = { workspace = true } itertools = { workspace = true } memcomparable = "0.2" -opendal = "0.47" +opendal = { workspace = true } parking_lot = { workspace = true } parquet = { workspace = true } paste = "1" diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 2c8be26c05b33..bccd77faf8905 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -84,7 +84,7 @@ mysql_common = { version = "0.32", default-features = false, features = [ ] } nexmark = { version = "0.2", features = ["serde"] } num-bigint = "0.4" -opendal = { version = "0.47", features = [ +opendal = { workspace = true, features = [ "executors-tokio", "services-fs", "services-gcs", diff --git a/src/object_store/Cargo.toml b/src/object_store/Cargo.toml index ec03b6fe74614..e821c2fc85090 100644 --- a/src/object_store/Cargo.toml +++ b/src/object_store/Cargo.toml @@ -31,7 +31,7 @@ hyper-rustls = { version = "0.24.2", features = ["webpki-roots"] } hyper-tls = "0.5.0" itertools = { workspace = true } madsim = "0.2.27" -opendal = { version = "0.47", features = [ +opendal = { workspace = true, features = [ "executors-tokio", "services-azblob", "services-fs", From 96498a4ba00f14dab5f3ec7bf08e02d6ab969042 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Wed, 10 Jul 2024 12:55:09 +0800 Subject: [PATCH 10/20] refactor(optimizer): unify watermark derivation and monotonicity derivation (#17554) Signed-off-by: Richard Chien --- .../testdata/input/generated_columns.yaml | 5 + .../testdata/output/generated_columns.yaml | 11 + src/frontend/src/expr/mod.rs | 8 + src/frontend/src/expr/utils.rs | 185 ------------ .../src/optimizer/plan_node/stream_project.rs | 16 +- .../optimizer/plan_node/stream_project_set.rs | 14 +- src/frontend/src/optimizer/property/mod.rs | 2 + .../src/optimizer/property/monotonicity.rs | 273 ++++++++++++++++++ .../stream/filter_with_now_to_join_rule.rs | 26 +- 9 files changed, 329 insertions(+), 211 deletions(-) create mode 100644 src/frontend/src/optimizer/property/monotonicity.rs diff --git a/src/frontend/planner_test/tests/testdata/input/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/input/generated_columns.yaml index bd6a8f453ab71..7078f6e797342 100644 --- a/src/frontend/planner_test/tests/testdata/input/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/input/generated_columns.yaml @@ -14,6 +14,11 @@ select proctime(); expected_outputs: - binder_error +- name: proctime cast to with timezone + sql: | + explain create table t1 (proc_time TIMESTAMPTZ AS proctime()); + expected_outputs: + - explain_output - name: proctime cast to without timezone sql: | explain create table t1 (proc_time TIMESTAMP AS proctime()); diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index 3ed95c1ac1463..92f6aaa47e175 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -26,6 +26,17 @@ Caused by: Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? +- name: proctime cast to with timezone + sql: | + explain create table t1 (proc_time TIMESTAMPTZ AS proctime()); + explain_output: | + StreamMaterialize { columns: [proc_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite, watermark_columns: [proc_time] } + └─StreamRowIdGen { row_id_index: 1 } + └─StreamUnion { all: true, output_watermarks: [$expr1] } + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamProject { exprs: [Proctime as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamDml { columns: [_row_id] } + └─StreamSource - name: proctime cast to without timezone sql: | explain create table t1 (proc_time TIMESTAMP AS proctime()); diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 89142d0e9b237..19ffbaed92e28 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -94,6 +94,14 @@ macro_rules! impl_expr_impl { $($t(Box<$t>),)* } + impl ExprImpl { + pub fn variant_name(&self) -> &'static str { + match self { + $(ExprImpl::$t(_) => stringify!($t),)* + } + } + } + $( impl From<$t> for ExprImpl { fn from(o: $t) -> ExprImpl { diff --git a/src/frontend/src/expr/utils.rs b/src/frontend/src/expr/utils.rs index cc49f3c215378..b5f68fed4dfbe 100644 --- a/src/frontend/src/expr/utils.rs +++ b/src/frontend/src/expr/utils.rs @@ -508,191 +508,6 @@ pub fn rewrite_now_to_proctime(expr: ExprImpl) -> ExprImpl { r.rewrite_expr(expr) } -/// analyze if the expression can derive a watermark from some input watermark. If it can -/// derive, return the input watermark column index -pub fn try_derive_watermark(expr: &ExprImpl) -> WatermarkDerivation { - let a = WatermarkAnalyzer {}; - a.visit_expr(expr) -} - -#[derive(Clone, Copy, Debug, PartialEq)] -pub enum WatermarkDerivation { - /// The expression will return a constant and not depends on its input. - Constant, - /// Can derive a watermark if an input column has watermark, the usize field is the input - /// column index. - Watermark(usize), - /// For nondecreasing functions, we can always produce watermarks from where they are called. - Nondecreasing, - /// Can not derive a watermark in any cases. - None, -} - -#[derive(Clone, Default)] -struct WatermarkAnalyzer {} - -impl WatermarkAnalyzer { - fn visit_expr(&self, expr: &ExprImpl) -> WatermarkDerivation { - match expr { - ExprImpl::InputRef(inner) => WatermarkDerivation::Watermark(inner.index()), - ExprImpl::Literal(_) => WatermarkDerivation::Constant, - ExprImpl::FunctionCall(inner) => self.visit_function_call(inner), - ExprImpl::FunctionCallWithLambda(inner) => self.visit_function_call(inner.base()), - ExprImpl::TableFunction(_) => WatermarkDerivation::None, - ExprImpl::Subquery(_) - | ExprImpl::AggCall(_) - | ExprImpl::CorrelatedInputRef(_) - | ExprImpl::WindowFunction(_) - | ExprImpl::Parameter(_) - | ExprImpl::Now(_) => unreachable!(), - ExprImpl::UserDefinedFunction(_) => WatermarkDerivation::None, - } - } - - fn visit_unary_op(&self, inputs: &[ExprImpl]) -> WatermarkDerivation { - assert_eq!(inputs.len(), 1); - self.visit_expr(&inputs[0]) - } - - fn visit_binary_op(&self, inputs: &[ExprImpl]) -> (WatermarkDerivation, WatermarkDerivation) { - assert_eq!(inputs.len(), 2); - (self.visit_expr(&inputs[0]), self.visit_expr(&inputs[1])) - } - - fn visit_ternary_op( - &self, - inputs: &[ExprImpl], - ) -> ( - WatermarkDerivation, - WatermarkDerivation, - WatermarkDerivation, - ) { - assert_eq!(inputs.len(), 3); - ( - self.visit_expr(&inputs[0]), - self.visit_expr(&inputs[1]), - self.visit_expr(&inputs[2]), - ) - } - - fn visit_function_call(&self, func_call: &FunctionCall) -> WatermarkDerivation { - use WatermarkDerivation::{Constant, Nondecreasing, Watermark}; - match func_call.func_type() { - ExprType::Unspecified => unreachable!(), - ExprType::Add => match self.visit_binary_op(func_call.inputs()) { - (Constant, Constant) => Constant, - (Constant, Watermark(idx)) | (Watermark(idx), Constant) => Watermark(idx), - (Constant, Nondecreasing) | (Nondecreasing, Constant) => Nondecreasing, - _ => WatermarkDerivation::None, - }, - ExprType::Subtract | ExprType::TumbleStart => { - if func_call.inputs().len() == 3 { - // With `offset` specified - // e.g., select * from tumble(t1, start, interval, offset); - assert_eq!(ExprType::TumbleStart, func_call.func_type()); - match self.visit_ternary_op(func_call.inputs()) { - (Constant, Constant, Constant) => Constant, - (Watermark(idx), Constant, Constant) => Watermark(idx), - (Nondecreasing, Constant, Constant) => Nondecreasing, - _ => WatermarkDerivation::None, - } - } else { - match self.visit_binary_op(func_call.inputs()) { - (Constant, Constant) => Constant, - (Watermark(idx), Constant) => Watermark(idx), - (Nondecreasing, Constant) => Nondecreasing, - _ => WatermarkDerivation::None, - } - } - } - ExprType::Multiply | ExprType::Divide | ExprType::Modulus => { - match self.visit_binary_op(func_call.inputs()) { - (Constant, Constant) => Constant, - // not meaningful to derive watermark for other situations - _ => WatermarkDerivation::None, - } - } - ExprType::AtTimeZone => match self.visit_binary_op(func_call.inputs()) { - (Constant, Constant) => Constant, - (derivation @ (Watermark(_) | Nondecreasing), Constant) => { - if !(func_call.return_type() == DataType::Timestamptz - && func_call.inputs()[0].return_type() == DataType::Timestamp) - && func_call.inputs()[1] - .as_literal() - .and_then(|literal| literal.get_data().as_ref()) - .map_or(true, |time_zone| { - !time_zone.as_utf8().eq_ignore_ascii_case("UTC") - }) - { - WatermarkDerivation::None - } else { - derivation - } - } - _ => WatermarkDerivation::None, - }, - ExprType::AddWithTimeZone | ExprType::SubtractWithTimeZone => { - // Requires time zone and interval to be literal, at least for now. - let time_zone = match &func_call.inputs()[2] { - ExprImpl::Literal(lit) => lit.get_data().as_ref().map(|s| s.as_utf8()), - _ => return WatermarkDerivation::None, - }; - let interval = match &func_call.inputs()[1] { - ExprImpl::Literal(lit) => lit.get_data().as_ref().map(|s| s.as_interval()), - _ => return WatermarkDerivation::None, - }; - // null zone or null interval is treated same as const `interval '1' second`, to be - // consistent with other match arms. - let zone_without_dst = time_zone.map_or(true, |s| s.eq_ignore_ascii_case("UTC")); - let quantitative_only = interval.map_or(true, |v| { - v.months() == 0 && (v.days() == 0 || zone_without_dst) - }); - match (self.visit_expr(&func_call.inputs()[0]), quantitative_only) { - (Constant, _) => Constant, - (Watermark(idx), true) => Watermark(idx), - (Nondecreasing, true) => Nondecreasing, - (Watermark(_) | Nondecreasing, false) => WatermarkDerivation::None, - (WatermarkDerivation::None, _) => WatermarkDerivation::None, - } - } - ExprType::DateTrunc => match func_call.inputs().len() { - 2 => match self.visit_binary_op(func_call.inputs()) { - (Constant, any_derivation) => any_derivation, - _ => WatermarkDerivation::None, - }, - 3 => match self.visit_ternary_op(func_call.inputs()) { - (Constant, Constant, Constant) => Constant, - (Constant, derivation @ (Watermark(_) | Nondecreasing), Constant) => { - let zone_without_dst = func_call.inputs()[2] - .as_literal() - .and_then(|literal| literal.get_data().as_ref()) - .map_or(false, |s| s.as_utf8().eq_ignore_ascii_case("UTC")); - if zone_without_dst { - derivation - } else { - WatermarkDerivation::None - } - } - _ => WatermarkDerivation::None, - }, - _ => unreachable!(), - }, - ExprType::SecToTimestamptz => self.visit_unary_op(func_call.inputs()), - ExprType::CharToTimestamptz => WatermarkDerivation::None, - ExprType::Cast => { - // TODO: need more derivation - WatermarkDerivation::None - } - ExprType::Case => { - // TODO: do we need derive watermark when every case can derive a common watermark? - WatermarkDerivation::None - } - ExprType::Proctime => Nondecreasing, - _ => WatermarkDerivation::None, - } - } -} - #[cfg(test)] mod tests { use risingwave_common::types::{DataType, ScalarImpl}; diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index e8ff1df6e82db..eae1bd5a34d5e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -20,10 +20,9 @@ use risingwave_pb::stream_plan::ProjectNode; use super::stream::prelude::*; use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::{ - try_derive_watermark, Expr, ExprImpl, ExprRewriter, ExprVisitor, WatermarkDerivation, -}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::{analyze_monotonicity, monotonicity_variants}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::ColIndexMappingRewriteExt; @@ -84,21 +83,22 @@ impl StreamProject { let mut nondecreasing_exprs = vec![]; let mut watermark_columns = FixedBitSet::with_capacity(core.exprs.len()); for (expr_idx, expr) in core.exprs.iter().enumerate() { - match try_derive_watermark(expr) { - WatermarkDerivation::Watermark(input_idx) => { + use monotonicity_variants::*; + match analyze_monotonicity(expr) { + FollowingInput(input_idx) => { if input.watermark_columns().contains(input_idx) { watermark_derivations.push((input_idx, expr_idx)); watermark_columns.insert(expr_idx); } } - WatermarkDerivation::Nondecreasing => { + Inherent(NonDecreasing) => { nondecreasing_exprs.push(expr_idx); watermark_columns.insert(expr_idx); } - WatermarkDerivation::Constant => { + Inherent(Constant) => { // XXX(rc): we can produce one watermark on each recovery for this case. } - WatermarkDerivation::None => {} + Inherent(_) | _FollowingInputInversely(_) => {} } } // Project executor won't change the append-only behavior of the stream, so it depends on diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index 2af5d54234363..b65d4e8da0b53 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -20,8 +20,9 @@ use risingwave_pb::stream_plan::ProjectSetNode; use super::stream::prelude::*; use super::utils::impl_distill_by_unit; use super::{generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::{try_derive_watermark, ExprRewriter, ExprVisitor, WatermarkDerivation}; +use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; +use crate::optimizer::property::{analyze_monotonicity, monotonicity_variants}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::ColIndexMappingRewriteExt; @@ -48,21 +49,22 @@ impl StreamProjectSet { let mut nondecreasing_exprs = vec![]; let mut watermark_columns = FixedBitSet::with_capacity(core.output_len()); for (expr_idx, expr) in core.select_list.iter().enumerate() { - match try_derive_watermark(expr) { - WatermarkDerivation::Watermark(input_idx) => { + use monotonicity_variants::*; + match analyze_monotonicity(expr) { + FollowingInput(input_idx) => { if input.watermark_columns().contains(input_idx) { watermark_derivations.push((input_idx, expr_idx)); watermark_columns.insert(expr_idx + 1); } } - WatermarkDerivation::Nondecreasing => { + Inherent(NonDecreasing) => { nondecreasing_exprs.push(expr_idx); watermark_columns.insert(expr_idx + 1); } - WatermarkDerivation::Constant => { + Inherent(Constant) => { // XXX(rc): we can produce one watermark on each recovery for this case. } - WatermarkDerivation::None => {} + Inherent(_) | _FollowingInputInversely(_) => {} } } diff --git a/src/frontend/src/optimizer/property/mod.rs b/src/frontend/src/optimizer/property/mod.rs index 69871d0003596..ae6ebe7e8288a 100644 --- a/src/frontend/src/optimizer/property/mod.rs +++ b/src/frontend/src/optimizer/property/mod.rs @@ -32,3 +32,5 @@ mod func_dep; pub use func_dep::*; mod cardinality; pub use cardinality::*; +mod monotonicity; +pub use monotonicity::*; diff --git a/src/frontend/src/optimizer/property/monotonicity.rs b/src/frontend/src/optimizer/property/monotonicity.rs new file mode 100644 index 0000000000000..87f74c25b83f4 --- /dev/null +++ b/src/frontend/src/optimizer/property/monotonicity.rs @@ -0,0 +1,273 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use enum_as_inner::EnumAsInner; +use risingwave_common::types::DataType; +use risingwave_pb::expr::expr_node::Type as ExprType; + +use crate::expr::{Expr, ExprImpl, FunctionCall, TableFunction}; + +/// Represents the derivation of the monotonicity of a column. +/// This enum aims to unify the "non-decreasing analysis" and watermark derivation. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EnumAsInner)] +pub enum MonotonicityDerivation { + /// The monotonicity of the column is inherent, meaning that it is derived from the column itself. + Inherent(Monotonicity), + /// The monotonicity of the column follows the monotonicity of the specified column in the input. + FollowingInput(usize), + /// The monotonicity of the column INVERSELY follows the monotonicity of the specified column in the input. + /// This is not used currently. + _FollowingInputInversely(usize), +} + +impl MonotonicityDerivation { + pub fn inverse(self) -> Self { + use MonotonicityDerivation::*; + match self { + Inherent(monotonicity) => Inherent(monotonicity.inverse()), + FollowingInput(idx) => _FollowingInputInversely(idx), + _FollowingInputInversely(idx) => FollowingInput(idx), + } + } +} + +/// Represents the monotonicity of a column. `NULL`s are considered largest when analyzing monotonicity. +#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, EnumAsInner)] +pub enum Monotonicity { + Constant, + NonDecreasing, + NonIncreasing, + Unknown, +} + +impl Monotonicity { + pub fn inverse(self) -> Self { + use Monotonicity::*; + match self { + Constant => Constant, + NonDecreasing => NonIncreasing, + NonIncreasing => NonDecreasing, + Unknown => Unknown, + } + } +} + +pub mod monotonicity_variants { + pub use super::Monotonicity::*; + pub use super::MonotonicityDerivation::*; +} + +/// Analyze the monotonicity of an expression. +pub fn analyze_monotonicity(expr: &ExprImpl) -> MonotonicityDerivation { + let analyzer = MonotonicityAnalyzer {}; + analyzer.visit_expr(expr) +} + +struct MonotonicityAnalyzer {} + +impl MonotonicityAnalyzer { + fn visit_expr(&self, expr: &ExprImpl) -> MonotonicityDerivation { + use monotonicity_variants::*; + match expr { + // recursion base + ExprImpl::InputRef(inner) => FollowingInput(inner.index()), + ExprImpl::Literal(_) => Inherent(Constant), + ExprImpl::Now(_) => Inherent(NonDecreasing), + ExprImpl::UserDefinedFunction(_) => Inherent(Unknown), + + // recursively visit children + ExprImpl::FunctionCall(inner) => self.visit_function_call(inner), + ExprImpl::FunctionCallWithLambda(inner) => self.visit_function_call(inner.base()), + ExprImpl::TableFunction(inner) => self.visit_table_function(inner), + + // the analyzer is not expected to be used when the following expression types are present + ExprImpl::Subquery(_) + | ExprImpl::AggCall(_) + | ExprImpl::CorrelatedInputRef(_) + | ExprImpl::WindowFunction(_) + | ExprImpl::Parameter(_) => panic!( + "Expression `{}` is not expected in the monotonicity analyzer", + expr.variant_name() + ), + } + } + + fn visit_unary_op(&self, inputs: &[ExprImpl]) -> MonotonicityDerivation { + assert_eq!(inputs.len(), 1); + self.visit_expr(&inputs[0]) + } + + fn visit_binary_op( + &self, + inputs: &[ExprImpl], + ) -> (MonotonicityDerivation, MonotonicityDerivation) { + assert_eq!(inputs.len(), 2); + (self.visit_expr(&inputs[0]), self.visit_expr(&inputs[1])) + } + + fn visit_ternary_op( + &self, + inputs: &[ExprImpl], + ) -> ( + MonotonicityDerivation, + MonotonicityDerivation, + MonotonicityDerivation, + ) { + assert_eq!(inputs.len(), 3); + ( + self.visit_expr(&inputs[0]), + self.visit_expr(&inputs[1]), + self.visit_expr(&inputs[2]), + ) + } + + fn visit_function_call(&self, func_call: &FunctionCall) -> MonotonicityDerivation { + use monotonicity_variants::*; + + fn time_zone_is_without_dst(time_zone: Option<&str>) -> bool { + #[allow(clippy::let_and_return)] // to make it more readable + let tz_is_utc = time_zone.map_or( + false, // conservative + |time_zone| time_zone.eq_ignore_ascii_case("UTC"), + ); + tz_is_utc // conservative + } + + match func_call.func_type() { + ExprType::Unspecified => unreachable!(), + ExprType::Add => match self.visit_binary_op(func_call.inputs()) { + (Inherent(Constant), any) | (any, Inherent(Constant)) => any, + (Inherent(NonDecreasing), Inherent(NonDecreasing)) => Inherent(NonDecreasing), + (Inherent(NonIncreasing), Inherent(NonIncreasing)) => Inherent(NonIncreasing), + _ => Inherent(Unknown), + }, + ExprType::Subtract => match self.visit_binary_op(func_call.inputs()) { + (any, Inherent(Constant)) => any, + (Inherent(Constant), any) => any.inverse(), + _ => Inherent(Unknown), + }, + ExprType::Multiply | ExprType::Divide | ExprType::Modulus => { + match self.visit_binary_op(func_call.inputs()) { + (Inherent(Constant), Inherent(Constant)) => Inherent(Constant), + _ => Inherent(Unknown), // let's be lazy here + } + } + ExprType::TumbleStart => { + if func_call.inputs().len() == 2 { + // without `offset`, args: `(start, interval)` + match self.visit_binary_op(func_call.inputs()) { + (any, Inherent(Constant)) => any, + _ => Inherent(Unknown), + } + } else { + // with `offset`, args: `(start, interval, offset)` + assert_eq!(ExprType::TumbleStart, func_call.func_type()); + match self.visit_ternary_op(func_call.inputs()) { + (any, Inherent(Constant), Inherent(Constant)) => any, + _ => Inherent(Unknown), + } + } + } + ExprType::AtTimeZone => match self.visit_binary_op(func_call.inputs()) { + (Inherent(Constant), Inherent(Constant)) => Inherent(Constant), + (any, Inherent(Constant)) => { + let time_zone = func_call.inputs()[1] + .as_literal() + .and_then(|literal| literal.get_data().as_ref()) + .map(|tz| tz.as_utf8().as_ref()); + // 1. For at_time_zone(timestamp, const timezone) -> timestamptz, when timestamp has some monotonicity, + // the result should have the same monotonicity. + // 2. For at_time_zone(timestamptz, const timezone) -> timestamp, when timestamptz has some monotonicity, + // the result only have the same monotonicity when the timezone is without DST (Daylight Saving Time). + if (func_call.inputs()[0].return_type() == DataType::Timestamp + && func_call.return_type() == DataType::Timestamptz) + || time_zone_is_without_dst(time_zone) + { + any + } else { + Inherent(Unknown) + } + } + _ => Inherent(Unknown), + }, + ExprType::DateTrunc => match func_call.inputs().len() { + 2 => match self.visit_binary_op(func_call.inputs()) { + (Inherent(Constant), any) => any, + _ => Inherent(Unknown), + }, + 3 => match self.visit_ternary_op(func_call.inputs()) { + (Inherent(Constant), Inherent(Constant), Inherent(Constant)) => { + Inherent(Constant) + } + (Inherent(Constant), any, Inherent(Constant)) => { + let time_zone = func_call.inputs()[2] + .as_literal() + .and_then(|literal| literal.get_data().as_ref()) + .map(|tz| tz.as_utf8().as_ref()); + if time_zone_is_without_dst(time_zone) { + any + } else { + Inherent(Unknown) + } + } + _ => Inherent(Unknown), + }, + _ => unreachable!(), + }, + ExprType::AddWithTimeZone | ExprType::SubtractWithTimeZone => { + // Requires time zone and interval to be literal, at least for now. + let time_zone = match &func_call.inputs()[2] { + ExprImpl::Literal(lit) => { + lit.get_data().as_ref().map(|tz| tz.as_utf8().as_ref()) + } + _ => return Inherent(Unknown), + }; + let interval = match &func_call.inputs()[1] { + ExprImpl::Literal(lit) => lit + .get_data() + .as_ref() + .map(|interval| interval.as_interval()), + _ => return Inherent(Unknown), + }; + let quantitative_only = interval.map_or( + true, // null interval is treated as `interval '1' second` + |v| v.months() == 0 && (v.days() == 0 || time_zone_is_without_dst(time_zone)), + ); + match (self.visit_expr(&func_call.inputs()[0]), quantitative_only) { + (Inherent(Constant), _) => Inherent(Constant), + (any, true) => any, + _ => Inherent(Unknown), + } + } + ExprType::SecToTimestamptz => self.visit_unary_op(func_call.inputs()), + ExprType::CharToTimestamptz => Inherent(Unknown), + ExprType::Cast => { + // TODO: need more derivation + Inherent(Unknown) + } + ExprType::Case => { + // TODO: do we need derive watermark when every case can derive a common watermark? + Inherent(Unknown) + } + ExprType::Proctime => Inherent(NonDecreasing), + _ => Inherent(Unknown), + } + } + + fn visit_table_function(&self, _table_func: &TableFunction) -> MonotonicityDerivation { + // TODO: derive monotonicity for table funcs like `generate_series` + use monotonicity_variants::*; + Inherent(Unknown) + } +} diff --git a/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs b/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs index cbdb65b4528a5..39dc88319816d 100644 --- a/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs +++ b/src/frontend/src/optimizer/rule/stream/filter_with_now_to_join_rule.rs @@ -15,11 +15,10 @@ use risingwave_common::types::DataType; use risingwave_pb::plan_common::JoinType; -use crate::expr::{ - try_derive_watermark, ExprRewriter, FunctionCall, InputRef, WatermarkDerivation, -}; +use crate::expr::{ExprRewriter, FunctionCall, InputRef}; use crate::optimizer::plan_node::generic::{self, GenericPlanRef}; use crate::optimizer::plan_node::{LogicalFilter, LogicalJoin, LogicalNow}; +use crate::optimizer::property::{analyze_monotonicity, monotonicity_variants}; use crate::optimizer::rule::{BoxedRule, Rule}; use crate::optimizer::PlanRef; use crate::utils::Condition; @@ -36,18 +35,19 @@ impl Rule for FilterWithNowToJoinRule { let mut now_filters = vec![]; let mut remainder = vec![]; - let mut rewriter = NowAsInputRef::new(lhs_len); - // If the `now` is not a valid dynamic filter expression, we will not push it down. filter.predicate().conjunctions.iter().for_each(|expr| { if let Some((input_expr, cmp, now_expr)) = expr.as_now_comparison_cond() { - let now_expr = rewriter.rewrite_expr(now_expr); - - // ensure that this expression will derive a watermark - if try_derive_watermark(&now_expr) != WatermarkDerivation::Watermark(lhs_len) { - remainder.push(expr.clone()); + // ensure that this expression is increasing + use monotonicity_variants::*; + if matches!(analyze_monotonicity(&now_expr), Inherent(NonDecreasing)) { + now_filters.push( + FunctionCall::new(cmp, vec![input_expr, now_expr]) + .unwrap() + .into(), + ); } else { - now_filters.push(FunctionCall::new(cmp, vec![input_expr, now_expr]).unwrap()); + remainder.push(expr.clone()); } } else { remainder.push(expr.clone()); @@ -60,13 +60,15 @@ impl Rule for FilterWithNowToJoinRule { } let mut new_plan = plan.inputs()[0].clone(); + let mut rewriter = NowAsInputRef::new(lhs_len); for now_filter in now_filters { + let now_filter = rewriter.rewrite_expr(now_filter); new_plan = LogicalJoin::new( new_plan, LogicalNow::new(generic::Now::update_current(plan.ctx())).into(), JoinType::LeftSemi, Condition { - conjunctions: vec![now_filter.into()], + conjunctions: vec![now_filter], }, ) .into() From b8bb2f7ab564a6a6a75d0c939a14b09f4869fb76 Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 10 Jul 2024 13:31:59 +0800 Subject: [PATCH 11/20] doc(dev-guide): merge design docs into dev guide (#17640) Signed-off-by: xxchan --- .pre-commit-config.yaml | 8 +-- docs/README.md | 24 +-------- docs/dev/README.md | 4 ++ docs/dev/src/SUMMARY.md | 25 ++++++++- docs/{ => dev/src/design}/aggregation.md | 6 +-- .../src/design}/architecture-design.md | 8 +-- docs/{ => dev/src/design}/backfill.md | 16 +++--- .../src/design}/batch-local-execution-mode.md | 6 +-- docs/{ => dev/src/design}/checkpoint.md | 4 +- docs/{ => dev/src/design}/consistent-hash.md | 14 ++--- .../src/design}/data-model-and-encoding.md | 7 +-- docs/{ => dev/src/design}/data-source.md | 2 +- docs/{ => dev/src/design}/keys.md | 2 +- docs/{ => dev/src/design}/meta-service.md | 5 +- .../src/design}/multi-object-store.md | 2 +- docs/{ => dev/src/design}/mv-on-mv.md | 4 +- .../src/design/relational-table.md} | 50 +++++++++++++----- docs/{ => dev/src/design}/shared-buffer.md | 2 +- .../src/design}/state-store-overview.md | 10 ++-- .../src/design}/streaming-overview.md | 5 +- .../images/aggregation/agg-components.png | Bin .../images/aggregation/init-agg-group.png | Bin .../architecture-design/architecture.svg | 0 .../architecture-design/batch-query.svg | 0 .../architecture-design/plan-fragments.svg | 0 .../architecture-design/stream-pipeline.png | Bin .../src}/images/backfill/backfill-sides.png | Bin .../src}/images/backfill/handle-poll.png | Bin .../{ => dev/src}/images/backfill/polling.png | Bin .../images/backfill/replication-example.png | Bin .../backfill/replication-replicated.png | Bin .../images/backfill/replication-simple.png | Bin docs/{ => dev/src}/images/backfill/schema.png | Bin .../batch-local-execution-mode/example1.svg | 0 .../batch-local-execution-mode/example2.svg | 0 .../frontend-flow.svg | 0 .../src}/images/checkpoint/checkpoint.svg | 0 .../src}/images/checkpoint/shared-buffer.svg | 0 .../images/consistent-hash/actor-data.svg | 0 .../consistent-hash/actor-state-table.svg | 0 .../consistent-hash/data-distribution.svg | 0 .../consistent-hash/data-redistribution-1.svg | 0 .../consistent-hash/data-redistribution-2.svg | 0 .../consistent-hash/storage-data-layout.svg | 0 .../images/data-model-and-encoding/chunk.svg | 0 .../data-model-and-encoding/row-format.svg | 0 .../images/data-source/data-source-arch.svg | 0 docs/{ => dev/src}/images/logo-title.svg | 0 docs/{ => dev/src}/images/logo.svg | 0 .../meta-service/cluster-deployment.svg | 0 .../src}/images/meta-service/notification.svg | 0 .../src}/images/mv-on-mv/mv-on-mv-01.svg | 0 .../src}/images/mv-on-mv/mv-on-mv-02.svg | 0 .../relational-table}/relational-table-01.svg | 0 .../relational-table}/relational-table-02.svg | 0 .../relational-table}/relational-table-03.svg | 0 .../state-store-overview-01.svg | 0 .../state-store-overview-02.svg | 0 .../state-store-overview-03.svg | 0 .../state-store-overview-04.svg | 0 .../state-store-overview-05.svg | 0 .../streaming-architecture.svg | 0 .../streaming-executor-and-compute-node.svg | 0 docs/{ => dev/src}/metrics.md | 2 +- .../relational-table-schema.md | 35 ------------ docs/rustdoc/README.md | 2 +- docs/rustdoc/index.md | 4 +- docs/rustdoc/rust.css | 44 +++++++++++++-- 68 files changed, 157 insertions(+), 134 deletions(-) rename docs/{ => dev/src/design}/aggregation.md (91%) rename docs/{ => dev/src/design}/architecture-design.md (95%) rename docs/{ => dev/src/design}/backfill.md (97%) rename docs/{ => dev/src/design}/batch-local-execution-mode.md (94%) rename docs/{ => dev/src/design}/checkpoint.md (97%) rename docs/{ => dev/src/design}/consistent-hash.md (92%) rename docs/{ => dev/src/design}/data-model-and-encoding.md (95%) rename docs/{ => dev/src/design}/data-source.md (98%) rename docs/{ => dev/src/design}/keys.md (99%) rename docs/{ => dev/src/design}/meta-service.md (95%) rename docs/{ => dev/src/design}/multi-object-store.md (99%) rename docs/{ => dev/src/design}/mv-on-mv.md (96%) rename docs/{relational_table/storing-state-using-relational-table.md => dev/src/design/relational-table.md} (64%) rename docs/{ => dev/src/design}/shared-buffer.md (99%) rename docs/{ => dev/src/design}/state-store-overview.md (96%) rename docs/{ => dev/src/design}/streaming-overview.md (97%) rename docs/{ => dev/src}/images/aggregation/agg-components.png (100%) rename docs/{ => dev/src}/images/aggregation/init-agg-group.png (100%) rename docs/{ => dev/src}/images/architecture-design/architecture.svg (100%) rename docs/{ => dev/src}/images/architecture-design/batch-query.svg (100%) rename docs/{ => dev/src}/images/architecture-design/plan-fragments.svg (100%) rename docs/{ => dev/src}/images/architecture-design/stream-pipeline.png (100%) rename docs/{ => dev/src}/images/backfill/backfill-sides.png (100%) rename docs/{ => dev/src}/images/backfill/handle-poll.png (100%) rename docs/{ => dev/src}/images/backfill/polling.png (100%) rename docs/{ => dev/src}/images/backfill/replication-example.png (100%) rename docs/{ => dev/src}/images/backfill/replication-replicated.png (100%) rename docs/{ => dev/src}/images/backfill/replication-simple.png (100%) rename docs/{ => dev/src}/images/backfill/schema.png (100%) rename docs/{ => dev/src}/images/batch-local-execution-mode/example1.svg (100%) rename docs/{ => dev/src}/images/batch-local-execution-mode/example2.svg (100%) rename docs/{ => dev/src}/images/batch-local-execution-mode/frontend-flow.svg (100%) rename docs/{ => dev/src}/images/checkpoint/checkpoint.svg (100%) rename docs/{ => dev/src}/images/checkpoint/shared-buffer.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/actor-data.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/actor-state-table.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/data-distribution.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/data-redistribution-1.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/data-redistribution-2.svg (100%) rename docs/{ => dev/src}/images/consistent-hash/storage-data-layout.svg (100%) rename docs/{ => dev/src}/images/data-model-and-encoding/chunk.svg (100%) rename docs/{ => dev/src}/images/data-model-and-encoding/row-format.svg (100%) rename docs/{ => dev/src}/images/data-source/data-source-arch.svg (100%) rename docs/{ => dev/src}/images/logo-title.svg (100%) rename docs/{ => dev/src}/images/logo.svg (100%) rename docs/{ => dev/src}/images/meta-service/cluster-deployment.svg (100%) rename docs/{ => dev/src}/images/meta-service/notification.svg (100%) rename docs/{ => dev/src}/images/mv-on-mv/mv-on-mv-01.svg (100%) rename docs/{ => dev/src}/images/mv-on-mv/mv-on-mv-02.svg (100%) rename docs/{images/relational-table-layer => dev/src/images/relational-table}/relational-table-01.svg (100%) rename docs/{images/relational-table-layer => dev/src/images/relational-table}/relational-table-02.svg (100%) rename docs/{images/relational-table-layer => dev/src/images/relational-table}/relational-table-03.svg (100%) rename docs/{ => dev/src}/images/state-store-overview/state-store-overview-01.svg (100%) rename docs/{ => dev/src}/images/state-store-overview/state-store-overview-02.svg (100%) rename docs/{ => dev/src}/images/state-store-overview/state-store-overview-03.svg (100%) rename docs/{ => dev/src}/images/state-store-overview/state-store-overview-04.svg (100%) rename docs/{ => dev/src}/images/state-store-overview/state-store-overview-05.svg (100%) rename docs/{ => dev/src}/images/streaming-overview/streaming-architecture.svg (100%) rename docs/{ => dev/src}/images/streaming-overview/streaming-executor-and-compute-node.svg (100%) rename docs/{ => dev/src}/metrics.md (98%) delete mode 100644 docs/relational_table/relational-table-schema.md diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index ab8ba3d9d7eb9..cb54c1606356e 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -7,6 +7,10 @@ repos: hooks: - id: end-of-file-fixer - id: trailing-whitespace +- repo: https://github.com/crate-ci/typos + rev: v1.23.1 + hooks: + - id: typos - repo: local hooks: - id: rustfmt @@ -14,10 +18,6 @@ repos: entry: rustfmt --edition 2021 language: system types: [rust] - - id: typos - name: typos - entry: typos -w - language: system - id: cargo sort name: cargo sort entry: cargo sort -g -w diff --git a/docs/README.md b/docs/README.md index e905cea7849ea..f371d9bda8f47 100644 --- a/docs/README.md +++ b/docs/README.md @@ -2,25 +2,5 @@ This directory contains RisingWave design documents that are intended to be used by contributors to understand our development process, and how we design and implement RisingWave. To learn about how to use RisingWave, check out the [RisingWave user documentation](https://www.risingwave.dev). -## Developer guide - -After you learn about the basics of RisingWave, take a look at our [developer guide](https://risingwavelabs.github.io/risingwave/) to get up to speed with the development process. - -## Table of Contents - -* [Architecture Design](./architecture-design.md) -* [An Overview of RisingWave Streaming Engine](./streaming-overview.md) -* [An Overview of RisingWave State Store](./state-store-overview.md) -* [Meta Service](./meta-service.md) -* [Create MView on Top of MView](./mv-on-mv.md) -* [Checkpoint](./checkpoint.md) -* [Design of Data Source](./data-source.md) -* [Data Model and Encoding](./data-model-and-encoding.md) -* [Design of Batch Local Execution Mode](./batch-local-execution-mode.md) -* [Consistent Hash](./consistent-hash.md) -* [Build RisingWave with Multiple Object Storage Backends](./multi-object-store.md) -* [Backfill](./backfill.md) - -## Images - -We recommend that you use [draw.io](https://app.diagrams.net/) to draw illustrations and export as SVG images, with "include a copy of my diagram" selected for further editing. +- `/dev` contains the source code for the [RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/) +- `/rustdoc` contains source code for the [crate level documentation](https://risingwavelabs.github.io/risingwave/rustdoc) diff --git a/docs/dev/README.md b/docs/dev/README.md index e19f10c08e3c3..7e47920d49400 100644 --- a/docs/dev/README.md +++ b/docs/dev/README.md @@ -28,3 +28,7 @@ including the `` marker at the place where you want the TOC. We use `mdbook-linkcheck` to validate URLs included in our documentation. `linkcheck` will be run automatically when you build with the instructions in the section above. + +## Images + +We recommend that you use [draw.io](https://app.diagrams.net/) to draw illustrations and export as SVG images, with "include a copy of my diagram" selected for further editing. diff --git a/docs/dev/src/SUMMARY.md b/docs/dev/src/SUMMARY.md index 76cf57c007c23..382cc0f80fec8 100644 --- a/docs/dev/src/SUMMARY.md +++ b/docs/dev/src/SUMMARY.md @@ -11,10 +11,11 @@ - [Testing](./tests/intro.md) - [Debugging](./debugging.md) - [Observability](./observability.md) + - [Metrics](./metrics.md) --- -# Benchmarking and Profiling +# Benchmarking and profiling - [CPU Profiling](./benchmark-and-profile/cpu-profiling.md) - [Memory (Heap) Profiling](./benchmark-and-profile/memory-profiling.md) @@ -27,6 +28,28 @@ - [Develop Connectors](./connector/intro.md) - [Continuous Integration](./ci.md) +--- + +# Design docs + + + +- [Architecture Design](./design/architecture-design.md) +- [An Overview of RisingWave Streaming Engine](./design/streaming-overview.md) +- [An Overview of RisingWave State Store](./design/state-store-overview.md) +- [Meta Service](./design/meta-service.md) +- [Create MView on Top of MView](./design/mv-on-mv.md) +- [Checkpoint](./design/checkpoint.md) +- [Design of Data Source](./design/data-source.md) +- [Data Model and Encoding](./design/data-model-and-encoding.md) +- [Design of Batch Local Execution Mode](./design/batch-local-execution-mode.md) +- [Consistent Hash](./design/consistent-hash.md) +- [Build RisingWave with Multiple Object Storage Backends](./design/multi-object-store.md) +- [Backfill](./design/backfill.md) +- [Aggregation](./design/aggregation.md) +- [Shared Buffer](./design/shared-buffer.md) +- [Relational Table](./design/relational-table.md) +- [Keys](./design/keys.md) + ## Row-based Encoding @@ -23,8 +15,6 @@ We implement a relational table layer as the bridge between executors and KV sta | join | table_id \| join_key \| pk | materialized value | | agg | table_id \| group_key | agg_value | -For the detailed schema, please check [doc](relational-table-schema.md) - ## Relational Table Layer [source code](https://github.com/risingwavelabs/risingwave/blob/4e66ca3d41435c64af26b5e0003258c4f7116822/src/storage/src/table/state_table.rs) @@ -36,13 +26,13 @@ Relational table layer consists of State Table, Mem Table and Storage Table. The State Table provides the table operations by these APIs: `get_row`, `scan`, `insert_row`, `delete_row` and `update_row`, which are the read and write interfaces for streaming executors. The Mem Table is an in-memory buffer for caching table operations during one epoch. The Storage Table is read only, and will output the partial columns upper level needs. -![Overview of Relational Table](../images/relational-table-layer/relational-table-01.svg) +![Overview of Relational Table](../images/relational-table/relational-table-01.svg) ### Write Path To write into KV state store, executors first perform operations on State Table, and these operations will be cached in Mem Table. Once a barrier flows through one executor, executor will flush the cached operations into state store. At this moment, State Table will covert these operations into kv pairs and write to state store with specific epoch. For example, an executor performs `insert(a, b, c)` and `delete(d, e, f)` through the State Table APIs, Mem Table first caches these two operations in memory. After receiving new barrier, State Table converts these two operations into KV operations by row-based format, and writes these KV operations into state store (Hummock). -![write example](../images/relational-table-layer/relational-table-03.svg) +![write example](../images/relational-table/relational-table-03.svg) ### Read Path In streaming mode, executors should be able to read the latest written data, which means uncommitted data is visible. The data in Mem Table (memory) is fresher than that in shared storage (state store). State Table provides both point-get and scan to read from state store by merging data from Mem Table and Storage Table. #### Get @@ -68,4 +58,36 @@ Get(pk = 3): [3, 3333, 3333] #### Scan Scan on relational table is implemented by `StateTableIter`, which is a merge iterator of `MemTableIter` and `StorageIter`. If a pk exists in both KV state store (shared storage) and memory (MemTable), result of `MemTableIter` is returned. For example, in the following figure, `StateTableIter` will generate `1->4->5->6` in order. -![Scan example](../images/relational-table-layer/relational-table-02.svg) +![Scan example](../images/relational-table/relational-table-02.svg) + + +## Example: HashAgg + +In this doc, we will take HashAgg with extreme state (`max`, `min`) or value state (`sum`, `count`) for example, and introduce a more detailed design for the internal table schema. + +[Code](https://github.com/risingwavelabs/risingwave/blob/7f9ad2240712aa0cfe3edffb4535d43b42f32cc5/src/frontend/src/optimizer/plan_node/logical_agg.rs#L144) + +### Table id +`table_id` is a globally unique id allocated in meta for each relational table object. Meta is responsible for traversing the Plan Tree and calculating the total number of Relational Tables needed. For example, the Hash Join Operator needs 2, one for the left table and one for the right table. The number of tables needed for Agg depends on the number of agg calls. + +### Value State (Sum, Count) +Query example: +```sql +select sum(v2), count(v3) from t group by v1 +``` + +This query will need to initiate 2 Relational Tables. The schema is `table_id/group_key`. + +### Extreme State (Max, Min) +Query example: +```sql +select max(v2), min(v3) from t group by v1 +``` + +This query will need to initiate 2 Relational Tables. If the upstream is not append-only, the schema becomes `table_id/group_key/sort_key/upstream_pk`. + +The order of `sort_key` depends on the agg call kind. For example, if it's `max()`, `sort_key` will order with `Ascending`. if it's `min()`, `sort_key` will order with `Descending`. +The `upstream_pk` is also appended to ensure the uniqueness of the key. +This design allows the streaming executor not to read all the data from the storage when the cache fails, but only a part of it. The streaming executor will try to write all streaming data to storage, because there may be `update` or `delete` operations in the stream, it's impossible to always guarantee correct results without storing all data. + +If `t` is created with append-only flag, the schema becomes `table_id/group_key`, which is the same for Value State. This is because in the append-only mode, there is no `update` or `delete` operation, so the cache will never miss. Therefore, we only need to write one value to the storage. diff --git a/docs/shared-buffer.md b/docs/dev/src/design/shared-buffer.md similarity index 99% rename from docs/shared-buffer.md rename to docs/dev/src/design/shared-buffer.md index 2b63a040b4c9a..7c7dac8f06e2d 100644 --- a/docs/shared-buffer.md +++ b/docs/dev/src/design/shared-buffer.md @@ -137,4 +137,4 @@ For all data a, b of the same type, we must ensure that: ``` in-memory representation of a < in-memory representation of b, iff memcomparable(a) < memcomparable(b) -``` \ No newline at end of file +``` diff --git a/docs/state-store-overview.md b/docs/dev/src/design/state-store-overview.md similarity index 96% rename from docs/state-store-overview.md rename to docs/dev/src/design/state-store-overview.md index 0fc64516ac52f..be8f3491550fc 100644 --- a/docs/state-store-overview.md +++ b/docs/dev/src/design/state-store-overview.md @@ -22,7 +22,7 @@ In RisingWave, all streaming executors store their data into a state store. This Reading this document requires prior knowledge of LSM-Tree-based KV storage engines, like RocksDB, LevelDB, etc. -![Overview of Architecture](images/state-store-overview/state-store-overview-01.svg) +![Overview of Architecture](../images/state-store-overview/state-store-overview-01.svg) Hummock consists of a manager service on the meta node, clients on worker nodes (including compute nodes, frontend nodes, and compactor nodes), and a shared storage to store files (SSTs). Every time a new write batch is produced, the Hummock client will upload those files to shared storage, and notify the Hummock manager of the new data. With compaction going on, new files will be added and unused files will be vacuumed. The Hummock manager will take care of the lifecycle of a file — is a file being used? can we delete a file? etc. @@ -104,7 +104,7 @@ The Hummock client will batch writes and generate SSTs to sync to the underlying After the SST is uploaded to an S3-compatible service, the Hummock client will let the Hummock manager know there's a new table. The list of all SSTs along with some metadata forms a ***version***. When the Hummock client adds new SSTs to the Hummock manager, a new version will be generated with the new set of SST files. -![Write Path](images/state-store-overview/state-store-overview-02.svg) +![Write Path](../images/state-store-overview/state-store-overview-02.svg) ### Read Path @@ -114,7 +114,7 @@ For every read operation (`scan`, `get`), we will first select SSTs that might c For `scan`, we simply select by overlapping key range. For point get, we will filter SSTs further by Bloom filter. After that, we will compose a single `MergeIterator` over all SSTs. The `MergeIterator` will return all keys in range along with their epochs. Then, we will create `UserIterator` over `MergeIterator`, and for all user keys, the user iterator will pick the first full key whose epoch <= read epoch. Therefore, users can perform a snapshot read from Hummock based on the given epoch. The snapshot should be acquired beforehand and released afterwards. -![Read Path](images/state-store-overview/state-store-overview-03.svg) +![Read Path](../images/state-store-overview/state-store-overview-03.svg) Hummock implements the following iterators: - `BlockIterator`: iterates a block of an SSTable. @@ -148,7 +148,7 @@ As mentioned in [Read Path](#read-path), reads are performed on a ***version*** The SQL frontend will get the latest epoch from the meta service. Then, it will embed the epoch number into SQL plans, so that all compute nodes will read from that epoch. In theory, both SQL frontend and compute nodes will ***pin the snapshot***, to handle the case that frontend goes down and the compute nodes are still reading from Hummock (#622). However, to simplify the process, currently we ***only pin on the frontend side***. -![Hummock Service](images/state-store-overview/state-store-overview-04.svg) +![Hummock Service](../images/state-store-overview/state-store-overview-04.svg) Hummock only guarantees that writes on one node can be immediately read from the same node. However, the worker nodes running batch queries might have a slightly outdated version when a batch query plan is received (due to the local version caching). Therefore, we have a `wait_epoch` interface to wait until the local cached version contains full data of one epoch. @@ -164,7 +164,7 @@ From the perspective of the streaming executors, when they receive a barrier, th Here we have two cases: Agg executors always persist and produce new write batches when receiving a barrier; Join executors (in the future when async flush gets implemented) will produce write batches within an epoch. -![Checkpoint in Streaming](images/state-store-overview/state-store-overview-05.svg) +![Checkpoint in Streaming](../images/state-store-overview/state-store-overview-05.svg) Streaming executors cannot control when data will be persisted — they can only write to Hummock's `shared buffer`. When a barrier flows across the system and is collected by the meta service, we can ensure that all executors have written their states of ***the previous epoch*** to the shared buffer, so we can initiate checkpoint process on all worker nodes, and upload SSTs to persistent remote storage. diff --git a/docs/streaming-overview.md b/docs/dev/src/design/streaming-overview.md similarity index 97% rename from docs/streaming-overview.md rename to docs/dev/src/design/streaming-overview.md index 2379fe2db13d3..b24eeaba51cb9 100644 --- a/docs/streaming-overview.md +++ b/docs/dev/src/design/streaming-overview.md @@ -26,7 +26,7 @@ In this document we give an overview of the RisingWave streaming engine. ## Architecture -![streaming-architecture](./images/streaming-overview/streaming-architecture.svg) +![streaming-architecture](../images/streaming-overview/streaming-architecture.svg) The overall architecture of RisingWave is depicted in the figure above. In brief, RisingWave streaming engine consists of three sets of nodes: frontend, compute nodes, and meta service. The frontend node consists of the serving layer, handling users' SQL requests concurrently. Underlying is the processing layer. Each compute node hosts a collection of long-running actors for stream processing. All actors access a shared persistence layer of storage (currently AWS S3) as its state storage. The meta service maintains all meta-information and coordinates the whole cluster. @@ -38,7 +38,7 @@ When receiving a create materialized view statement at the frontend, a materiali 4. Initializing the job at the backend. The meta service notifies all compute nodes to start serving streaming pipelines. ## Actors, executors, and states -![streaming-executor](./images/streaming-overview/streaming-executor-and-compute-node.svg) +![streaming-executor](../images/streaming-overview/streaming-executor-and-compute-node.svg) ### Actors @@ -75,4 +75,3 @@ See more detailed descriptions on [Checkpoint](./checkpoint.md). ### Fault tolerance When the streaming engine crashes down, the system must globally rollback to a previous consistent snapshot. To achieve this, whenever the meta detects the failover of some certain compute node or any undergoing checkpoint procedure, it triggers a recovery process. After rebuilding the streaming pipeline, each executor will reset its local state from a consistent snapshot on the storage and recover its computation. - diff --git a/docs/images/aggregation/agg-components.png b/docs/dev/src/images/aggregation/agg-components.png similarity index 100% rename from docs/images/aggregation/agg-components.png rename to docs/dev/src/images/aggregation/agg-components.png diff --git a/docs/images/aggregation/init-agg-group.png b/docs/dev/src/images/aggregation/init-agg-group.png similarity index 100% rename from docs/images/aggregation/init-agg-group.png rename to docs/dev/src/images/aggregation/init-agg-group.png diff --git a/docs/images/architecture-design/architecture.svg b/docs/dev/src/images/architecture-design/architecture.svg similarity index 100% rename from docs/images/architecture-design/architecture.svg rename to docs/dev/src/images/architecture-design/architecture.svg diff --git a/docs/images/architecture-design/batch-query.svg b/docs/dev/src/images/architecture-design/batch-query.svg similarity index 100% rename from docs/images/architecture-design/batch-query.svg rename to docs/dev/src/images/architecture-design/batch-query.svg diff --git a/docs/images/architecture-design/plan-fragments.svg b/docs/dev/src/images/architecture-design/plan-fragments.svg similarity index 100% rename from docs/images/architecture-design/plan-fragments.svg rename to docs/dev/src/images/architecture-design/plan-fragments.svg diff --git a/docs/images/architecture-design/stream-pipeline.png b/docs/dev/src/images/architecture-design/stream-pipeline.png similarity index 100% rename from docs/images/architecture-design/stream-pipeline.png rename to docs/dev/src/images/architecture-design/stream-pipeline.png diff --git a/docs/images/backfill/backfill-sides.png b/docs/dev/src/images/backfill/backfill-sides.png similarity index 100% rename from docs/images/backfill/backfill-sides.png rename to docs/dev/src/images/backfill/backfill-sides.png diff --git a/docs/images/backfill/handle-poll.png b/docs/dev/src/images/backfill/handle-poll.png similarity index 100% rename from docs/images/backfill/handle-poll.png rename to docs/dev/src/images/backfill/handle-poll.png diff --git a/docs/images/backfill/polling.png b/docs/dev/src/images/backfill/polling.png similarity index 100% rename from docs/images/backfill/polling.png rename to docs/dev/src/images/backfill/polling.png diff --git a/docs/images/backfill/replication-example.png b/docs/dev/src/images/backfill/replication-example.png similarity index 100% rename from docs/images/backfill/replication-example.png rename to docs/dev/src/images/backfill/replication-example.png diff --git a/docs/images/backfill/replication-replicated.png b/docs/dev/src/images/backfill/replication-replicated.png similarity index 100% rename from docs/images/backfill/replication-replicated.png rename to docs/dev/src/images/backfill/replication-replicated.png diff --git a/docs/images/backfill/replication-simple.png b/docs/dev/src/images/backfill/replication-simple.png similarity index 100% rename from docs/images/backfill/replication-simple.png rename to docs/dev/src/images/backfill/replication-simple.png diff --git a/docs/images/backfill/schema.png b/docs/dev/src/images/backfill/schema.png similarity index 100% rename from docs/images/backfill/schema.png rename to docs/dev/src/images/backfill/schema.png diff --git a/docs/images/batch-local-execution-mode/example1.svg b/docs/dev/src/images/batch-local-execution-mode/example1.svg similarity index 100% rename from docs/images/batch-local-execution-mode/example1.svg rename to docs/dev/src/images/batch-local-execution-mode/example1.svg diff --git a/docs/images/batch-local-execution-mode/example2.svg b/docs/dev/src/images/batch-local-execution-mode/example2.svg similarity index 100% rename from docs/images/batch-local-execution-mode/example2.svg rename to docs/dev/src/images/batch-local-execution-mode/example2.svg diff --git a/docs/images/batch-local-execution-mode/frontend-flow.svg b/docs/dev/src/images/batch-local-execution-mode/frontend-flow.svg similarity index 100% rename from docs/images/batch-local-execution-mode/frontend-flow.svg rename to docs/dev/src/images/batch-local-execution-mode/frontend-flow.svg diff --git a/docs/images/checkpoint/checkpoint.svg b/docs/dev/src/images/checkpoint/checkpoint.svg similarity index 100% rename from docs/images/checkpoint/checkpoint.svg rename to docs/dev/src/images/checkpoint/checkpoint.svg diff --git a/docs/images/checkpoint/shared-buffer.svg b/docs/dev/src/images/checkpoint/shared-buffer.svg similarity index 100% rename from docs/images/checkpoint/shared-buffer.svg rename to docs/dev/src/images/checkpoint/shared-buffer.svg diff --git a/docs/images/consistent-hash/actor-data.svg b/docs/dev/src/images/consistent-hash/actor-data.svg similarity index 100% rename from docs/images/consistent-hash/actor-data.svg rename to docs/dev/src/images/consistent-hash/actor-data.svg diff --git a/docs/images/consistent-hash/actor-state-table.svg b/docs/dev/src/images/consistent-hash/actor-state-table.svg similarity index 100% rename from docs/images/consistent-hash/actor-state-table.svg rename to docs/dev/src/images/consistent-hash/actor-state-table.svg diff --git a/docs/images/consistent-hash/data-distribution.svg b/docs/dev/src/images/consistent-hash/data-distribution.svg similarity index 100% rename from docs/images/consistent-hash/data-distribution.svg rename to docs/dev/src/images/consistent-hash/data-distribution.svg diff --git a/docs/images/consistent-hash/data-redistribution-1.svg b/docs/dev/src/images/consistent-hash/data-redistribution-1.svg similarity index 100% rename from docs/images/consistent-hash/data-redistribution-1.svg rename to docs/dev/src/images/consistent-hash/data-redistribution-1.svg diff --git a/docs/images/consistent-hash/data-redistribution-2.svg b/docs/dev/src/images/consistent-hash/data-redistribution-2.svg similarity index 100% rename from docs/images/consistent-hash/data-redistribution-2.svg rename to docs/dev/src/images/consistent-hash/data-redistribution-2.svg diff --git a/docs/images/consistent-hash/storage-data-layout.svg b/docs/dev/src/images/consistent-hash/storage-data-layout.svg similarity index 100% rename from docs/images/consistent-hash/storage-data-layout.svg rename to docs/dev/src/images/consistent-hash/storage-data-layout.svg diff --git a/docs/images/data-model-and-encoding/chunk.svg b/docs/dev/src/images/data-model-and-encoding/chunk.svg similarity index 100% rename from docs/images/data-model-and-encoding/chunk.svg rename to docs/dev/src/images/data-model-and-encoding/chunk.svg diff --git a/docs/images/data-model-and-encoding/row-format.svg b/docs/dev/src/images/data-model-and-encoding/row-format.svg similarity index 100% rename from docs/images/data-model-and-encoding/row-format.svg rename to docs/dev/src/images/data-model-and-encoding/row-format.svg diff --git a/docs/images/data-source/data-source-arch.svg b/docs/dev/src/images/data-source/data-source-arch.svg similarity index 100% rename from docs/images/data-source/data-source-arch.svg rename to docs/dev/src/images/data-source/data-source-arch.svg diff --git a/docs/images/logo-title.svg b/docs/dev/src/images/logo-title.svg similarity index 100% rename from docs/images/logo-title.svg rename to docs/dev/src/images/logo-title.svg diff --git a/docs/images/logo.svg b/docs/dev/src/images/logo.svg similarity index 100% rename from docs/images/logo.svg rename to docs/dev/src/images/logo.svg diff --git a/docs/images/meta-service/cluster-deployment.svg b/docs/dev/src/images/meta-service/cluster-deployment.svg similarity index 100% rename from docs/images/meta-service/cluster-deployment.svg rename to docs/dev/src/images/meta-service/cluster-deployment.svg diff --git a/docs/images/meta-service/notification.svg b/docs/dev/src/images/meta-service/notification.svg similarity index 100% rename from docs/images/meta-service/notification.svg rename to docs/dev/src/images/meta-service/notification.svg diff --git a/docs/images/mv-on-mv/mv-on-mv-01.svg b/docs/dev/src/images/mv-on-mv/mv-on-mv-01.svg similarity index 100% rename from docs/images/mv-on-mv/mv-on-mv-01.svg rename to docs/dev/src/images/mv-on-mv/mv-on-mv-01.svg diff --git a/docs/images/mv-on-mv/mv-on-mv-02.svg b/docs/dev/src/images/mv-on-mv/mv-on-mv-02.svg similarity index 100% rename from docs/images/mv-on-mv/mv-on-mv-02.svg rename to docs/dev/src/images/mv-on-mv/mv-on-mv-02.svg diff --git a/docs/images/relational-table-layer/relational-table-01.svg b/docs/dev/src/images/relational-table/relational-table-01.svg similarity index 100% rename from docs/images/relational-table-layer/relational-table-01.svg rename to docs/dev/src/images/relational-table/relational-table-01.svg diff --git a/docs/images/relational-table-layer/relational-table-02.svg b/docs/dev/src/images/relational-table/relational-table-02.svg similarity index 100% rename from docs/images/relational-table-layer/relational-table-02.svg rename to docs/dev/src/images/relational-table/relational-table-02.svg diff --git a/docs/images/relational-table-layer/relational-table-03.svg b/docs/dev/src/images/relational-table/relational-table-03.svg similarity index 100% rename from docs/images/relational-table-layer/relational-table-03.svg rename to docs/dev/src/images/relational-table/relational-table-03.svg diff --git a/docs/images/state-store-overview/state-store-overview-01.svg b/docs/dev/src/images/state-store-overview/state-store-overview-01.svg similarity index 100% rename from docs/images/state-store-overview/state-store-overview-01.svg rename to docs/dev/src/images/state-store-overview/state-store-overview-01.svg diff --git a/docs/images/state-store-overview/state-store-overview-02.svg b/docs/dev/src/images/state-store-overview/state-store-overview-02.svg similarity index 100% rename from docs/images/state-store-overview/state-store-overview-02.svg rename to docs/dev/src/images/state-store-overview/state-store-overview-02.svg diff --git a/docs/images/state-store-overview/state-store-overview-03.svg b/docs/dev/src/images/state-store-overview/state-store-overview-03.svg similarity index 100% rename from docs/images/state-store-overview/state-store-overview-03.svg rename to docs/dev/src/images/state-store-overview/state-store-overview-03.svg diff --git a/docs/images/state-store-overview/state-store-overview-04.svg b/docs/dev/src/images/state-store-overview/state-store-overview-04.svg similarity index 100% rename from docs/images/state-store-overview/state-store-overview-04.svg rename to docs/dev/src/images/state-store-overview/state-store-overview-04.svg diff --git a/docs/images/state-store-overview/state-store-overview-05.svg b/docs/dev/src/images/state-store-overview/state-store-overview-05.svg similarity index 100% rename from docs/images/state-store-overview/state-store-overview-05.svg rename to docs/dev/src/images/state-store-overview/state-store-overview-05.svg diff --git a/docs/images/streaming-overview/streaming-architecture.svg b/docs/dev/src/images/streaming-overview/streaming-architecture.svg similarity index 100% rename from docs/images/streaming-overview/streaming-architecture.svg rename to docs/dev/src/images/streaming-overview/streaming-architecture.svg diff --git a/docs/images/streaming-overview/streaming-executor-and-compute-node.svg b/docs/dev/src/images/streaming-overview/streaming-executor-and-compute-node.svg similarity index 100% rename from docs/images/streaming-overview/streaming-executor-and-compute-node.svg rename to docs/dev/src/images/streaming-overview/streaming-executor-and-compute-node.svg diff --git a/docs/metrics.md b/docs/dev/src/metrics.md similarity index 98% rename from docs/metrics.md rename to docs/dev/src/metrics.md index b0216c07fc83e..14d98c7a365ea 100644 --- a/docs/metrics.md +++ b/docs/dev/src/metrics.md @@ -5,7 +5,7 @@ It covers what each metric measures, and what information we may derive from it. ## Barrier Latency -Prerequisite: [Checkpoint](./checkpoint.md) +Prerequisite: [Checkpoint](./design/checkpoint.md) This metric measures the duration from which a barrier is injected into **all** sources in the stream graph, to the barrier flown through all executors in the graph. diff --git a/docs/relational_table/relational-table-schema.md b/docs/relational_table/relational-table-schema.md deleted file mode 100644 index 64cd615feda25..0000000000000 --- a/docs/relational_table/relational-table-schema.md +++ /dev/null @@ -1,35 +0,0 @@ -# Relational Table Schema - -We introduce the rough row-based encoding format in [relational states](storing-state-using-relational-table.md#row-based-encoding) - -In this doc, we will take HashAgg with extreme state (`max`, `min`) or value state (`sum`, `count`) for example, and introduce a more detailed design for the internal table schema. - -[Code](https://github.com/risingwavelabs/risingwave/blob/7f9ad2240712aa0cfe3edffb4535d43b42f32cc5/src/frontend/src/optimizer/plan_node/logical_agg.rs#L144) - -## Table id -`table_id` is a globally unique id allocated in meta for each relational table object. Meta is responsible for traversing the Plan Tree and calculating the total number of Relational Tables needed. For example, the Hash Join Operator needs 2, one for the left table and one for the right table. The number of tables needed for Agg depends on the number of agg calls. - -## Value State (Sum, Count) -Query example: -```sql -select sum(v2), count(v3) from t group by v1 -``` - -This query will need to initiate 2 Relational Tables. The schema is `table_id/group_key`. - -## Extreme State (Max, Min) -Query example: -```sql -select max(v2), min(v3) from t group by v1 -``` - -This query will need to initiate 2 Relational Tables. If the upstream is not append-only, the schema becomes `table_id/group_key/sort_key/upstream_pk`. - -The order of `sort_key` depends on the agg call kind. For example, if it's `max()`, `sort_key` will order with `Ascending`. if it's `min()`, `sort_key` will order with `Descending`. -The `upstream_pk` is also appended to ensure the uniqueness of the key. -This design allows the streaming executor not to read all the data from the storage when the cache fails, but only a part of it. The streaming executor will try to write all streaming data to storage, because there may be `update` or `delete` operations in the stream, it's impossible to always guarantee correct results without storing all data. - -If `t` is created with append-only flag, the schema becomes `table_id/group_key`, which is the same for Value State. This is because in the append-only mode, there is no `update` or `delete` operation, so the cache will never miss. Therefore, we only need to write one value to the storage. - - - diff --git a/docs/rustdoc/README.md b/docs/rustdoc/README.md index 1b3e70e1113c2..0adf956748290 100644 --- a/docs/rustdoc/README.md +++ b/docs/rustdoc/README.md @@ -1,6 +1,6 @@ This folder contains files for generating a nice rustdoc index page. -Online version (for latest main): +Online version (for latest main): To build and open locally, run the following command in the project root: diff --git a/docs/rustdoc/index.md b/docs/rustdoc/index.md index cfb74b8055b8a..a76edb23cb2b4 100644 --- a/docs/rustdoc/index.md +++ b/docs/rustdoc/index.md @@ -4,9 +4,7 @@ Welcome to an overview of the developer documentations of RisingWave! ## Developer Docs -To learn how to develop RisingWave, see the [RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/). - -The [design docs](https://github.com/risingwavelabs/risingwave/blob/main/docs/README.md) covers some high-level ideas of how we built RisingWave. +To learn how to develop RisingWave, and access high-level design docs, see the [RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/). ## Crate Docs diff --git a/docs/rustdoc/rust.css b/docs/rustdoc/rust.css index 71cf5e3df0004..9c76bb08c3898 100644 --- a/docs/rustdoc/rust.css +++ b/docs/rustdoc/rust.css @@ -1,18 +1,21 @@ /* This file is copied from the Rust Project, which is dual-licensed under -Apache 2.0 and MIT terms. */ +Apache 2.0 and MIT terms. https: //github.com/rust-lang/rust/blob/7d640b670e521a0491ea1e49082d1cb5632e2562/src/doc/rust.css +*/ /* General structure */ body { + font-family: serif; margin: 0 auto; padding: 0 15px; font-size: 18px; - color: #333; + color: #000; line-height: 1.428571429; -webkit-box-sizing: unset; -moz-box-sizing: unset; box-sizing: unset; + background: #fff; } @media (min-width: 768px) { body { @@ -20,6 +23,14 @@ body { } } +h1, +h2, +h3, +h4, +h5, +h6 { + font-family: sans-serif; +} h2, h3, h4, h5, h6 { font-weight: 400; line-height: 1.1; @@ -37,8 +48,8 @@ h4, h5, h6 { margin-bottom: 10px; padding: 5px 10px; } -h5, h6 { - color: black; +h5, +h6 { text-decoration: underline; } @@ -135,6 +146,31 @@ h1 a:link, h1 a:visited, h2 a:link, h2 a:visited, h3 a:link, h3 a:visited, h4 a:link, h4 a:visited, h5 a:link, h5 a:visited {color: black;} +h1, +h2, +h3, +h4, +h5 { + /* This is needed to be able to position the doc-anchor. Ideally there + would be a
around the whole document, but we don't have that. */ + position: relative; +} + +a.doc-anchor { + color: black; + display: none; + position: absolute; + left: -20px; + /* We add this padding so that when the cursor moves from the heading's text to the anchor, + the anchor doesn't disappear. */ + padding-right: 5px; + /* And this padding is used to make the anchor larger and easier to click on. */ + padding-left: 3px; +} + +*:hover>.doc-anchor { + display: block; +} /* Code */ pre, code { From 4d34bd321073b88dce91ea0b05b4abfaa4aca980 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 10 Jul 2024 15:35:49 +0800 Subject: [PATCH 12/20] feat: remote input subscribe on barrier mutation (#17612) --- src/stream/src/executor/exchange/input.rs | 22 ++- src/stream/src/task/barrier_manager.rs | 46 +++-- .../src/task/barrier_manager/managed_state.rs | 184 ++++++++++++------ src/stream/src/task/barrier_manager/tests.rs | 9 +- 4 files changed, 167 insertions(+), 94 deletions(-) diff --git a/src/stream/src/executor/exchange/input.rs b/src/stream/src/executor/exchange/input.rs index 4c5648ab1dd49..bd348e65defca 100644 --- a/src/stream/src/executor/exchange/input.rs +++ b/src/stream/src/executor/exchange/input.rs @@ -15,7 +15,7 @@ use std::pin::Pin; use std::task::{Context, Poll}; -use anyhow::Context as _; +use anyhow::{anyhow, Context as _}; use futures::pin_mut; use futures_async_stream::try_stream; use pin_project::pin_project; @@ -147,6 +147,7 @@ impl RemoteInput { metrics: Arc, batched_permits_limit: usize, ) { + let self_actor_id = up_down_ids.1; let client = client_pool.get_by_addr(upstream_addr).await?; let (stream, permits_tx) = client .get_stream(up_down_ids.0, up_down_ids.1, up_down_frag.0, up_down_frag.1) @@ -162,6 +163,7 @@ impl RemoteInput { let span: await_tree::Span = format!("RemoteInput (actor {up_actor_id})").into(); let mut batched_permits_accumulated = 0; + let mut mutation_subscriber = None; pin_mut!(stream); while let Some(data_res) = stream.next().verbose_instrument_await(span.clone()).await { @@ -203,10 +205,22 @@ impl RemoteInput { barrier.mutation.is_none(), "Mutation should be erased in remote side" ); - let mutation = local_barrier_manager - .read_barrier_mutation(barrier) + let mutation_subscriber = + mutation_subscriber.get_or_insert_with(|| { + local_barrier_manager + .subscribe_barrier_mutation(self_actor_id, barrier) + }); + + let mutation = mutation_subscriber + .recv() .await - .context("Read barrier mutation error")?; + .ok_or_else(|| { + anyhow!("failed to receive mutation of barrier {:?}", barrier) + }) + .map(|(prev_epoch, mutation)| { + assert_eq!(prev_epoch, barrier.epoch.prev); + mutation + })?; barrier.mutation = mutation; } } diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 1098f505004c4..d20759f017a2b 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -31,7 +31,7 @@ use rw_futures_util::{pending_on_none, AttachedFuture}; use thiserror_ext::AsReport; use tokio::select; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; -use tokio::sync::oneshot; +use tokio::sync::{mpsc, oneshot}; use tokio::task::JoinHandle; use tonic::{Code, Status}; @@ -48,6 +48,7 @@ mod tests; pub use progress::CreateMviewProgress; use risingwave_common::catalog::TableId; +use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; use risingwave_hummock_sdk::{LocalSstableInfo, SyncResult}; @@ -187,6 +188,8 @@ impl CreateActorContext { } } +pub(super) type SubscribeMutationItem = (u64, Option>); + pub(super) enum LocalBarrierEvent { ReportActorCollected { actor_id: ActorId, @@ -197,9 +200,10 @@ pub(super) enum LocalBarrierEvent { actor: ActorId, state: BackfillState, }, - ReadBarrierMutation { - barrier: Barrier, - mutation_sender: oneshot::Sender>>, + SubscribeBarrierMutation { + actor_id: ActorId, + epoch: EpochPair, + mutation_sender: mpsc::UnboundedSender, }, #[cfg(test)] Flush(oneshot::Sender<()>), @@ -515,11 +519,13 @@ impl LocalBarrierWorker { } => { self.update_create_mview_progress(current_epoch, actor, state); } - LocalBarrierEvent::ReadBarrierMutation { - barrier, + LocalBarrierEvent::SubscribeBarrierMutation { + actor_id, + epoch, mutation_sender, } => { - self.read_barrier_mutation(barrier, mutation_sender); + self.state + .subscribe_actor_mutation(actor_id, epoch.prev, mutation_sender); } #[cfg(test)] LocalBarrierEvent::Flush(sender) => sender.send(()).unwrap(), @@ -642,15 +648,6 @@ impl LocalBarrierWorker { Ok(()) } - /// Read mutation from barrier state. - fn read_barrier_mutation( - &mut self, - barrier: Barrier, - sender: oneshot::Sender>>, - ) { - self.state.read_barrier_mutation(&barrier, sender); - } - /// Register sender for source actors, used to send barriers. fn register_sender(&mut self, actor_id: ActorId, senders: Vec>) { tracing::debug!( @@ -907,17 +904,18 @@ impl LocalBarrierManager { } /// When a `RemoteInput` get a barrier, it should wait and read the barrier mutation from the barrier manager. - pub async fn read_barrier_mutation( + pub fn subscribe_barrier_mutation( &self, - barrier: &Barrier, - ) -> StreamResult>> { - let (tx, rx) = oneshot::channel(); - self.send_event(LocalBarrierEvent::ReadBarrierMutation { - barrier: barrier.clone(), + actor_id: ActorId, + first_barrier: &Barrier, + ) -> mpsc::UnboundedReceiver { + let (tx, rx) = mpsc::unbounded_channel(); + self.send_event(LocalBarrierEvent::SubscribeBarrierMutation { + actor_id, + epoch: first_barrier.epoch, mutation_sender: tx, }); - rx.await - .map_err(|_| anyhow!("barrier manager maybe reset").into()) + rx } } diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 6c70525aa4b3a..40b47ee26e8f7 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::assert_matches::assert_matches; -use std::collections::btree_map::Entry; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::fmt::{Debug, Display, Formatter}; use std::future::Future; @@ -34,10 +33,10 @@ use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgres use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; use rw_futures_util::pending_on_none; use thiserror_ext::AsReport; -use tokio::sync::oneshot; +use tokio::sync::mpsc; use super::progress::BackfillState; -use super::BarrierCompleteResult; +use super::{BarrierCompleteResult, SubscribeMutationItem}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, Mutation}; @@ -69,13 +68,6 @@ impl Debug for IssuedState { /// The state machine of local barrier manager. #[derive(Debug)] enum ManagedBarrierStateInner { - /// Received barrier from actors in other compute nodes in remote input, however no `send_barrier` - /// request from the meta service is issued. - Stashed { - /// Senders registered by the remote input. - mutation_senders: Vec>>>, - }, - /// Meta service has issued a `send_barrier` request. We're collecting barriers now. Issued(IssuedState), @@ -133,9 +125,6 @@ impl Display for ManagedBarrierStateDebugInfo<'_> { for (epoch, barrier_state) in self.epoch_barrier_state_map { write!(f, "> Epoch {}: ", epoch)?; match &barrier_state.inner { - ManagedBarrierStateInner::Stashed { .. } => { - write!(f, "Stashed")?; - } ManagedBarrierStateInner::Issued(state) => { write!(f, "Issued [{:?}]. Remaining actors: [", state.kind)?; let mut is_prev_epoch_issued = false; @@ -193,12 +182,26 @@ impl Display for ManagedBarrierStateDebugInfo<'_> { } } +#[derive(Default)] +struct ActorMutationSubscribers { + pending_subscribers: BTreeMap>>, + started_subscribers: Vec>, +} + +impl ActorMutationSubscribers { + fn is_empty(&self) -> bool { + self.pending_subscribers.is_empty() && self.started_subscribers.is_empty() + } +} + pub(super) struct ManagedBarrierState { /// Record barrier state for each epoch of concurrent checkpoints. /// /// The key is `prev_epoch`, and the first value is `curr_epoch` epoch_barrier_state_map: BTreeMap, + mutation_subscribers: HashMap, + /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. pub(super) create_mview_progress: HashMap>, @@ -231,6 +234,7 @@ impl ManagedBarrierState { ) -> Self { Self { epoch_barrier_state_map: BTreeMap::default(), + mutation_subscribers: Default::default(), create_mview_progress: Default::default(), state_store, streaming_metrics, @@ -246,39 +250,69 @@ impl ManagedBarrierState { } } - pub fn read_barrier_mutation( + pub(super) fn subscribe_actor_mutation( &mut self, - barrier: &Barrier, - sender: oneshot::Sender>>, + actor_id: ActorId, + start_prev_epoch: u64, + tx: mpsc::UnboundedSender, ) { - match self.epoch_barrier_state_map.entry(barrier.epoch.prev) { - Entry::Vacant(v) => { - v.insert(BarrierState { - curr_epoch: barrier.epoch.curr, - inner: ManagedBarrierStateInner::Stashed { - mutation_senders: vec![sender], - }, - }); - } - Entry::Occupied(mut o) => { - let state = o.get_mut(); - match &mut state.inner { - ManagedBarrierStateInner::Stashed { - ref mut mutation_senders, - } => { - mutation_senders.push(sender); - } - ManagedBarrierStateInner::Issued(IssuedState { mutation, .. }) => { - let _ = sender.send(mutation.clone()); - } - _ => { - panic!( - "cannot read barrier mutation {:?} at current state: {:?}", - barrier.epoch, state.inner - ) + let subscribers = self.mutation_subscribers.entry(actor_id).or_default(); + if let Some(state) = self.epoch_barrier_state_map.get(&start_prev_epoch) { + match &state.inner { + ManagedBarrierStateInner::Issued(issued_state) => { + assert!(issued_state.remaining_actors.contains(&actor_id)); + for (prev_epoch, state) in + self.epoch_barrier_state_map.range(start_prev_epoch..) + { + match &state.inner { + ManagedBarrierStateInner::Issued(issued_state) => { + if issued_state.remaining_actors.contains(&actor_id) { + if tx + .send((*prev_epoch, issued_state.mutation.clone())) + .is_err() + { + // No more subscribe on the mutation. Simply return. + return; + } + } else { + // The barrier no more collect from such actor. End subscribe on mutation. + return; + } + } + state @ ManagedBarrierStateInner::AllCollected + | state @ ManagedBarrierStateInner::Completed(_) => { + unreachable!( + "should be Issued when having new subscriber, but current state: {:?}", + state + ) + } + } } + subscribers.started_subscribers.push(tx); + } + state @ ManagedBarrierStateInner::AllCollected + | state @ ManagedBarrierStateInner::Completed(_) => { + unreachable!( + "should be Issued when having new subscriber, but current state: {:?}", + state + ) } } + } else { + // Barrier has not issued yet. Store the pending tx + if let Some((last_epoch, _)) = self.epoch_barrier_state_map.last_key_value() { + assert!( + *last_epoch < start_prev_epoch, + "later barrier {} has been issued, but skip the start epoch {:?}", + last_epoch, + start_prev_epoch + ); + } + subscribers + .pending_subscribers + .entry(start_prev_epoch) + .or_default() + .push(tx); } } @@ -300,7 +334,7 @@ impl ManagedBarrierState { ManagedBarrierStateInner::AllCollected | ManagedBarrierStateInner::Completed(_) => { continue; } - ManagedBarrierStateInner::Stashed { .. } | ManagedBarrierStateInner::Issued(_) => { + ManagedBarrierStateInner::Issued(_) => { break; } } @@ -445,16 +479,12 @@ impl ManagedBarrierState { ); match self.epoch_barrier_state_map.get_mut(&barrier.epoch.prev) { - Some(&mut BarrierState { - inner: ManagedBarrierStateInner::Stashed { .. }, - .. - }) - | None => { + None => { // If the barrier's state is stashed, this occurs exclusively in scenarios where the barrier has not been // injected by the barrier manager, or the barrier message is blocked at the `RemoteInput` side waiting for injection. // Given these conditions, it's inconceivable for an actor to attempt collect at this point. panic!( - "cannot collect new actor barrier {:?} at current state: Stashed or None", + "cannot collect new actor barrier {:?} at current state: None", barrier.epoch, ) } @@ -497,26 +527,56 @@ impl ManagedBarrierState { .streaming_metrics .barrier_inflight_latency .start_timer(); - match self.epoch_barrier_state_map.get_mut(&barrier.epoch.prev) { - Some(&mut BarrierState { - inner: - ManagedBarrierStateInner::Stashed { - ref mut mutation_senders, - }, - .. - }) => { - for sender in mutation_senders.drain(..) { - let _ = sender.send(barrier.mutation.clone()); - } - } - Some(BarrierState { ref inner, .. }) => { + if let Some(BarrierState { ref inner, .. }) = + self.epoch_barrier_state_map.get_mut(&barrier.epoch.prev) + { + { panic!( "barrier epochs{:?} state has already been `Issued`. Current state: {:?}", barrier.epoch, inner ); } - None => {} }; + + for (actor_id, subscribers) in &mut self.mutation_subscribers { + if actor_ids_to_collect.contains(actor_id) { + if let Some((first_epoch, _)) = subscribers.pending_subscribers.first_key_value() { + assert!( + *first_epoch >= barrier.epoch.prev, + "barrier epoch {:?} skip subscribed epoch {}", + barrier.epoch, + first_epoch + ); + if *first_epoch == barrier.epoch.prev { + subscribers.started_subscribers.extend( + subscribers + .pending_subscribers + .pop_first() + .expect("should exist") + .1, + ); + } + } + subscribers.started_subscribers.retain(|tx| { + tx.send((barrier.epoch.prev, barrier.mutation.clone())) + .is_ok() + }); + } else { + subscribers.started_subscribers.clear(); + if let Some((first_epoch, _)) = subscribers.pending_subscribers.first_key_value() { + assert!( + *first_epoch > barrier.epoch.prev, + "barrier epoch {:?} skip subscribed epoch {}", + barrier.epoch, + first_epoch + ); + } + } + } + + self.mutation_subscribers + .retain(|_, subscribers| !subscribers.is_empty()); + self.epoch_barrier_state_map.insert( barrier.epoch.prev, BarrierState { diff --git a/src/stream/src/task/barrier_manager/tests.rs b/src/stream/src/task/barrier_manager/tests.rs index 82a08e1d66117..60b3867a1a0c0 100644 --- a/src/stream/src/task/barrier_manager/tests.rs +++ b/src/stream/src/task/barrier_manager/tests.rs @@ -180,10 +180,11 @@ async fn test_managed_barrier_collection_separately() -> StreamResult<()> { // Prepare the barrier let curr_epoch = test_epoch(2); let barrier = Barrier::new_test_barrier(curr_epoch); - let epoch = barrier.epoch.prev; + + let mut mutation_subscriber = manager.subscribe_barrier_mutation(extra_actor_id, &barrier); // Read the mutation after receiving the barrier from remote input. - let mut mutation_reader = pin!(manager.read_barrier_mutation(&barrier)); + let mut mutation_reader = pin!(mutation_subscriber.recv()); assert!(poll_fn(|cx| Poll::Ready(mutation_reader.as_mut().poll(cx).is_pending())).await); request_tx @@ -200,8 +201,8 @@ async fn test_managed_barrier_collection_separately() -> StreamResult<()> { })) .unwrap(); - let mutation = mutation_reader.await.unwrap(); - assert_eq!(mutation, barrier.mutation); + let (epoch, mutation) = mutation_reader.await.unwrap(); + assert_eq!((epoch, &mutation), (barrier.epoch.prev, &barrier.mutation)); // Collect a barrier before sending manager.collect(extra_actor_id, &barrier); From 0987e27d057f45195cece40aa2420058bef1e5a0 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 10 Jul 2024 15:38:01 +0800 Subject: [PATCH 13/20] refactor: graceful shutdown on meta node & unify election path (#17608) Signed-off-by: Bugen Zhao --- src/cmd/src/lib.rs | 3 +- src/cmd_all/src/standalone.rs | 3 +- src/meta/node/src/lib.rs | 42 +-- src/meta/node/src/server.rs | 285 +++++++----------- src/meta/service/src/meta_member_service.rs | 57 ++-- src/meta/src/manager/idle.rs | 23 +- src/meta/src/rpc/election/dummy.rs | 73 +++++ src/meta/src/rpc/election/etcd.rs | 8 +- src/meta/src/rpc/election/mod.rs | 7 +- src/meta/src/rpc/election/sql.rs | 6 +- src/meta/src/rpc/metrics.rs | 6 +- .../src/compaction_test_runner.rs | 2 +- src/tests/simulation/src/cluster.rs | 7 +- 13 files changed, 244 insertions(+), 278 deletions(-) create mode 100644 src/meta/src/rpc/election/dummy.rs diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index 7d21fc44b4e57..9017f58606f7c 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -44,8 +44,7 @@ pub fn compute(opts: ComputeNodeOpts) -> ! { pub fn meta(opts: MetaNodeOpts) -> ! { init_risingwave_logger(LoggerSettings::from_opts(&opts)); - // TODO(shutdown): pass the shutdown token - main_okk(|_| risingwave_meta_node::start(opts)); + main_okk(|shutdown| risingwave_meta_node::start(opts, shutdown)); } pub fn frontend(opts: FrontendOpts) -> ! { diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 4f8c208c89aa3..325f2f8ff395b 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -194,9 +194,10 @@ pub async fn standalone( is_in_memory = matches!(opts.backend, Some(MetaBackend::Mem)); tracing::info!("starting meta-node thread with cli args: {:?}", opts); + let shutdown = shutdown.clone(); let _meta_handle = tokio::spawn(async move { let dangerous_max_idle_secs = opts.dangerous_max_idle_secs; - risingwave_meta_node::start(opts).await; + risingwave_meta_node::start(opts, shutdown).await; tracing::warn!("meta is stopped, shutdown all nodes"); if let Some(idle_exit_secs) = dangerous_max_idle_secs { eprintln!("{}", diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index a7600ba930a15..5fd658c8a6581 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -26,6 +26,7 @@ use redact::Secret; use risingwave_common::config::OverrideConfig; use risingwave_common::util::meta_addr::MetaAddressStrategy; use risingwave_common::util::resource_util; +use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; use risingwave_meta::*; @@ -204,7 +205,10 @@ use risingwave_common::config::{load_config, MetaBackend, RwConfig}; use tracing::info; /// Start meta node -pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { +pub fn start( + opts: MetaNodeOpts, + shutdown: CancellationToken, +) -> Pin + Send>> { // WARNING: don't change the function signature. Making it `async fn` will cause // slow compile in release mode. Box::pin(async move { @@ -324,7 +328,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { max_timeout_ms / 1000 } + MIN_TIMEOUT_INTERVAL_SEC; - let (mut join_handle, leader_lost_handle, shutdown_send) = rpc_serve( + rpc_serve( add_info, backend, max_heartbeat_interval, @@ -428,42 +432,10 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { }, config.system.into_init_system_params(), Default::default(), + shutdown, ) .await .unwrap(); - - tracing::info!("Meta server listening at {}", listen_addr); - - match leader_lost_handle { - None => { - tokio::select! { - _ = tokio::signal::ctrl_c() => { - tracing::info!("receive ctrl+c"); - shutdown_send.send(()).unwrap(); - join_handle.await.unwrap() - } - res = &mut join_handle => res.unwrap(), - }; - } - Some(mut handle) => { - tokio::select! { - _ = &mut handle => { - tracing::info!("receive leader lost signal"); - // When we lose leadership, we will exit as soon as possible. - } - _ = tokio::signal::ctrl_c() => { - tracing::info!("receive ctrl+c"); - shutdown_send.send(()).unwrap(); - join_handle.await.unwrap(); - handle.abort(); - } - res = &mut join_handle => { - res.unwrap(); - handle.abort(); - }, - }; - } - }; }) } diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 9098ee1429c82..55f069e5e0104 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -16,9 +16,7 @@ use std::sync::Arc; use std::time::Duration; use anyhow::Context; -use either::Either; use etcd_client::ConnectOptions; -use futures::future::join_all; use otlp_embedded::TraceServiceServer; use regex::Regex; use risingwave_common::monitor::{RouterExt, TcpConfig}; @@ -26,11 +24,13 @@ use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::{report_scarf_enabled, report_to_scarf, telemetry_env_enabled}; +use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common_service::{MetricsManager, TracingExtractLayer}; use risingwave_meta::barrier::StreamRpcManager; use risingwave_meta::controller::catalog::CatalogController; use risingwave_meta::controller::cluster::ClusterController; use risingwave_meta::manager::{MetaStoreImpl, MetadataManager, SystemParamsManagerImpl}; +use risingwave_meta::rpc::election::dummy::DummyElectionClient; use risingwave_meta::rpc::intercept::MetricsMiddlewareLayer; use risingwave_meta::rpc::ElectionClientRef; use risingwave_meta::stream::ScaleController; @@ -76,10 +76,7 @@ use risingwave_pb::user::user_service_server::UserServiceServer; use risingwave_rpc_client::ComputeClientPool; use sea_orm::{ConnectionTrait, DbBackend}; use thiserror_ext::AsReport; -use tokio::sync::oneshot::{channel as OneChannel, Receiver as OneReceiver}; use tokio::sync::watch; -use tokio::sync::watch::{Receiver as WatchReceiver, Sender as WatchSender}; -use tokio::task::JoinHandle; use crate::backup_restore::BackupManager; use crate::barrier::{BarrierScheduler, GlobalBarrierManager}; @@ -124,6 +121,9 @@ pub mod started { } } +/// A wrapper around [`rpc_serve_with_store`] that dispatches different store implementations. +/// +/// For the timing of returning, see [`rpc_serve_with_store`]. pub async fn rpc_serve( address_info: AddressInfo, meta_store_backend: MetaStoreBackend, @@ -132,7 +132,8 @@ pub async fn rpc_serve( opts: MetaOpts, init_system_params: SystemParams, init_session_config: SessionConfig, -) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { + shutdown: CancellationToken, +) -> MetaResult<()> { match meta_store_backend { MetaStoreBackend::Etcd { endpoints, @@ -168,27 +169,34 @@ pub async fn rpc_serve( rpc_serve_with_store( MetaStoreImpl::Kv(meta_store), - Some(election_client), + election_client, address_info, max_cluster_heartbeat_interval, lease_interval_secs, opts, init_system_params, init_session_config, + shutdown, ) + .await } MetaStoreBackend::Mem => { let meta_store = MemStore::new().into_ref(); + let dummy_election_client = Arc::new(DummyElectionClient::new( + address_info.advertise_addr.clone(), + )); rpc_serve_with_store( MetaStoreImpl::Kv(meta_store), - None, + dummy_election_client, address_info, max_cluster_heartbeat_interval, lease_interval_secs, opts, init_system_params, init_session_config, + shutdown, ) + .await } MetaStoreBackend::Sql { endpoint } => { let max_connection = if DbBackend::Sqlite.is_prefix_of(&endpoint) { @@ -225,130 +233,120 @@ pub async fn rpc_serve( rpc_serve_with_store( MetaStoreImpl::Sql(meta_store_sql), - Some(election_client), + election_client, address_info, max_cluster_heartbeat_interval, lease_interval_secs, opts, init_system_params, init_session_config, + shutdown, ) + .await } } } -#[expect(clippy::type_complexity)] -pub fn rpc_serve_with_store( +/// Bootstraps the follower or leader service based on the election status. +/// +/// Returns when the `shutdown` token is triggered, or when leader status is lost, or if the leader +/// service fails to start. +pub async fn rpc_serve_with_store( meta_store_impl: MetaStoreImpl, - election_client: Option, + election_client: ElectionClientRef, address_info: AddressInfo, max_cluster_heartbeat_interval: Duration, lease_interval_secs: u64, opts: MetaOpts, init_system_params: SystemParams, init_session_config: SessionConfig, -) -> MetaResult<(JoinHandle<()>, Option>, WatchSender<()>)> { - let (svc_shutdown_tx, svc_shutdown_rx) = watch::channel(()); + shutdown: CancellationToken, +) -> MetaResult<()> { + // TODO(shutdown): directly use cancellation token + let (election_shutdown_tx, election_shutdown_rx) = watch::channel(()); - let leader_lost_handle = if let Some(election_client) = election_client.clone() { - let stop_rx = svc_shutdown_tx.subscribe(); + let election_handle = tokio::spawn({ + let shutdown = shutdown.clone(); + let election_client = election_client.clone(); - let handle = tokio::spawn(async move { + async move { while let Err(e) = election_client - .run_once(lease_interval_secs as i64, stop_rx.clone()) + .run_once(lease_interval_secs as i64, election_shutdown_rx.clone()) .await { tracing::error!(error = %e.as_report(), "election error happened"); } - }); + // Leader lost, shutdown the service. + shutdown.cancel(); + } + }); - Some(handle) - } else { - None - }; + // Spawn and run the follower service if not the leader. + // Watch the leader status and switch to the leader service when elected. + // TODO: the branch seems to be always hit since the default value of `is_leader` is false until + // the election is done (unless using `DummyElectionClient`). + if !election_client.is_leader() { + // The follower service can be shutdown separately if we're going to be the leader. + let follower_shutdown = shutdown.child_token(); + + let follower_handle = tokio::spawn(start_service_as_election_follower( + follower_shutdown.clone(), + address_info.clone(), + election_client.clone(), + )); - let join_handle = tokio::spawn(async move { - if let Some(election_client) = election_client.clone() { - let mut is_leader_watcher = election_client.subscribe(); - let mut svc_shutdown_rx_clone = svc_shutdown_rx.clone(); - let (follower_shutdown_tx, follower_shutdown_rx) = OneChannel::<()>(); + // Watch and wait until we become the leader. + let mut is_leader_watcher = election_client.subscribe(); + while !*is_leader_watcher.borrow_and_update() { tokio::select! { - _ = svc_shutdown_rx_clone.changed() => return, + // External shutdown signal. Directly return without switching to leader. + _ = shutdown.cancelled() => return Ok(()), + res = is_leader_watcher.changed() => { if res.is_err() { tracing::error!("leader watcher recv failed"); } } } - let svc_shutdown_rx_clone = svc_shutdown_rx.clone(); - - // If not the leader, spawn a follower. - let follower_handle: Option> = if !*is_leader_watcher.borrow() { - let address_info_clone = address_info.clone(); - - let election_client_ = election_client.clone(); - Some(tokio::spawn(async move { - start_service_as_election_follower( - svc_shutdown_rx_clone, - follower_shutdown_rx, - address_info_clone, - Some(election_client_), - ) - .await; - })) - } else { - None - }; + } - let mut svc_shutdown_rx_clone = svc_shutdown_rx.clone(); - while !*is_leader_watcher.borrow_and_update() { - tokio::select! { - _ = svc_shutdown_rx_clone.changed() => { - return; - } - res = is_leader_watcher.changed() => { - if res.is_err() { - tracing::error!("leader watcher recv failed"); - } - } - } - } + tracing::info!("elected as leader, shutting down follower services"); + follower_shutdown.cancel(); + let _ = follower_handle.await; + } - if let Some(handle) = follower_handle { - let _res = follower_shutdown_tx.send(()); - let _ = handle.await; - } - }; + // Run the leader service. + let result = start_service_as_election_leader( + meta_store_impl, + address_info, + max_cluster_heartbeat_interval, + opts, + init_system_params, + init_session_config, + election_client, + shutdown, + ) + .await; - start_service_as_election_leader( - meta_store_impl, - address_info, - max_cluster_heartbeat_interval, - opts, - init_system_params, - init_session_config, - election_client, - svc_shutdown_rx, - ) - .await - .expect("Unable to start leader services"); - }); + // Leader service has stopped, shutdown the election service to gracefully resign. + election_shutdown_tx.send(()).ok(); + let _ = election_handle.await; - Ok((join_handle, leader_lost_handle, svc_shutdown_tx)) + result } -/// Starts all services needed for the meta follower node +/// Starts all services needed for the meta follower node. +/// +/// Returns when the `shutdown` token is triggered. pub async fn start_service_as_election_follower( - mut svc_shutdown_rx: WatchReceiver<()>, - follower_shutdown_rx: OneReceiver<()>, + shutdown: CancellationToken, address_info: AddressInfo, - election_client: Option, + election_client: ElectionClientRef, ) { - let meta_member_srv = MetaMemberServiceImpl::new(match election_client { - None => Either::Right(address_info.clone()), - Some(election_client) => Either::Left(election_client), - }); + tracing::info!("starting follower services"); + + let meta_member_srv = MetaMemberServiceImpl::new(election_client); let health_srv = HealthServiceImpl::new(); @@ -366,35 +364,21 @@ pub async fn start_service_as_election_follower( tcp_nodelay: true, keepalive_duration: None, }, - async move { - tokio::select! { - // shutdown service if all services should be shut down - res = svc_shutdown_rx.changed() => { - match res { - Ok(_) => tracing::info!("Shutting down services"), - Err(_) => tracing::error!("Service shutdown sender dropped") - } - }, - // shutdown service if follower becomes leader - res = follower_shutdown_rx => { - match res { - Ok(_) => tracing::info!("Shutting down follower services"), - Err(_) => tracing::error!("Follower service shutdown sender dropped") - } - }, - } - }, + shutdown.clone().cancelled_owned(), ); + let server_handle = tokio::spawn(server); started::set(); - server.await; + + // Wait for the shutdown signal. + shutdown.cancelled().await; + // Wait for the server to shutdown. This is necessary because we may be transitioning from follower + // to leader, and conflicts on the services must be avoided. + let _ = server_handle.await; } -/// Starts all services needed for the meta leader node -/// Only call this function once, since initializing the services multiple times will result in an -/// inconsistent state +/// Starts all services needed for the meta leader node. /// -/// ## Returns -/// Returns an error if the service initialization failed +/// Returns when the `shutdown` token is triggered, or if the service initialization fails. pub async fn start_service_as_election_leader( meta_store_impl: MetaStoreImpl, address_info: AddressInfo, @@ -402,10 +386,11 @@ pub async fn start_service_as_election_leader( opts: MetaOpts, init_system_params: SystemParams, init_session_config: SessionConfig, - election_client: Option, - mut svc_shutdown_rx: WatchReceiver<()>, + election_client: ElectionClientRef, + shutdown: CancellationToken, ) -> MetaResult<()> { - tracing::info!("Defining leader services"); + tracing::info!("starting leader services"); + let env = MetaSrvEnv::new( opts.clone(), init_system_params, @@ -479,10 +464,7 @@ pub async fn start_service_as_election_leader( .unwrap(); let object_store_media_type = hummock_manager.object_store_media_type(); - let meta_member_srv = MetaMemberServiceImpl::new(match election_client.clone() { - None => Either::Right(address_info.clone()), - Some(election_client) => Either::Left(election_client), - }); + let meta_member_srv = MetaMemberServiceImpl::new(election_client.clone()); let prometheus_client = opts.prometheus_endpoint.as_ref().map(|x| { use std::str::FromStr; @@ -504,7 +486,7 @@ pub async fn start_service_as_election_leader( let trace_srv = otlp_embedded::TraceServiceImpl::new(trace_state.clone()); #[cfg(not(madsim))] - let dashboard_task = if let Some(ref dashboard_addr) = address_info.dashboard_addr { + let _dashboard_task = if let Some(ref dashboard_addr) = address_info.dashboard_addr { let dashboard_service = crate::dashboard::DashboardService { dashboard_addr: *dashboard_addr, prometheus_client, @@ -537,6 +519,7 @@ pub async fn start_service_as_election_leader( ); let (sink_manager, shutdown_handle) = SinkCoordinatorManager::start_worker(); + // TODO(shutdown): remove this as there's no need to gracefully shutdown some of these sub-tasks. let mut sub_tasks = vec![shutdown_handle]; let stream_rpc_manager = StreamRpcManager::new(env.clone()); @@ -721,17 +704,17 @@ pub async fn start_service_as_election_leader( sub_tasks.push(stream_manager.start_auto_parallelism_monitor()); } } - let (idle_send, idle_recv) = tokio::sync::oneshot::channel(); - sub_tasks.push(IdleManager::start_idle_checker( + + let _idle_checker_handle = IdleManager::start_idle_checker( env.idle_manager_ref(), Duration::from_secs(30), - idle_send, - )); + shutdown.clone(), + ); let (abort_sender, abort_recv) = tokio::sync::oneshot::channel(); let notification_mgr = env.notification_manager_ref(); let stream_abort_handler = tokio::spawn(async move { - abort_recv.await.unwrap(); + let _ = abort_recv.await; notification_mgr.abort_all().await; compactor_manager.abort_all_compactors(); }); @@ -762,33 +745,6 @@ pub async fn start_service_as_election_leader( sub_tasks.push(pair); } - let shutdown_all = async move { - let mut handles = Vec::with_capacity(sub_tasks.len()); - - for (join_handle, shutdown_sender) in sub_tasks { - if let Err(_err) = shutdown_sender.send(()) { - continue; - } - - handles.push(join_handle); - } - - // The barrier manager can't be shutdown gracefully if it's under recovering, try to - // abort it using timeout. - match tokio::time::timeout(Duration::from_secs(1), join_all(handles)).await { - Ok(results) => { - for result in results { - if result.is_err() { - tracing::warn!("Failed to join shutdown"); - } - } - } - Err(_e) => { - tracing::warn!("Join shutdown timeout"); - } - } - }; - tracing::info!("Assigned cluster id {:?}", *env.cluster_id()); tracing::info!("Starting meta services"); @@ -832,28 +788,15 @@ pub async fn start_service_as_election_leader( tcp_nodelay: true, keepalive_duration: None, }, - async move { - tokio::select! { - res = svc_shutdown_rx.changed() => { - match res { - Ok(_) => tracing::info!("Shutting down services"), - Err(_) => tracing::error!("Service shutdown receiver dropped") - } - shutdown_all.await; - }, - _ = idle_recv => { - shutdown_all.await; - }, - } - }, + shutdown.clone().cancelled_owned(), ); started::set(); - server.await; + let _server_handle = tokio::spawn(server); - #[cfg(not(madsim))] - if let Some(dashboard_task) = dashboard_task { - dashboard_task.abort(); - } + // Wait for the shutdown signal. + shutdown.cancelled().await; + // TODO(shutdown): may warn user if there's any other node still running in the cluster. + // TODO(shutdown): do we have any other shutdown tasks? Ok(()) } diff --git a/src/meta/service/src/meta_member_service.rs b/src/meta/service/src/meta_member_service.rs index b8f5d9ebf92c4..946337d248485 100644 --- a/src/meta/service/src/meta_member_service.rs +++ b/src/meta/service/src/meta_member_service.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use either::Either; use risingwave_common::util::addr::HostAddr; use risingwave_meta::rpc::ElectionClientRef; use risingwave_pb::common::HostAddress; @@ -20,17 +19,14 @@ use risingwave_pb::meta::meta_member_service_server::MetaMemberService; use risingwave_pb::meta::{MembersRequest, MembersResponse, MetaMember}; use tonic::{Request, Response, Status}; -use crate::AddressInfo; #[derive(Clone)] pub struct MetaMemberServiceImpl { - election_client_or_self: Either, + election_client: ElectionClientRef, } impl MetaMemberServiceImpl { - pub fn new(election_client_or_self: Either) -> Self { - MetaMemberServiceImpl { - election_client_or_self, - } + pub fn new(election_client: ElectionClientRef) -> Self { + MetaMemberServiceImpl { election_client } } } @@ -41,39 +37,20 @@ impl MetaMemberService for MetaMemberServiceImpl { &self, _request: Request, ) -> Result, Status> { - let members = match &self.election_client_or_self { - Either::Left(election_client) => { - let mut members = vec![]; - for member in election_client.get_members().await? { - let host_addr = member - .id - .parse::() - .map_err(|err| Status::from_error(err.into()))?; - members.push(MetaMember { - address: Some(HostAddress { - host: host_addr.host, - port: host_addr.port.into(), - }), - is_leader: member.is_leader, - }) - } - - members - } - Either::Right(self_as_leader) => { - let host_addr = self_as_leader - .advertise_addr - .parse::() - .map_err(|err| Status::from_error(err.into()))?; - vec![MetaMember { - address: Some(HostAddress { - host: host_addr.host, - port: host_addr.port.into(), - }), - is_leader: true, - }] - } - }; + let mut members = vec![]; + for member in self.election_client.get_members().await? { + let host_addr = member + .id + .parse::() + .map_err(|err| Status::from_error(err.into()))?; + members.push(MetaMember { + address: Some(HostAddress { + host: host_addr.host, + port: host_addr.port.into(), + }), + is_leader: member.is_leader, + }) + } Ok(Response::new(MembersResponse { members })) } diff --git a/src/meta/src/manager/idle.rs b/src/meta/src/manager/idle.rs index afa9fbf860932..431575e665b88 100644 --- a/src/meta/src/manager/idle.rs +++ b/src/meta/src/manager/idle.rs @@ -16,7 +16,7 @@ use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Arc; use std::time::{Duration, Instant}; -use tokio::sync::oneshot::Sender; +use risingwave_common::util::tokio_util::sync::CancellationToken; use tokio::task::JoinHandle; /// `IdleManager` keeps track of latest activity and report whether the meta service has been @@ -77,24 +77,17 @@ impl IdleManager { pub fn start_idle_checker( idle_manager: IdleManagerRef, check_interval: Duration, - idle_send: tokio::sync::oneshot::Sender<()>, - ) -> (JoinHandle<()>, Sender<()>) { + shutdown: CancellationToken, + ) -> JoinHandle<()> { let dur = idle_manager.get_config_max_idle(); if !dur.is_zero() { tracing::warn!("--dangerous-max-idle-secs is set. The meta server will be automatically stopped after idle for {:?}.", dur) } - let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); - let join_handle = tokio::spawn(async move { + tokio::spawn(async move { let mut min_interval = tokio::time::interval(check_interval); loop { - tokio::select! { - _ = min_interval.tick() => {}, - _ = &mut shutdown_rx => { - tracing::info!("Idle checker is stopped"); - return; - } - } + min_interval.tick().await; if idle_manager.is_exceeding_max_idle() { break; } @@ -104,9 +97,9 @@ impl IdleManager { idle_manager.get_config_max_idle() ); tracing::warn!("Idle checker is shutting down the server"); - let _ = idle_send.send(()); - }); - (join_handle, shutdown_tx) + + shutdown.cancel(); + }) } } diff --git a/src/meta/src/rpc/election/dummy.rs b/src/meta/src/rpc/election/dummy.rs new file mode 100644 index 0000000000000..567958dd08600 --- /dev/null +++ b/src/meta/src/rpc/election/dummy.rs @@ -0,0 +1,73 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use tokio::sync::watch::{self, Receiver, Sender}; + +use crate::{ElectionClient, ElectionMember, MetaResult}; + +/// A dummy implementation of [`ElectionClient`] for scenarios where only one meta node is running, +/// typically for testing purposes such as an in-memory meta store. +/// +/// This can be used to unify the code paths no matter there's HA or not. +pub struct DummyElectionClient { + id: String, + + /// A dummy watcher that never changes, indicating we are always the leader. + dummy_watcher: Sender, +} + +impl DummyElectionClient { + pub fn new(id: String) -> Self { + Self { + id, + dummy_watcher: watch::channel(true).0, + } + } + + fn self_member(&self) -> ElectionMember { + ElectionMember { + id: self.id.clone(), + is_leader: true, + } + } +} + +#[async_trait::async_trait] +impl ElectionClient for DummyElectionClient { + fn id(&self) -> MetaResult { + Ok(self.id.clone()) + } + + async fn run_once(&self, _ttl: i64, mut stop: Receiver<()>) -> MetaResult<()> { + // Only exit when the stop signal is received. + let _ = stop.changed().await; + Ok(()) + } + + fn subscribe(&self) -> Receiver { + self.dummy_watcher.subscribe() + } + + async fn leader(&self) -> MetaResult> { + Ok(Some(self.self_member())) + } + + async fn get_members(&self) -> MetaResult> { + Ok(vec![self.self_member()]) + } + + fn is_leader(&self) -> bool { + true + } +} diff --git a/src/meta/src/rpc/election/etcd.rs b/src/meta/src/rpc/election/etcd.rs index 96b16f537356e..6834591764360 100644 --- a/src/meta/src/rpc/election/etcd.rs +++ b/src/meta/src/rpc/election/etcd.rs @@ -36,7 +36,7 @@ pub struct EtcdElectionClient { #[async_trait::async_trait] impl ElectionClient for EtcdElectionClient { - async fn is_leader(&self) -> bool { + fn is_leader(&self) -> bool { *self.is_leader_sender.borrow() } @@ -404,7 +404,7 @@ mod tests { let leader = new_followers.into_iter().next().unwrap(); - assert!(leader.1.is_leader().await); + assert!(leader.1.is_leader()); } #[tokio::test] @@ -434,7 +434,7 @@ mod tests { let mut leaders = vec![]; let mut followers = vec![]; for (sender, client) in clients { - if client.is_leader().await { + if client.is_leader() { leaders.push((sender, client)); } else { followers.push((sender, client)); @@ -476,7 +476,7 @@ mod tests { } for client in &clients { - assert!(!client.1.is_leader().await); + assert!(!client.1.is_leader()); } for (stop_sender, client) in &clients { diff --git a/src/meta/src/rpc/election/mod.rs b/src/meta/src/rpc/election/mod.rs index 0c65d497b677e..9b34d19ce2244 100644 --- a/src/meta/src/rpc/election/mod.rs +++ b/src/meta/src/rpc/election/mod.rs @@ -11,6 +11,8 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. + +pub mod dummy; pub mod etcd; pub mod sql; @@ -34,9 +36,12 @@ pub trait ElectionClient: Send + Sync + 'static { } fn id(&self) -> MetaResult; + /// Run the long-running election process. + /// + /// Returns when the leader status is lost, or the stop signal is received. async fn run_once(&self, ttl: i64, stop: Receiver<()>) -> MetaResult<()>; fn subscribe(&self) -> Receiver; async fn leader(&self) -> MetaResult>; async fn get_members(&self) -> MetaResult>; - async fn is_leader(&self) -> bool; + fn is_leader(&self) -> bool; } diff --git a/src/meta/src/rpc/election/sql.rs b/src/meta/src/rpc/election/sql.rs index 9ec5bd199cf76..62694aaa3ded3 100644 --- a/src/meta/src/rpc/election/sql.rs +++ b/src/meta/src/rpc/election/sql.rs @@ -781,7 +781,7 @@ where .collect()) } - async fn is_leader(&self) -> bool { + fn is_leader(&self) -> bool { *self.is_leader_sender.borrow() } } @@ -842,7 +842,7 @@ mod tests { loop { receiver.changed().await.unwrap(); if *receiver.borrow() { - assert!(sql_election_client.is_leader().await); + assert!(sql_election_client.is_leader()); break; } } @@ -874,7 +874,7 @@ mod tests { let mut is_leaders = vec![]; for client in clients { - is_leaders.push(client.is_leader().await); + is_leaders.push(client.is_leader()); } assert!(is_leaders.iter().filter(|&x| *x).count() <= 1); diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 28520720e98fe..618ac9c26436c 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -717,7 +717,7 @@ impl Default for MetaMetrics { pub fn start_worker_info_monitor( metadata_manager: MetadataManager, - election_client: Option, + election_client: ElectionClientRef, interval: Duration, meta_metrics: Arc, ) -> (JoinHandle<()>, Sender<()>) { @@ -754,9 +754,7 @@ pub fn start_worker_info_monitor( .with_label_values(&[(worker_type.as_str_name())]) .set(worker_num as i64); } - if let Some(client) = &election_client - && let Ok(meta_members) = client.get_members().await - { + if let Ok(meta_members) = election_client.get_members().await { meta_metrics .worker_num .with_label_values(&[WorkerType::Meta.as_str_name()]) diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 195715e130877..de5e747624098 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -154,7 +154,7 @@ pub async fn start_meta_node(listen_addr: String, state_store: String, config_pa "enable_compaction_deterministic should be set" ); - risingwave_meta_node::start(meta_opts).await + risingwave_meta_node::start(meta_opts, CancellationToken::new() /* dummy */).await } async fn start_compactor_node( diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index c733288757ba2..8e2ffece2fed9 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -455,7 +455,12 @@ impl Cluster { .create_node() .name(format!("meta-{i}")) .ip([192, 168, 1, i as u8].into()) - .init(move || risingwave_meta_node::start(opts.clone())) + .init(move || { + risingwave_meta_node::start( + opts.clone(), + CancellationToken::new(), // dummy + ) + }) .build(); } From 16b358a25c79db20fa1999e4deb9d9ebe468149c Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 10 Jul 2024 20:18:44 +0800 Subject: [PATCH 14/20] doc(dev-guide): reorg design docs (#17649) Signed-off-by: xxchan --- docs/dev/src/SUMMARY.md | 24 +++++++++---------- docs/dev/src/connector/intro.md | 3 +++ .../data-source.md => connector/source.md} | 16 +++++-------- .../dev/src/design/data-model-and-encoding.md | 9 ++----- docs/dev/src/design/meta-service.md | 10 +------- docs/dev/src/design/multi-object-store.md | 3 +-- docs/dev/src/design/mv-on-mv.md | 2 +- docs/dev/src/design/shared-buffer.md | 10 +------- docs/dev/src/design/state-store-overview.md | 14 +---------- docs/dev/src/design/streaming-overview.md | 12 +--------- 10 files changed, 28 insertions(+), 75 deletions(-) rename docs/dev/src/{design/data-source.md => connector/source.md} (89%) diff --git a/docs/dev/src/SUMMARY.md b/docs/dev/src/SUMMARY.md index 382cc0f80fec8..37a1484f0f36b 100644 --- a/docs/dev/src/SUMMARY.md +++ b/docs/dev/src/SUMMARY.md @@ -26,29 +26,27 @@ # Specialized topics - [Develop Connectors](./connector/intro.md) + - [Source](./connector/source.md) - [Continuous Integration](./ci.md) --- # Design docs - - - [Architecture Design](./design/architecture-design.md) -- [An Overview of RisingWave Streaming Engine](./design/streaming-overview.md) -- [An Overview of RisingWave State Store](./design/state-store-overview.md) +- [Streaming Engine](./design/streaming-overview.md) + - [Checkpoint](./design/checkpoint.md) + - [Aggregation](./design/aggregation.md) + - [MView on Top of MView](./design/mv-on-mv.md) + - [Backfill](./design/backfill.md) +- [State Store](./design/state-store-overview.md) + - [Shared Buffer](./design/shared-buffer.md) + - [Relational Table](./design/relational-table.md) + - [Multiple Object Storage Backends](./design/multi-object-store.md) - [Meta Service](./design/meta-service.md) -- [Create MView on Top of MView](./design/mv-on-mv.md) -- [Checkpoint](./design/checkpoint.md) -- [Design of Data Source](./design/data-source.md) - [Data Model and Encoding](./design/data-model-and-encoding.md) -- [Design of Batch Local Execution Mode](./design/batch-local-execution-mode.md) +- [Batch Local Execution Mode](./design/batch-local-execution-mode.md) - [Consistent Hash](./design/consistent-hash.md) -- [Build RisingWave with Multiple Object Storage Backends](./design/multi-object-store.md) -- [Backfill](./design/backfill.md) -- [Aggregation](./design/aggregation.md) -- [Shared Buffer](./design/shared-buffer.md) -- [Relational Table](./design/relational-table.md) - [Keys](./design/keys.md) + ## Components RisingWave's data source covers four parts: connectors, enumerators, ConnectorSource and SourceExecutor. @@ -41,7 +37,7 @@ pub trait SplitReader: Sized { ### Enumerators -`Enumerator` periodically requests upstream to discover changes in splits, and in most cases the number of splits only increases. The enumerator is a separate task that runs on the [meta](./meta-service.md). If the upstream split changes, the enumerator notifies the connector by means of config change to change the subscription relationship. +`Enumerator` periodically requests upstream to discover changes in splits, and in most cases the number of splits only increases. The enumerator is a separate task that runs on the [meta](../design/meta-service.md). If the upstream split changes, the enumerator notifies the connector by means of config change to change the subscription relationship. All enumerators need to implement the following trait. diff --git a/docs/dev/src/design/data-model-and-encoding.md b/docs/dev/src/design/data-model-and-encoding.md index 580a885bc79f9..c2a70c40909c6 100644 --- a/docs/dev/src/design/data-model-and-encoding.md +++ b/docs/dev/src/design/data-model-and-encoding.md @@ -1,11 +1,6 @@ # Data Model and Encoding -- [Data Model and Encoding](#data-model-and-encoding) - - [Data Model](#data-model) - - [In-Memory Encoding](#in-memory-encoding) - - [On-Disk Encoding](#on-disk-encoding) - - + ## Data Model @@ -26,7 +21,7 @@ Primitive data types: - Strings: `VARCHAR` - Temporals: `DATE`, `TIMESTAMP`, `TIMESTAMP WITH TIME ZONE`, `TIME`, `INTERVAL` -Composite data types (WIP): +Composite data types: - `Struct`: A structure with a list of named, strong-typed fields. - `List`: A variable-length list of values with same data type. diff --git a/docs/dev/src/design/meta-service.md b/docs/dev/src/design/meta-service.md index bab66c6077e9c..4ec7b01d10e4f 100644 --- a/docs/dev/src/design/meta-service.md +++ b/docs/dev/src/design/meta-service.md @@ -1,14 +1,6 @@ # Meta Service -- [Meta Service](#meta-service) - - [Background](#background) - - [Meta Store](#meta-store) - - [Types of Metadata](#types-of-metadata) - - [Catalog](#catalog) - - [Storage](#storage) - - [Push on Updates](#push-on-updates) - - + ## Background diff --git a/docs/dev/src/design/multi-object-store.md b/docs/dev/src/design/multi-object-store.md index 4ac6ef1a41805..699cd036f419d 100644 --- a/docs/dev/src/design/multi-object-store.md +++ b/docs/dev/src/design/multi-object-store.md @@ -1,7 +1,6 @@ # Build RisingWave with Multiple Object Storage Backends - - + ## Overview As a cloud-neutral database, RisingWave supports running on different (object) storage backends. Currently, these storage products include diff --git a/docs/dev/src/design/mv-on-mv.md b/docs/dev/src/design/mv-on-mv.md index 8f810a6a62f33..d1091c0c0ac82 100644 --- a/docs/dev/src/design/mv-on-mv.md +++ b/docs/dev/src/design/mv-on-mv.md @@ -1,4 +1,4 @@ -# Create MView on Top of MView +# MView on Top of MView ## Background diff --git a/docs/dev/src/design/shared-buffer.md b/docs/dev/src/design/shared-buffer.md index 7c7dac8f06e2d..846d8bd1d064a 100644 --- a/docs/dev/src/design/shared-buffer.md +++ b/docs/dev/src/design/shared-buffer.md @@ -1,14 +1,6 @@ # The Hummock Shared Buffer -Table of contents: - -- [Introduction](#introduction) -- [Part 1: Async Checkpoint](#part-1-async-checkpoint) - - [Write Path](#write-path) - - [Read Path](#read-path) -- [Part 2: Write Anytime / Async Flush](#part-2-write-anytime--async-flush) - - [A New Merge Iterator](#a-new-merge-iterator) - - [Considerations](#considerations) + ## Introduction diff --git a/docs/dev/src/design/state-store-overview.md b/docs/dev/src/design/state-store-overview.md index be8f3491550fc..a11d9528d2586 100644 --- a/docs/dev/src/design/state-store-overview.md +++ b/docs/dev/src/design/state-store-overview.md @@ -1,18 +1,6 @@ # An Overview of RisingWave State Store -- [An Overview of RisingWave State Store](#an-overview-of-risingwave-state-store) - - [Overview](#overview) - - [Architecture](#architecture) - - [The Hummock User API](#the-hummock-user-api) - - [Hummock Internals](#hummock-internals) - - [Storage Format](#storage-format) - - [Write Path](#write-path) - - [Read Path](#read-path) - - [Compaction](#compaction) - - [Transaction Management with Hummock Manager](#transaction-management-with-hummock-manager) - - [Checkpointing in Streaming](#checkpointing-in-streaming) - - + ## Overview diff --git a/docs/dev/src/design/streaming-overview.md b/docs/dev/src/design/streaming-overview.md index b24eeaba51cb9..d4a80c2a66c48 100644 --- a/docs/dev/src/design/streaming-overview.md +++ b/docs/dev/src/design/streaming-overview.md @@ -1,16 +1,6 @@ # An Overview of the RisingWave Streaming Engine -- [An Overview of the RisingWave Streaming Engine](#an-overview-of-risingwave-streaming-engine) - - [Overview](#overview) - - [Architecture](#architecture) - - [Actors, executors, and states](#actors-executors-and-states) - - [Actors](#actors) - - [Executors](#executors) - - [Checkpoint, Consistency, and Fault tolerance](#checkpoint-consistency-and-fault-tolerance) - - [Barrier based checkpoint](#barrier-based-checkpoint) - - [Fault tolerance](#fault-tolerance) - - + ## Overview From ab93f9effea4f0b3ec76a82bc56cddbf4410334c Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 11 Jul 2024 11:43:40 +0800 Subject: [PATCH 15/20] feat(storage): skip refill of trival_move version delta (#17623) --- .../hummock_sdk/src/compaction_group/hummock_version_ext.rs | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 1aba344e166a0..e8e9dcdc3680a 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -400,6 +400,12 @@ impl HummockVersion { pub fn build_sst_delta_infos(&self, version_delta: &HummockVersionDelta) -> Vec { let mut infos = vec![]; + // Skip trivial move delta for refiller + // The trivial move task only changes the position of the sst in the lsm, it does not modify the object information corresponding to the sst, and does not need to re-execute the refill. + if version_delta.trivial_move { + return infos; + } + for (group_id, group_deltas) in &version_delta.group_deltas { let mut info = SstDeltaInfo::default(); From d5fc35c61a7384a62499c309a114e2efa63662b5 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Thu, 11 Jul 2024 13:08:14 +0800 Subject: [PATCH 16/20] fix(storage): always clean table watermark below table safe epoch (#17656) --- src/ctl/src/cmd_impl/meta/migration.rs | 2 +- .../rw_catalog/rw_hummock_version_deltas.rs | 2 +- src/meta/src/hummock/manager/compaction.rs | 12 ++- src/meta/src/hummock/model/ext/hummock.rs | 2 +- .../compaction_group/hummock_version_ext.rs | 100 +++++++++--------- src/storage/hummock_sdk/src/version.rs | 10 +- 6 files changed, 70 insertions(+), 58 deletions(-) diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 614fcd6be225c..0f7a5e7e0c428 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -748,7 +748,7 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an id: Set(vd.id as _), prev_id: Set(vd.prev_id as _), max_committed_epoch: Set(vd.max_committed_epoch as _), - safe_epoch: Set(vd.safe_epoch as _), + safe_epoch: Set(vd.visible_table_safe_epoch() as _), trivial_move: Set(vd.trivial_move), full_version_delta: Set((&vd.to_protobuf()).into()), }) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs index a46312b817c8d..2608ca868119b 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version_deltas.rs @@ -39,7 +39,7 @@ async fn read(reader: &SysCatalogReaderImpl) -> Result HummockVersionTransaction<'a> { })), }; group_deltas.push(group_delta); - version_delta.safe_epoch = std::cmp::max( + let new_visible_table_safe_epoch = std::cmp::max( version_delta.latest_version().visible_table_safe_epoch(), compact_task.watermark, ); - if version_delta.latest_version().visible_table_safe_epoch() < version_delta.safe_epoch { + version_delta.set_safe_epoch(new_visible_table_safe_epoch); + if version_delta.latest_version().visible_table_safe_epoch() < new_visible_table_safe_epoch + { version_delta.with_latest_version(|version, version_delta| { for (table_id, info) in version.state_table_info.info() { - let new_safe_epoch = min(version_delta.safe_epoch, info.committed_epoch); + let new_safe_epoch = min(new_visible_table_safe_epoch, info.committed_epoch); if new_safe_epoch > info.safe_epoch { - if new_safe_epoch != version_delta.safe_epoch { + if new_safe_epoch != version_delta.visible_table_safe_epoch() { warn!( new_safe_epoch, committed_epoch = info.committed_epoch, - global_safe_epoch = version_delta.safe_epoch, + global_safe_epoch = new_visible_table_safe_epoch, table_id = table_id.table_id, "table has different safe epoch to global" ); diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index 5cd1eb08402ec..5becdcc9b9850 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -218,7 +218,7 @@ impl Transactional for HummockVersionDelta { id: Set(self.id as _), prev_id: Set(self.prev_id as _), max_committed_epoch: Set(self.max_committed_epoch as _), - safe_epoch: Set(self.safe_epoch as _), + safe_epoch: Set(self.visible_table_safe_epoch() as _), trivial_move: Set(self.trivial_move), full_version_delta: Set(FullVersionDelta::from(&self.to_protobuf())), }; diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index e8e9dcdc3680a..760941ba95cfc 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -628,7 +628,7 @@ impl HummockVersion { } self.id = version_delta.id; self.max_committed_epoch = version_delta.max_committed_epoch; - self.set_safe_epoch(version_delta.safe_epoch); + self.set_safe_epoch(version_delta.visible_table_safe_epoch()); // apply to table watermark @@ -651,20 +651,22 @@ impl HummockVersion { } } for (table_id, table_watermarks) in &self.table_watermarks { - if let Some(table_delta) = version_delta.state_table_info_delta.get(table_id) - && let Some(Some(prev_table)) = changed_table_info.get(table_id) - && table_delta.safe_epoch > prev_table.safe_epoch + let safe_epoch = if let Some(state_table_info) = + self.state_table_info.info().get(table_id) + && let Some((oldest_epoch, _)) = table_watermarks.watermarks.first() + && state_table_info.safe_epoch > *oldest_epoch { // safe epoch has progressed, need further clear. + state_table_info.safe_epoch } else { - // safe epoch not progressed. No need to truncate + // safe epoch not progressed or the table has been removed. No need to truncate continue; - } + }; let table_watermarks = modified_table_watermarks .entry(*table_id) .or_insert_with(|| Some((**table_watermarks).clone())); if let Some(table_watermarks) = table_watermarks { - table_watermarks.clear_stale_epoch_watermark(version_delta.safe_epoch); + table_watermarks.clear_stale_epoch_watermark(safe_epoch); } } // apply the staging table watermark to hummock version @@ -1402,50 +1404,50 @@ mod tests { ), ), ]); - let version_delta = HummockVersionDelta { - id: 1, - group_deltas: HashMap::from_iter([ - ( - 2, - GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { - group_config: Some(CompactionConfig { - max_level: 6, - ..Default::default() - }), + let mut version_delta = HummockVersionDelta::default(); + version_delta.id = 1; + version_delta.group_deltas = HashMap::from_iter([ + ( + 2, + GroupDeltas { + group_deltas: vec![GroupDelta { + delta_type: Some(DeltaType::GroupConstruct(GroupConstruct { + group_config: Some(CompactionConfig { + max_level: 6, ..Default::default() - })), - }], - }, - ), - ( - 0, - GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::GroupDestroy(GroupDestroy {})), - }], - }, - ), - ( - 1, - GroupDeltas { - group_deltas: vec![GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx: 1, - inserted_table_infos: vec![SstableInfo { - object_id: 1, - sst_id: 1, - ..Default::default() - }], + }), + ..Default::default() + })), + }], + }, + ), + ( + 0, + GroupDeltas { + group_deltas: vec![GroupDelta { + delta_type: Some(DeltaType::GroupDestroy(GroupDestroy {})), + }], + }, + ), + ( + 1, + GroupDeltas { + group_deltas: vec![GroupDelta { + delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { + level_idx: 1, + inserted_table_infos: vec![SstableInfo { + object_id: 1, + sst_id: 1, ..Default::default() - })), - }], - }, - ), - ]), - ..Default::default() - }; + }], + ..Default::default() + })), + }], + }, + ), + ]); + let version_delta = version_delta; + version.apply_version_delta(&version_delta); let mut cg1 = build_initial_compaction_group_levels( 1, diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 0315ee9b8f298..611e177db74d6 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -401,7 +401,7 @@ pub struct HummockVersionDelta { pub prev_id: u64, pub group_deltas: HashMap, pub max_committed_epoch: u64, - pub safe_epoch: u64, + safe_epoch: u64, pub trivial_move: bool, pub new_table_watermarks: HashMap, pub removed_table_ids: HashSet, @@ -537,4 +537,12 @@ impl HummockVersionDelta { })) .collect() } + + pub fn visible_table_safe_epoch(&self) -> u64 { + self.safe_epoch + } + + pub fn set_safe_epoch(&mut self, safe_epoch: u64) { + self.safe_epoch = safe_epoch; + } } From 87a29098e15797df1d6b4e6dbca44804269c88b1 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Thu, 11 Jul 2024 00:28:33 -0500 Subject: [PATCH 17/20] fix(risedev): correct docker schema registry configurations (#17653) --- src/risedevtool/src/bin/risedev-dev.rs | 12 +++++++++--- src/risedevtool/src/task/kafka_service.rs | 5 +++++ src/risedevtool/src/task/schema_registry_service.rs | 10 ++++++++-- 3 files changed, 22 insertions(+), 5 deletions(-) diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 02cef8655db2f..8b9b9493d6acb 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -284,9 +284,15 @@ fn task_main( ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); let mut service = SchemaRegistryService::new(c.clone()); service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; + if c.user_managed { + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; + task.execute(&mut ctx)?; + } else { + let mut task = + risedev::LogReadyCheckTask::new("Server started, listening for requests")?; + task.execute(&mut ctx)?; + } ctx.pb .set_message(format!("schema registry http://{}:{}", c.address, c.port)); } diff --git a/src/risedevtool/src/task/kafka_service.rs b/src/risedevtool/src/task/kafka_service.rs index 7c415b6d9749a..24c6dff2e3e36 100644 --- a/src/risedevtool/src/task/kafka_service.rs +++ b/src/risedevtool/src/task/kafka_service.rs @@ -62,6 +62,11 @@ impl DockerServiceConfig for KafkaConfig { "KAFKA_INTER_BROKER_LISTENER_NAME".to_owned(), "HOST".to_owned(), ), + // https://docs.confluent.io/platform/current/installation/docker/config-reference.html#example-configurations + ( + "KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR".to_owned(), + "1".to_owned(), + ), ("CLUSTER_ID".to_owned(), "RiseDevRiseDevRiseDev1".to_owned()), ] } diff --git a/src/risedevtool/src/task/schema_registry_service.rs b/src/risedevtool/src/task/schema_registry_service.rs index 5c5eba4fa8f35..95a84a562ff07 100644 --- a/src/risedevtool/src/task/schema_registry_service.rs +++ b/src/risedevtool/src/task/schema_registry_service.rs @@ -15,6 +15,9 @@ use super::docker_service::{DockerService, DockerServiceConfig}; use crate::SchemaRegistryConfig; +/// Schema Registry listener port in the container. +const SCHEMA_REGISTRY_LISTENER_PORT: &str = "8081"; + impl DockerServiceConfig for SchemaRegistryConfig { fn id(&self) -> String { self.id.clone() @@ -43,7 +46,7 @@ impl DockerServiceConfig for SchemaRegistryConfig { ("SCHEMA_REGISTRY_HOST_NAME".to_owned(), self.address.clone()), ( "SCHEMA_REGISTRY_LISTENERS".to_owned(), - format!("http://{}:{}", self.address, self.port), + format!("http://{}:{}", "0.0.0.0", SCHEMA_REGISTRY_LISTENER_PORT), ), ( "SCHEMA_REGISTRY_KAFKASTORE_BOOTSTRAP_SERVERS".to_owned(), @@ -53,7 +56,10 @@ impl DockerServiceConfig for SchemaRegistryConfig { } fn ports(&self) -> Vec<(String, String)> { - vec![(self.port.to_string(), "8081".to_owned())] + vec![( + self.port.to_string(), + SCHEMA_REGISTRY_LISTENER_PORT.to_owned(), + )] } fn data_path(&self) -> Option { From 8c5f3e2cc36a9f47fdd8781b628cc4bd1243a469 Mon Sep 17 00:00:00 2001 From: August Date: Thu, 11 Jul 2024 13:47:36 +0800 Subject: [PATCH 18/20] feat(frontend): support async function for captured context and impl pg_is_in_recovery(), rw_recovery_status() (#17641) --- proto/expr.proto | 2 + proto/meta.proto | 14 ++++ src/expr/macro/src/context.rs | 79 ++++++++++++++++--- src/expr/macro/src/lib.rs | 2 + src/frontend/src/binder/expr/function.rs | 3 +- .../src/expr/function_impl/context.rs | 1 + src/frontend/src/expr/function_impl/mod.rs | 1 + .../expr/function_impl/rw_recovery_status.rs | 52 ++++++++++++ src/frontend/src/expr/pure.rs | 4 +- src/frontend/src/meta_client.rs | 8 +- .../src/optimizer/plan_expr_visitor/strong.rs | 2 + src/frontend/src/scheduler/local.rs | 4 +- src/frontend/src/test_utils.rs | 8 +- src/meta/node/src/server.rs | 3 +- src/meta/service/src/cluster_service.rs | 21 ++++- src/meta/src/barrier/mod.rs | 19 ++++- src/rpc_client/src/meta_client.rs | 9 +++ 17 files changed, 211 insertions(+), 21 deletions(-) create mode 100644 src/frontend/src/expr/function_impl/rw_recovery_status.rs diff --git a/proto/expr.proto b/proto/expr.proto index 3bb54fae6f0bb..0dc1a96d78617 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -307,6 +307,8 @@ message ExprNode { HAS_TABLE_PRIVILEGE = 2407; HAS_ANY_COLUMN_PRIVILEGE = 2408; HAS_SCHEMA_PRIVILEGE = 2409; + PG_IS_IN_RECOVERY = 2411; + RW_RECOVERY_STATUS = 2412; // EXTERNAL ICEBERG_TRANSFORM = 2201; diff --git a/proto/meta.proto b/proto/meta.proto index 0d2a1b8832915..3284887590182 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -362,12 +362,26 @@ message ListAllNodesResponse { repeated common.WorkerNode nodes = 2; } +message GetClusterRecoveryStatusRequest {} + +enum RecoveryStatus { + STATUS_UNSPECIFIED = 0; + STATUS_STARTING = 1; + STATUS_RECOVERING = 2; + STATUS_RUNNING = 3; +} + +message GetClusterRecoveryStatusResponse { + RecoveryStatus status = 1; +} + service ClusterService { rpc AddWorkerNode(AddWorkerNodeRequest) returns (AddWorkerNodeResponse); rpc ActivateWorkerNode(ActivateWorkerNodeRequest) returns (ActivateWorkerNodeResponse); rpc DeleteWorkerNode(DeleteWorkerNodeRequest) returns (DeleteWorkerNodeResponse); rpc UpdateWorkerNodeSchedulability(UpdateWorkerNodeSchedulabilityRequest) returns (UpdateWorkerNodeSchedulabilityResponse); rpc ListAllNodes(ListAllNodesRequest) returns (ListAllNodesResponse); + rpc GetClusterRecoveryStatus(GetClusterRecoveryStatusRequest) returns (GetClusterRecoveryStatusResponse); } enum SubscribeType { diff --git a/src/expr/macro/src/context.rs b/src/expr/macro/src/context.rs index afae444e03c45..47b0638284435 100644 --- a/src/expr/macro/src/context.rs +++ b/src/expr/macro/src/context.rs @@ -16,7 +16,7 @@ use itertools::Itertools; use proc_macro2::TokenStream; use quote::{quote, quote_spanned, ToTokens}; use syn::parse::{Parse, ParseStream}; -use syn::{Error, FnArg, Ident, ItemFn, Result, Token, Type, Visibility}; +use syn::{Error, FnArg, Ident, ItemFn, Pat, PatType, Result, ReturnType, Token, Type, Visibility}; use crate::utils::extend_vis_with_super; @@ -138,7 +138,47 @@ pub(super) fn generate_captured_function( mut user_fn: ItemFn, ) -> Result { let CaptureContextAttr { captures } = attr; - let orig_user_fn = user_fn.clone(); + let is_async = user_fn.sig.asyncness.is_some(); + let mut orig_user_fn = user_fn.clone(); + if is_async { + // Modify the return type to impl Future + Send + 'static for the original function. + let output_type = match &orig_user_fn.sig.output { + ReturnType::Type(_, ty) => ty.clone(), + ReturnType::Default => Box::new(syn::parse_quote!(())), + }; + orig_user_fn.sig.output = ReturnType::Type( + syn::token::RArrow::default(), + Box::new( + syn::parse_quote!(impl std::future::Future + Send + 'static), + ), + ); + orig_user_fn.sig.asyncness = None; + + // Generate clone statements for each input + let input_def: Vec = orig_user_fn + .sig + .inputs + .iter() + .map(|arg| { + if let FnArg::Typed(PatType { pat, .. }) = arg { + if let Pat::Ident(ident) = pat.as_ref() { + let ident_name = &ident.ident; + return quote! { + let #ident_name = #ident_name.clone(); + }; + } + } + quote! {} + }) + .collect(); + + // Wrap the original function body in async move { ... }. + let orig_body = &orig_user_fn.block; + orig_user_fn.block = Box::new(syn::parse_quote!({ + #(#input_def)* + async move { #orig_body } + })); + } let sig = &mut user_fn.sig; @@ -151,6 +191,11 @@ pub(super) fn generate_captured_function( sig.ident = new_name; } + if is_async { + // Ensure the function is async + sig.asyncness = Some(syn::token::Async::default()); + } + // Modify the inputs of sig. let inputs = &mut sig.inputs; if inputs.len() < captures.len() { @@ -200,21 +245,33 @@ pub(super) fn generate_captured_function( )); }; let name = arg.pat.into_token_stream(); - scoped = quote_spanned! { context.span()=> - // TODO: Can we add an assertion here that `&<<#context::Type> as Deref>::Target` is same as `#arg.ty`? - #context::try_with(|#name| { - #scoped - }).flatten() - } + // TODO: Can we add an assertion here that `&<<#context::Type> as Deref>::Target` is same as `#arg.ty`? + scoped = if is_async { + quote_spanned! { context.span()=> + #context::try_with(|#name| { #scoped }) + } + } else { + quote_spanned! { context.span()=> + #context::try_with(|#name| { #scoped }).flatten() + } + }; } scoped }; let new_user_fn = { let vis = user_fn.vis; let sig = user_fn.sig; - quote! { - #vis #sig { - {#new_body}.map_err(Into::into) + if is_async { + quote! { + #vis #sig { + {#new_body}?.await + } + } + } else { + quote! { + #vis #sig { + {#new_body}.map_err(Into::into) + } } } }; diff --git a/src/expr/macro/src/lib.rs b/src/expr/macro/src/lib.rs index ceaef0d6d2406..3a905165c2ee2 100644 --- a/src/expr/macro/src/lib.rs +++ b/src/expr/macro/src/lib.rs @@ -637,6 +637,8 @@ pub fn capture_context(attr: TokenStream, item: TokenStream) -> TokenStream { fn inner(attr: TokenStream, item: TokenStream) -> Result { let attr: CaptureContextAttr = syn::parse(attr)?; let user_fn: ItemFn = syn::parse(item)?; + + // Generate captured function generate_captured_function(attr, user_fn) } match inner(attr, item) { diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 95466021863e4..fbe6b8930a684 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1423,7 +1423,8 @@ impl Binder { ("col_description", raw_call(ExprType::ColDescription)), ("obj_description", raw_literal(ExprImpl::literal_varchar("".to_string()))), ("shobj_description", raw_literal(ExprImpl::literal_varchar("".to_string()))), - ("pg_is_in_recovery", raw_literal(ExprImpl::literal_bool(false))), + ("pg_is_in_recovery", raw_call(ExprType::PgIsInRecovery)), + ("rw_recovery_status", raw_call(ExprType::RwRecoveryStatus)), // internal ("rw_vnode", raw_call(ExprType::Vnode)), ("rw_test_paid_tier", raw_call(ExprType::TestPaidTier)), // for testing purposes diff --git a/src/frontend/src/expr/function_impl/context.rs b/src/frontend/src/expr/function_impl/context.rs index 74cc5001043ad..4aca6219051a1 100644 --- a/src/frontend/src/expr/function_impl/context.rs +++ b/src/frontend/src/expr/function_impl/context.rs @@ -26,4 +26,5 @@ define_context! { pub(super) AUTH_CONTEXT: Arc, pub(super) DB_NAME: String, pub(super) SEARCH_PATH: SearchPath, + pub(super) META_CLIENT: Arc, } diff --git a/src/frontend/src/expr/function_impl/mod.rs b/src/frontend/src/expr/function_impl/mod.rs index a0cff36840b42..4a1e0600dbc0d 100644 --- a/src/frontend/src/expr/function_impl/mod.rs +++ b/src/frontend/src/expr/function_impl/mod.rs @@ -22,3 +22,4 @@ mod pg_get_viewdef; mod pg_index_column_has_property; mod pg_indexes_size; mod pg_relation_size; +mod rw_recovery_status; diff --git a/src/frontend/src/expr/function_impl/rw_recovery_status.rs b/src/frontend/src/expr/function_impl/rw_recovery_status.rs new file mode 100644 index 0000000000000..746dce01e694a --- /dev/null +++ b/src/frontend/src/expr/function_impl/rw_recovery_status.rs @@ -0,0 +1,52 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Write; +use std::sync::Arc; + +use risingwave_expr::{capture_context, function, ExprError, Result}; +use risingwave_pb::meta::RecoveryStatus; + +use super::context::META_CLIENT; +use crate::meta_client::FrontendMetaClient; + +#[function("rw_recovery_status() -> varchar", volatile)] +async fn rw_recovery_status(writer: &mut impl Write) -> Result<()> { + writer + .write_str( + rw_recovery_status_impl_captured() + .await? + .as_str_name() + .strip_prefix("STATUS_") + .unwrap(), + ) + .unwrap(); + Ok(()) +} + +#[function("pg_is_in_recovery() -> boolean", volatile)] +async fn pg_is_in_recovery() -> Result { + let status = rw_recovery_status_impl_captured().await?; + Ok(status != RecoveryStatus::StatusRunning) +} + +#[capture_context(META_CLIENT)] +async fn rw_recovery_status_impl( + meta_client: &Arc, +) -> Result { + meta_client + .get_cluster_recovery_status() + .await + .map_err(|e| ExprError::Internal(e.into())) +} diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index fe87eb6c2280c..1882e70c12b93 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -275,7 +275,9 @@ impl ExprVisitor for ImpureAnalyzer { | Type::HasTablePrivilege | Type::HasAnyColumnPrivilege | Type::HasSchemaPrivilege - | Type::MakeTimestamptz => self.impure = true, + | Type::MakeTimestamptz + | Type::PgIsInRecovery + | Type::RwRecoveryStatus => self.impure = true, } } } diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index c54ebcd0aecb9..bb8a1d803a34f 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -33,7 +33,7 @@ use risingwave_pb::meta::list_fragment_distribution_response::FragmentDistributi use risingwave_pb::meta::list_object_dependencies_response::PbObjectDependencies; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; use risingwave_pb::meta::list_table_fragments_response::TableFragmentInfo; -use risingwave_pb::meta::{EventLog, PbThrottleTarget}; +use risingwave_pb::meta::{EventLog, PbThrottleTarget, RecoveryStatus}; use risingwave_rpc_client::error::Result; use risingwave_rpc_client::{HummockMetaClient, MetaClient}; @@ -133,6 +133,8 @@ pub trait FrontendMetaClient: Send + Sync { min_epoch: u64, max_count: u32, ) -> Result>; + + async fn get_cluster_recovery_status(&self) -> Result; } pub struct FrontendMetaClientImpl(pub MetaClient); @@ -336,4 +338,8 @@ impl FrontendMetaClient for FrontendMetaClientImpl { .list_change_log_epochs(table_id, min_epoch, max_count) .await } + + async fn get_cluster_recovery_status(&self) -> Result { + self.0.get_cluster_recovery_status().await + } } diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index 0cf1cd0a07a35..ed24ba75b524a 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -305,6 +305,8 @@ impl Strong { | ExprType::PgRelationSize | ExprType::PgGetSerialSequence | ExprType::PgIndexColumnHasProperty + | ExprType::PgIsInRecovery + | ExprType::RwRecoveryStatus | ExprType::IcebergTransform | ExprType::HasTablePrivilege | ExprType::HasAnyColumnPrivilege diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 4484d6283f71e..966358da3dedc 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -150,6 +150,7 @@ impl LocalQueryExecution { let search_path = self.session.config().search_path(); let time_zone = self.session.config().timezone(); let timeout = self.timeout; + let meta_client = self.front_env.meta_client_ref(); let sender1 = sender.clone(); let exec = async move { @@ -171,7 +172,7 @@ impl LocalQueryExecution { use risingwave_expr::expr_context::TIME_ZONE; use crate::expr::function_impl::context::{ - AUTH_CONTEXT, CATALOG_READER, DB_NAME, SEARCH_PATH, USER_INFO_READER, + AUTH_CONTEXT, CATALOG_READER, DB_NAME, META_CLIENT, SEARCH_PATH, USER_INFO_READER, }; // box is necessary, otherwise the size of `exec` will double each time it is nested. @@ -181,6 +182,7 @@ impl LocalQueryExecution { let exec = async move { SEARCH_PATH::scope(search_path, exec).await }.boxed(); let exec = async move { AUTH_CONTEXT::scope(auth_context, exec).await }.boxed(); let exec = async move { TIME_ZONE::scope(time_zone, exec).await }.boxed(); + let exec = async move { META_CLIENT::scope(meta_client, exec).await }.boxed(); if let Some(timeout) = timeout { let exec = async move { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 67bf2794f51b0..24453f766c72c 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -55,7 +55,9 @@ use risingwave_pb::meta::list_fragment_distribution_response::FragmentDistributi use risingwave_pb::meta::list_object_dependencies_response::PbObjectDependencies; use risingwave_pb::meta::list_table_fragment_states_response::TableFragmentState; use risingwave_pb::meta::list_table_fragments_response::TableFragmentInfo; -use risingwave_pb::meta::{EventLog, PbTableParallelism, PbThrottleTarget, SystemParams}; +use risingwave_pb::meta::{ + EventLog, PbTableParallelism, PbThrottleTarget, RecoveryStatus, SystemParams, +}; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::{GrantPrivilege, UserInfo}; @@ -1084,6 +1086,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { unimplemented!() } + async fn get_cluster_recovery_status(&self) -> RpcResult { + Ok(RecoveryStatus::StatusRunning) + } + async fn list_change_log_epochs( &self, _table_id: u32, diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 55f069e5e0104..74310c75374e5 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -612,7 +612,8 @@ pub async fn start_service_as_election_leader( scale_controller.clone(), ); - let cluster_srv = ClusterServiceImpl::new(metadata_manager.clone()); + let cluster_srv = + ClusterServiceImpl::new(metadata_manager.clone(), barrier_manager.context().clone()); let stream_srv = StreamServiceImpl::new( env.clone(), barrier_scheduler.clone(), diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index ef7295a6d5136..842dde71efc4b 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_meta::barrier::BarrierManagerRef; use risingwave_meta::manager::MetadataManager; use risingwave_meta_model_v2::WorkerId; use risingwave_pb::common::worker_node::State; @@ -19,7 +20,8 @@ use risingwave_pb::common::HostAddress; use risingwave_pb::meta::cluster_service_server::ClusterService; use risingwave_pb::meta::{ ActivateWorkerNodeRequest, ActivateWorkerNodeResponse, AddWorkerNodeRequest, - AddWorkerNodeResponse, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse, ListAllNodesRequest, + AddWorkerNodeResponse, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse, + GetClusterRecoveryStatusRequest, GetClusterRecoveryStatusResponse, ListAllNodesRequest, ListAllNodesResponse, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse, }; @@ -31,11 +33,15 @@ use crate::MetaError; #[derive(Clone)] pub struct ClusterServiceImpl { metadata_manager: MetadataManager, + barrier_manager: BarrierManagerRef, } impl ClusterServiceImpl { - pub fn new(metadata_manager: MetadataManager) -> Self { - ClusterServiceImpl { metadata_manager } + pub fn new(metadata_manager: MetadataManager, barrier_manager: BarrierManagerRef) -> Self { + ClusterServiceImpl { + metadata_manager, + barrier_manager, + } } } @@ -176,4 +182,13 @@ impl ClusterService for ClusterServiceImpl { nodes: node_list, })) } + + async fn get_cluster_recovery_status( + &self, + _request: Request, + ) -> Result, Status> { + Ok(Response::new(GetClusterRecoveryStatusResponse { + status: self.barrier_manager.get_recovery_status() as _, + })) + } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index bb6737735dd44..e04730bd6143d 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -38,7 +38,7 @@ use risingwave_hummock_sdk::{ExtendedSstableInfo, HummockSstableObjectId}; use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::meta::subscribe_response::{Info, Operation}; -use risingwave_pb::meta::PausedReason; +use risingwave_pb::meta::{PausedReason, PbRecoveryStatus}; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; use risingwave_pb::stream_service::BarrierCompleteResponse; use thiserror_ext::AsReport; @@ -141,6 +141,19 @@ struct Scheduled { checkpoint: bool, } +impl From<&BarrierManagerStatus> for PbRecoveryStatus { + fn from(status: &BarrierManagerStatus) -> Self { + match status { + BarrierManagerStatus::Starting => Self::StatusStarting, + BarrierManagerStatus::Recovering(reason) => match reason { + RecoveryReason::Bootstrap => Self::StatusStarting, + RecoveryReason::Failover(_) | RecoveryReason::Adhoc => Self::StatusRecovering, + }, + BarrierManagerStatus::Running => Self::StatusRunning, + } + } +} + #[derive(Clone)] pub struct GlobalBarrierManagerContext { status: Arc>, @@ -1069,6 +1082,10 @@ impl GlobalBarrierManagerContext { } } + pub fn get_recovery_status(&self) -> PbRecoveryStatus { + (&**self.status.load()).into() + } + /// Set barrier manager status. fn set_status(&self, new_status: BarrierManagerStatus) { self.status.store(Arc::new(new_status)); diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index cd295abc9ba39..76d5666c4ec43 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -940,6 +940,14 @@ impl MetaClient { Ok(resp) } + pub async fn get_cluster_recovery_status(&self) -> Result { + let resp = self + .inner + .get_cluster_recovery_status(GetClusterRecoveryStatusRequest {}) + .await?; + Ok(resp.get_status().unwrap()) + } + pub async fn get_cluster_info(&self) -> Result { let request = GetClusterInfoRequest {}; let resp = self.inner.get_cluster_info(request).await?; @@ -1941,6 +1949,7 @@ macro_rules! for_all_meta_rpc { ,{ cluster_client, delete_worker_node, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse } ,{ cluster_client, update_worker_node_schedulability, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse } ,{ cluster_client, list_all_nodes, ListAllNodesRequest, ListAllNodesResponse } + ,{ cluster_client, get_cluster_recovery_status, GetClusterRecoveryStatusRequest, GetClusterRecoveryStatusResponse } ,{ heartbeat_client, heartbeat, HeartbeatRequest, HeartbeatResponse } ,{ stream_client, flush, FlushRequest, FlushResponse } ,{ stream_client, pause, PauseRequest, PauseResponse } From df5cc5fc5b42dea050ddd5cdf55695a28f6d7586 Mon Sep 17 00:00:00 2001 From: Li0k Date: Thu, 11 Jul 2024 15:05:03 +0800 Subject: [PATCH 19/20] feat(storage): refactor commit_epoch code (#17235) --- proto/stream_service.proto | 3 +- src/meta/src/barrier/mod.rs | 10 +- src/meta/src/hummock/manager/commit_epoch.rs | 347 ++++++------------ src/meta/src/hummock/manager/context.rs | 6 +- src/meta/src/hummock/manager/tests.rs | 59 ++- src/meta/src/hummock/manager/transaction.rs | 79 +++- src/meta/src/hummock/manager/utils.rs | 1 + .../src/hummock/mock_hummock_meta_client.rs | 6 +- src/meta/src/hummock/test_utils.rs | 7 +- .../compaction_group/hummock_version_ext.rs | 47 +-- src/storage/hummock_sdk/src/lib.rs | 67 +--- .../compactor/shared_buffer_compact.rs | 11 +- src/storage/src/hummock/sstable/builder.rs | 2 +- src/stream/src/task/barrier_manager.rs | 2 - 14 files changed, 248 insertions(+), 399 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 85b12d8ed5fa1..fd97bde853487 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -71,7 +71,8 @@ message BarrierCompleteResponse { common.Status status = 2; repeated CreateMviewProgress create_mview_progress = 3; message GroupedSstableInfo { - uint64 compaction_group_id = 1; + reserved 1; + reserved "compaction_group_id"; hummock.SstableInfo sst = 2; map table_stats_map = 3; } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index e04730bd6143d..60bbb4cee7fac 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -31,10 +31,11 @@ use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY; use risingwave_common::util::epoch::{Epoch, INVALID_EPOCH}; use risingwave_hummock_sdk::change_log::build_table_change_log_delta; +use risingwave_hummock_sdk::table_stats::from_prost_table_stats_map; use risingwave_hummock_sdk::table_watermark::{ merge_multiple_new_table_watermarks, TableWatermarks, }; -use risingwave_hummock_sdk::{ExtendedSstableInfo, HummockSstableObjectId}; +use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo}; use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -1167,17 +1168,16 @@ fn collect_commit_epoch_info( epochs: &Vec, ) -> CommitEpochInfo { let mut sst_to_worker: HashMap = HashMap::new(); - let mut synced_ssts: Vec = vec![]; + let mut synced_ssts: Vec = vec![]; let mut table_watermarks = Vec::with_capacity(resps.len()); let mut old_value_ssts = Vec::with_capacity(resps.len()); for resp in resps { let ssts_iter = resp.synced_sstables.into_iter().map(|grouped| { let sst_info = grouped.sst.expect("field not None"); sst_to_worker.insert(sst_info.get_object_id(), resp.worker_id); - ExtendedSstableInfo::new( - grouped.compaction_group_id, + LocalSstableInfo::new( sst_info, - grouped.table_stats_map, + from_prost_table_stats_map(grouped.table_stats_map), ) }); synced_ssts.extend(ssts_iter); diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 9a494bc509b4f..3fe407dba6209 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -14,20 +14,20 @@ use std::collections::{BTreeMap, HashMap, HashSet}; -use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::split_sst; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::table_stats::{ - add_prost_table_stats_map, purge_prost_table_stats, PbTableStatsMap, + add_prost_table_stats_map, purge_prost_table_stats, to_prost_table_stats_map, PbTableStatsMap, }; use risingwave_hummock_sdk::table_watermark::TableWatermarks; +use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_hummock_sdk::{ - CompactionGroupId, ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, + CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, }; use risingwave_pb::hummock::compact_task::{self}; -use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version_delta::ChangeLogDelta; -use risingwave_pb::hummock::{GroupDelta, HummockSnapshot, IntraLevelDelta, StateTableInfoDelta}; +use risingwave_pb::hummock::{HummockSnapshot, SstableInfo}; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::transaction::{ @@ -48,7 +48,7 @@ pub struct NewTableFragmentInfo { } pub struct CommitEpochInfo { - pub sstables: Vec, + pub sstables: Vec, pub new_table_watermarks: HashMap, pub sst_to_context: HashMap, pub new_table_fragment_info: Option, @@ -59,7 +59,7 @@ pub struct CommitEpochInfo { impl CommitEpochInfo { pub fn new( - sstables: Vec, + sstables: Vec, new_table_watermarks: HashMap, sst_to_context: HashMap, new_table_fragment_info: Option, @@ -84,7 +84,7 @@ impl HummockManager { pub async fn commit_epoch_for_test( &self, epoch: HummockEpoch, - sstables: Vec>, + sstables: Vec>, sst_to_context: HashMap, ) -> Result<()> { let tables = self @@ -143,7 +143,10 @@ impl HummockManager { // Consume and aggregate table stats. let mut table_stats_change = PbTableStatsMap::default(); for s in &mut sstables { - add_prost_table_stats_map(&mut table_stats_change, &std::mem::take(&mut s.table_stats)); + add_prost_table_stats_map( + &mut table_stats_change, + &to_prost_table_stats_map(std::mem::take(&mut s.table_stats)), + ); } let mut version = HummockVersionTransaction::new( @@ -152,216 +155,47 @@ impl HummockManager { self.env.notification_manager(), &self.metrics, ); - let mut new_version_delta = version.new_delta(); - - new_version_delta.max_committed_epoch = epoch; - new_version_delta.new_table_watermarks = new_table_watermarks; - new_version_delta.change_log_delta = change_log_delta; - let mut table_compaction_group_mapping = new_version_delta - .latest_version() - .state_table_info - .build_table_compaction_group_id(); - - let mut new_table_ids = None; + let state_table_info = &version.latest_version().state_table_info; + let mut table_compaction_group_mapping = state_table_info.build_table_compaction_group_id(); + let mut new_table_ids = HashMap::new(); // Add new table if let Some(new_fragment_table_info) = new_table_fragment_info { - let new_table_ids = new_table_ids.insert(HashMap::new()); if !new_fragment_table_info.internal_table_ids.is_empty() { - for table_id in &new_fragment_table_info.internal_table_ids { - if let Some(info) = new_version_delta - .latest_version() - .state_table_info - .info() - .get(table_id) - { - return Err(Error::CompactionGroup(format!( - "table {} already exist {:?}", - table_id.table_id, info, - ))); - } - } - - for table_id in &new_fragment_table_info.internal_table_ids { - table_compaction_group_mapping - .insert(*table_id, StaticCompactionGroupId::StateDefault as u64); - new_table_ids.insert(*table_id, StaticCompactionGroupId::StateDefault as u64); - } + on_handle_add_new_table( + state_table_info, + &new_fragment_table_info.internal_table_ids, + StaticCompactionGroupId::StateDefault as u64, + &mut table_compaction_group_mapping, + &mut new_table_ids, + )?; } - if let Some(table_id) = new_fragment_table_info.mv_table_id { - if let Some(info) = new_version_delta - .latest_version() - .state_table_info - .info() - .get(&table_id) - { - return Err(Error::CompactionGroup(format!( - "table {} already exist {:?}", - table_id.table_id, info, - ))); - } - let _ = table_compaction_group_mapping - .insert(table_id, StaticCompactionGroupId::MaterializedView as u64); - new_table_ids.insert(table_id, StaticCompactionGroupId::MaterializedView as u64); - } - } - - let mut incorrect_ssts = vec![]; - let mut new_sst_id_number = 0; - for ExtendedSstableInfo { - compaction_group_id, - sst_info: sst, - .. - } in &mut sstables - { - let is_sst_belong_to_group_declared = match new_version_delta - .latest_version() - .levels - .get(compaction_group_id) - { - Some(_compaction_group) => sst.table_ids.iter().all(|t| { - table_compaction_group_mapping - .get(&TableId::new(*t)) - .map(|table_cg_id| table_cg_id == compaction_group_id) - .unwrap_or(false) - }), - None => false, - }; - if !is_sst_belong_to_group_declared { - let mut group_table_ids: BTreeMap<_, Vec> = BTreeMap::new(); - for table_id in sst.get_table_ids() { - match table_compaction_group_mapping.get(&TableId::new(*table_id)) { - Some(compaction_group_id) => { - group_table_ids - .entry(*compaction_group_id) - .or_default() - .push(*table_id); - } - None => { - tracing::warn!( - "table {} in SST {} doesn't belong to any compaction group", - table_id, - sst.get_object_id(), - ); - } - } - } - let is_trivial_adjust = group_table_ids.len() == 1 - && group_table_ids.first_key_value().unwrap().1.len() - == sst.get_table_ids().len(); - if is_trivial_adjust { - *compaction_group_id = *group_table_ids.first_key_value().unwrap().0; - // is_sst_belong_to_group_declared = true; - } else { - new_sst_id_number += group_table_ids.len(); - incorrect_ssts.push((std::mem::take(sst), group_table_ids)); - *compaction_group_id = - StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId; - } - } - } - let mut new_sst_id = next_sstable_object_id(&self.env, new_sst_id_number).await?; - let original_sstables = std::mem::take(&mut sstables); - sstables.reserve_exact(original_sstables.len() - incorrect_ssts.len() + new_sst_id_number); - let mut incorrect_ssts = incorrect_ssts.into_iter(); - for original_sstable in original_sstables { - if original_sstable.compaction_group_id - == StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId - { - let (sst, group_table_ids) = incorrect_ssts.next().unwrap(); - let mut branch_groups = HashMap::new(); - for (group_id, _match_ids) in group_table_ids { - let mut branch_sst = sst.clone(); - branch_sst.sst_id = new_sst_id; - sstables.push(ExtendedSstableInfo::with_compaction_group( - group_id, branch_sst, - )); - branch_groups.insert(group_id, new_sst_id); - new_sst_id += 1; - } - } else { - sstables.push(original_sstable); + if let Some(mv_table_id) = new_fragment_table_info.mv_table_id { + on_handle_add_new_table( + state_table_info, + &[mv_table_id], + StaticCompactionGroupId::MaterializedView as u64, + &mut table_compaction_group_mapping, + &mut new_table_ids, + )?; } } - let mut modified_compaction_groups = vec![]; - // Append SSTs to a new version. - for (compaction_group_id, sstables) in &sstables - .into_iter() - // the sort is stable sort, and will not change the order within compaction group. - // Do a sort so that sst in the same compaction group can be consecutive - .sorted_by_key( - |ExtendedSstableInfo { - compaction_group_id, - .. - }| *compaction_group_id, - ) - .group_by( - |ExtendedSstableInfo { - compaction_group_id, - .. - }| *compaction_group_id, - ) - { - modified_compaction_groups.push(compaction_group_id); - let group_sstables = sstables - .into_iter() - .map(|ExtendedSstableInfo { sst_info, .. }| sst_info) - .collect_vec(); - - let group_deltas = &mut new_version_delta - .group_deltas - .entry(compaction_group_id) - .or_default() - .group_deltas; - let l0_sub_level_id = epoch; - let group_delta = GroupDelta { - delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { - level_idx: 0, - inserted_table_infos: group_sstables.clone(), - l0_sub_level_id, - ..Default::default() - })), - }; - group_deltas.push(group_delta); - } + let commit_sstables = self + .correct_commit_ssts(sstables, &table_compaction_group_mapping) + .await?; - // update state table info - new_version_delta.with_latest_version(|version, delta| { - if let Some(new_table_ids) = new_table_ids { - for (table_id, cg_id) in new_table_ids { - delta.state_table_info_delta.insert( - table_id, - StateTableInfoDelta { - committed_epoch: epoch, - safe_epoch: epoch, - compaction_group_id: cg_id, - }, - ); - } - } - for (table_id, info) in version.state_table_info.info() { - assert!( - delta - .state_table_info_delta - .insert( - *table_id, - StateTableInfoDelta { - committed_epoch: epoch, - safe_epoch: info.safe_epoch, - compaction_group_id: info.compaction_group_id, - } - ) - .is_none(), - "newly added table exists previously: {:?}", - table_id - ); - } - }); + let modified_compaction_groups: Vec<_> = commit_sstables.keys().cloned().collect(); - new_version_delta.pre_apply(); + version.pre_commit_epoch( + epoch, + commit_sstables, + new_table_ids, + new_table_watermarks, + change_log_delta, + ); // TODO: remove the sanity check when supporting partial checkpoint assert_eq!(1, table_committed_epoch.len()); @@ -411,6 +245,7 @@ impl HummockManager { ); table_metrics.inc_write_throughput(stats_value as u64); } + commit_multi_var!(self.meta_store_ref(), version, version_stats)?; let snapshot = HummockSnapshot { @@ -430,36 +265,15 @@ impl HummockManager { ); } + drop(versioning_guard); tracing::trace!("new committed epoch {}", epoch); - let mut table_groups = HashMap::::default(); - for (table_id, info) in versioning.current_version.state_table_info.info() { - table_groups.insert( - table_id.table_id, - versioning - .current_version - .state_table_info - .compaction_group_member_tables() - .get(&info.compaction_group_id) - .expect("should exist") - .len(), - ); - } - drop(versioning_guard); // Don't trigger compactions if we enable deterministic compaction if !self.env.opts.compaction_deterministic_test { // commit_epoch may contains SSTs from any compaction group for id in &modified_compaction_groups { self.try_send_compaction_request(*id, compact_task::TaskType::Dynamic); } - if !table_stats_change.is_empty() { - table_stats_change.retain(|table_id, _| { - table_groups - .get(table_id) - .map(|table_count| *table_count > 1) - .unwrap_or(false) - }); - } if !table_stats_change.is_empty() { self.collect_table_write_throughput(table_stats_change); } @@ -486,4 +300,79 @@ impl HummockManager { } } } + + async fn correct_commit_ssts( + &self, + sstables: Vec, + table_compaction_group_mapping: &HashMap, + ) -> Result>> { + let mut new_sst_id_number = 0; + let mut sst_to_cg_vec = Vec::with_capacity(sstables.len()); + for commit_sst in sstables { + let mut group_table_ids: BTreeMap> = BTreeMap::new(); + for table_id in commit_sst.sst_info.get_table_ids() { + match table_compaction_group_mapping.get(&TableId::new(*table_id)) { + Some(cg_id_from_meta) => { + group_table_ids + .entry(*cg_id_from_meta) + .or_default() + .push(*table_id); + } + None => { + tracing::warn!( + "table {} in SST {} doesn't belong to any compaction group", + table_id, + commit_sst.sst_info.get_object_id(), + ); + } + } + } + + new_sst_id_number += group_table_ids.len(); + sst_to_cg_vec.push((commit_sst, group_table_ids)); + } + + // Generate new SST IDs for each compaction group + // `next_sstable_object_id` will update the global SST ID and reserve the new SST IDs + // So we need to get the new SST ID first and then split the SSTs + let mut new_sst_id = next_sstable_object_id(&self.env, new_sst_id_number).await?; + let mut commit_sstables: BTreeMap> = BTreeMap::new(); + + for (mut sst, group_table_ids) in sst_to_cg_vec { + for (group_id, _match_ids) in group_table_ids { + let branch_sst = split_sst(&mut sst.sst_info, &mut new_sst_id); + commit_sstables + .entry(group_id) + .or_default() + .push(branch_sst); + } + } + + Ok(commit_sstables) + } +} + +fn on_handle_add_new_table( + state_table_info: &HummockVersionStateTableInfo, + table_ids: &[TableId], + compaction_group_id: CompactionGroupId, + table_compaction_group_mapping: &mut HashMap, + new_table_ids: &mut HashMap, +) -> Result<()> { + if table_ids.is_empty() { + return Err(Error::CompactionGroup("empty table ids".to_string())); + } + + for table_id in table_ids { + if let Some(info) = state_table_info.info().get(table_id) { + return Err(Error::CompactionGroup(format!( + "table {} already exist {:?}", + table_id.table_id, info, + ))); + } + table_compaction_group_mapping.insert(*table_id, compaction_group_id); + new_table_ids.insert(*table_id, compaction_group_id); + } + + Ok(()) } diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 982a94fd5f9db..d281ea4f7e4c7 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, + HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, INVALID_VERSION_ID, }; use risingwave_pb::hummock::{ @@ -197,7 +197,7 @@ impl HummockManager { pub async fn commit_epoch_sanity_check( &self, epoch: HummockEpoch, - sstables: &[ExtendedSstableInfo], + sstables: &[LocalSstableInfo], sst_to_context: &HashMap, current_version: &HummockVersion, ) -> Result<()> { @@ -246,7 +246,7 @@ impl HummockManager { }; let sst_infos = sstables .iter() - .map(|ExtendedSstableInfo { sst_info, .. }| sst_info.clone()) + .map(|LocalSstableInfo { sst_info, .. }| sst_info.clone()) .collect_vec(); if compactor .send_event(ResponseEvent::ValidationTask(ValidationTask { diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index c2abc138e4944..5e677248903de 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -29,8 +29,8 @@ use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::table_stats::{to_prost_table_stats_map, TableStats, TableStatsMap}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - CompactionGroupId, ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, - HummockVersionId, LocalSstableInfo, FIRST_VERSION_ID, + CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, + LocalSstableInfo, FIRST_VERSION_ID, }; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; @@ -79,14 +79,8 @@ fn gen_sstable_info(sst_id: u64, idx: usize, table_ids: Vec) -> SstableInfo } } -fn gen_extend_sstable_info( - sst_id: u64, - group_id: u64, - idx: usize, - table_ids: Vec, -) -> ExtendedSstableInfo { - ExtendedSstableInfo { - compaction_group_id: group_id, +fn gen_local_sstable_info(sst_id: u64, idx: usize, table_ids: Vec) -> LocalSstableInfo { + LocalSstableInfo { sst_info: gen_sstable_info(sst_id, idx, table_ids), table_stats: Default::default(), } @@ -1188,7 +1182,7 @@ async fn test_extend_objects_to_delete() { hummock_manager .commit_epoch_for_test( new_epoch, - Vec::::new(), + Vec::::new(), Default::default(), ) .await @@ -1236,7 +1230,6 @@ async fn test_version_stats() { .into_iter() .enumerate() .map(|(idx, table_ids)| LocalSstableInfo { - compaction_group_id: StaticCompactionGroupId::StateDefault as _, sst_info: SstableInfo { object_id: sst_ids[idx], sst_id: sst_ids[idx], @@ -1345,8 +1338,7 @@ async fn test_split_compaction_group_on_commit() { .register_table_ids_for_test(&[(101, 3)]) .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1434,8 +1426,7 @@ async fn test_split_compaction_group_on_demand_basic() { .register_table_ids_for_test(&[(101, 2)]) .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1451,8 +1442,7 @@ async fn test_split_compaction_group_on_demand_basic() { }, table_stats: Default::default(), }; - let sst_2 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_2 = LocalSstableInfo { sst_info: SstableInfo { object_id: 11, sst_id: 11, @@ -1534,8 +1524,7 @@ async fn test_split_compaction_group_on_demand_basic() { async fn test_split_compaction_group_on_demand_non_trivial() { let (_env, hummock_manager, _, worker_node) = setup_compute_env(80).await; let context_id = worker_node.id; - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1620,8 +1609,7 @@ async fn test_split_compaction_group_trivial_expired() { .register_table_ids_for_test(&[(101, 2)]) .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1637,8 +1625,7 @@ async fn test_split_compaction_group_trivial_expired() { }, table_stats: Default::default(), }; - let sst_2 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_2 = LocalSstableInfo { sst_info: SstableInfo { object_id: 11, sst_id: 11, @@ -1788,8 +1775,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1924,8 +1910,7 @@ async fn test_compaction_task_expiration_due_to_split_group() { .register_table_ids_for_test(&[(101, 2)]) .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -1941,8 +1926,7 @@ async fn test_compaction_task_expiration_due_to_split_group() { }, table_stats: Default::default(), }; - let sst_2 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_2 = LocalSstableInfo { sst_info: SstableInfo { object_id: 11, sst_id: 11, @@ -2017,7 +2001,7 @@ async fn test_move_tables_between_compaction_group() { .register_table_ids_for_test(&[(102, 2)]) .await .unwrap(); - let sst_1 = gen_extend_sstable_info(10, 2, 1, vec![100, 101, 102]); + let sst_1 = gen_local_sstable_info(10, 1, vec![100, 101, 102]); hummock_manager .commit_epoch_for_test(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) .await @@ -2040,7 +2024,7 @@ async fn test_move_tables_between_compaction_group() { ) .await .unwrap()); - let sst_2 = gen_extend_sstable_info(14, 2, 1, vec![101, 102]); + let sst_2 = gen_local_sstable_info(14, 1, vec![101, 102]); hummock_manager .commit_epoch_for_test(31, vec![sst_2.clone()], HashMap::from([(14, context_id)])) .await @@ -2189,7 +2173,7 @@ async fn test_partition_level() { .register_table_ids_for_test(&[(101, 2)]) .await .unwrap(); - let sst_1 = gen_extend_sstable_info(10, 2, 1, vec![100, 101]); + let sst_1 = gen_local_sstable_info(10, 1, vec![100, 101]); hummock_manager .commit_epoch_for_test(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) .await @@ -2226,11 +2210,12 @@ async fn test_partition_level() { .len(), 1 ); + let mut global_sst_id = 13; const MB: u64 = 1024 * 1024; let mut selector = default_compaction_selector(); for epoch in 31..100 { - let mut sst = gen_extend_sstable_info(global_sst_id, new_group_id, 10, vec![100]); + let mut sst = gen_local_sstable_info(global_sst_id, 10, vec![100]); sst.sst_info.file_size = 10 * MB; sst.sst_info.uncompressed_file_size = 10 * MB; hummock_manager @@ -2304,8 +2289,7 @@ async fn test_unregister_moved_table() { .register_table_ids_for_test(&[(101, 2)]) .await .unwrap(); - let sst_1 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_1 = LocalSstableInfo { sst_info: SstableInfo { object_id: 10, sst_id: 10, @@ -2321,8 +2305,7 @@ async fn test_unregister_moved_table() { }, table_stats: Default::default(), }; - let sst_2 = ExtendedSstableInfo { - compaction_group_id: 2, + let sst_2 = LocalSstableInfo { sst_info: SstableInfo { object_id: 11, sst_id: 11, diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index c467e95adfdbe..94a3fa4696937 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -12,12 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; +use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; -use risingwave_hummock_sdk::HummockVersionId; -use risingwave_pb::hummock::HummockVersionStats; +use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, HummockVersionId}; +use risingwave_pb::hummock::group_delta::DeltaType; +use risingwave_pb::hummock::hummock_version_delta::ChangeLogDelta; +use risingwave_pb::hummock::{ + GroupDelta, HummockVersionStats, IntraLevelDelta, SstableInfo, StateTableInfoDelta, +}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use crate::manager::NotificationManager; @@ -101,6 +107,73 @@ impl<'a> HummockVersionTransaction<'a> { version.apply_version_delta(&delta); deltas.push(delta); } + + pub(super) fn pre_commit_epoch( + &mut self, + epoch: HummockEpoch, + commit_sstables: BTreeMap>, + new_table_ids: HashMap, + new_table_watermarks: HashMap, + change_log_delta: HashMap, + ) { + let mut new_version_delta = self.new_delta(); + new_version_delta.max_committed_epoch = epoch; + new_version_delta.new_table_watermarks = new_table_watermarks; + new_version_delta.change_log_delta = change_log_delta; + + // Append SSTs to a new version. + for (compaction_group_id, inserted_table_infos) in commit_sstables { + let group_deltas = &mut new_version_delta + .group_deltas + .entry(compaction_group_id) + .or_default() + .group_deltas; + let l0_sub_level_id = epoch; + let group_delta = GroupDelta { + delta_type: Some(DeltaType::IntraLevel(IntraLevelDelta { + level_idx: 0, + inserted_table_infos, + l0_sub_level_id, + ..Default::default() + })), + }; + group_deltas.push(group_delta); + } + + // update state table info + new_version_delta.with_latest_version(|version, delta| { + for (table_id, cg_id) in new_table_ids { + delta.state_table_info_delta.insert( + table_id, + StateTableInfoDelta { + committed_epoch: epoch, + safe_epoch: epoch, + compaction_group_id: cg_id, + }, + ); + } + + for (table_id, info) in version.state_table_info.info() { + assert!( + delta + .state_table_info_delta + .insert( + *table_id, + StateTableInfoDelta { + committed_epoch: epoch, + safe_epoch: info.safe_epoch, + compaction_group_id: info.compaction_group_id, + } + ) + .is_none(), + "newly added table exists previously: {:?}", + table_id + ); + } + }); + + new_version_delta.pre_apply(); + } } impl<'a> InMemValTransaction for HummockVersionTransaction<'a> { diff --git a/src/meta/src/hummock/manager/utils.rs b/src/meta/src/hummock/manager/utils.rs index 3d8cb04546284..41a9d5885f889 100644 --- a/src/meta/src/hummock/manager/utils.rs +++ b/src/meta/src/hummock/manager/utils.rs @@ -52,6 +52,7 @@ macro_rules! commit_multi_var { } }; } + pub(crate) use commit_multi_var; use risingwave_hummock_sdk::SstObjectIdRange; diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index dea226e28b047..1657eb4aeef1b 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -180,11 +180,7 @@ impl HummockMetaClient for MockHummockMetaClient { ); self.hummock_manager .commit_epoch(CommitEpochInfo::new( - sync_result - .uncommitted_ssts - .into_iter() - .map(|sst| sst.into()) - .collect(), + sync_result.uncommitted_ssts, new_table_watermark, sst_to_worker, None, diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 32cbd4932adf6..5ffbe47fa2d9c 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -41,12 +41,7 @@ use crate::rpc::metrics::MetaMetrics; pub fn to_local_sstable_info(ssts: &[SstableInfo]) -> Vec { ssts.iter() - .map(|sst| { - LocalSstableInfo::with_compaction_group( - StaticCompactionGroupId::StateDefault.into(), - sst.clone(), - ) - }) + .map(|sst| LocalSstableInfo::for_test(sst.clone())) .collect_vec() } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 760941ba95cfc..70872b4e2583c 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -295,7 +295,6 @@ impl HummockVersion { group_id: CompactionGroupId, member_table_ids: HashSet, new_sst_start_id: u64, - allow_trivial_split: bool, ) { let mut new_sst_id = new_sst_start_id; if parent_group_id == StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId @@ -329,12 +328,8 @@ impl HummockVersion { } // Remove SST from sub level may result in empty sub level. It will be purged // whenever another compaction task is finished. - let insert_table_infos = split_sst_info_for_level( - &member_table_ids, - allow_trivial_split, - sub_level, - &mut new_sst_id, - ); + let insert_table_infos = + split_sst_info_for_level(&member_table_ids, sub_level, &mut new_sst_id); sub_level .table_infos .extract_if(|sst_info| sst_info.table_ids.is_empty()) @@ -364,12 +359,8 @@ impl HummockVersion { } } for (idx, level) in parent_levels.levels.iter_mut().enumerate() { - let insert_table_infos = split_sst_info_for_level( - &member_table_ids, - allow_trivial_split, - level, - &mut new_sst_id, - ); + let insert_table_infos = + split_sst_info_for_level(&member_table_ids, level, &mut new_sst_id); cur_levels.levels[idx].total_file_size += insert_table_infos .iter() .map(|sst| sst.file_size) @@ -526,7 +517,6 @@ impl HummockVersion { *compaction_group_id, member_table_ids, group_construct.get_new_sst_start_id(), - group_construct.version < CompatibilityVersion::NoTrivialSplit as _, ); } else if let Some(group_change) = &summary.group_table_change { // TODO: may deprecate this branch? This enum variant is not created anywhere @@ -541,7 +531,6 @@ impl HummockVersion { group_change.target_group_id, HashSet::from_iter(group_change.table_ids.clone()), group_change.new_sst_start_id, - group_change.version() == CompatibilityVersion::VersionUnspecified, ); let levels = self @@ -952,7 +941,6 @@ pub fn build_initial_compaction_group_levels( fn split_sst_info_for_level( member_table_ids: &HashSet, - allow_trivial_split: bool, level: &mut Level, new_sst_id: &mut u64, ) -> Vec { @@ -967,22 +955,8 @@ fn split_sst_info_for_level( .cloned() .collect_vec(); if !removed_table_ids.is_empty() { - let is_trivial = - allow_trivial_split && removed_table_ids.len() == sst_info.table_ids.len(); - let mut branch_table_info = sst_info.clone(); - branch_table_info.sst_id = *new_sst_id; - *new_sst_id += 1; - if is_trivial { - // This is a compatibility design. we only clear the table-ids for files which would - // be removed in later code. In the version-delta generated by new - // version meta-service, there will be no trivial split, and we will create - // a reference for every sstable split to two groups. - sst_info.table_ids.clear(); - } else { - sst_info.sst_id = *new_sst_id; - *new_sst_id += 1; - } - insert_table_infos.push(branch_table_info); + let branch_sst = split_sst(sst_info, new_sst_id); + insert_table_infos.push(branch_sst); } } insert_table_infos @@ -1314,6 +1288,15 @@ pub fn validate_version(version: &HummockVersion) -> Vec { res } +pub fn split_sst(sst_info: &mut SstableInfo, new_sst_id: &mut u64) -> SstableInfo { + let mut branch_table_info = sst_info.clone(); + branch_table_info.sst_id = *new_sst_id; + sst_info.sst_id = *new_sst_id + 1; + *new_sst_id += 1; + + branch_table_info +} + #[cfg(test)] mod tests { use std::collections::HashMap; diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 97e1a334dcf98..76be3fd901c4f 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -33,9 +33,8 @@ use risingwave_common::util::epoch::EPOCH_SPILL_TIME_MASK; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::hummock::SstableInfo; -use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; -use crate::table_stats::{to_prost_table_stats_map, PbTableStatsMap, TableStatsMap}; +use crate::table_stats::TableStatsMap; pub mod change_log; pub mod compact; @@ -106,42 +105,20 @@ pub struct SyncResult { #[derive(Debug, Clone)] pub struct LocalSstableInfo { - pub compaction_group_id: CompactionGroupId, pub sst_info: SstableInfo, pub table_stats: TableStatsMap, } impl LocalSstableInfo { - pub fn new( - compaction_group_id: CompactionGroupId, - sst_info: SstableInfo, - table_stats: TableStatsMap, - ) -> Self { + pub fn new(sst_info: SstableInfo, table_stats: TableStatsMap) -> Self { Self { - compaction_group_id, sst_info, table_stats, } } - pub fn with_compaction_group( - compaction_group_id: CompactionGroupId, - sst_info: SstableInfo, - ) -> Self { - Self::new(compaction_group_id, sst_info, TableStatsMap::default()) - } - - pub fn with_stats(sst_info: SstableInfo, table_stats: TableStatsMap) -> Self { - Self::new( - StaticCompactionGroupId::StateDefault as CompactionGroupId, - sst_info, - table_stats, - ) - } - pub fn for_test(sst_info: SstableInfo) -> Self { Self { - compaction_group_id: StaticCompactionGroupId::StateDefault as CompactionGroupId, sst_info, table_stats: Default::default(), } @@ -152,47 +129,9 @@ impl LocalSstableInfo { } } -#[derive(Debug, Clone)] -pub struct ExtendedSstableInfo { - pub compaction_group_id: CompactionGroupId, - pub sst_info: SstableInfo, - pub table_stats: PbTableStatsMap, -} - -impl ExtendedSstableInfo { - pub fn new( - compaction_group_id: CompactionGroupId, - sst_info: SstableInfo, - table_stats: PbTableStatsMap, - ) -> Self { - Self { - compaction_group_id, - sst_info, - table_stats, - } - } - - pub fn with_compaction_group( - compaction_group_id: CompactionGroupId, - sst_info: SstableInfo, - ) -> Self { - Self::new(compaction_group_id, sst_info, PbTableStatsMap::default()) - } -} - -impl From for ExtendedSstableInfo { - fn from(value: LocalSstableInfo) -> Self { - Self { - compaction_group_id: value.compaction_group_id, - sst_info: value.sst_info, - table_stats: to_prost_table_stats_map(value.table_stats), - } - } -} - impl PartialEq for LocalSstableInfo { fn eq(&self, other: &Self) -> bool { - self.compaction_group_id == other.compaction_group_id && self.sst_info == other.sst_info + self.sst_info == other.sst_info } } diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index 52c8cc1710bc2..00680f5906ccf 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -83,7 +83,6 @@ pub async fn compact( let mut new_value_futures = vec![]; for (id, group_payload) in grouped_payload { - let id_copy = id; new_value_futures.push( compact_shared_buffer::( context.clone(), @@ -91,15 +90,7 @@ pub async fn compact( filter_key_extractor_manager.clone(), group_payload, ) - .map_ok(move |results| { - results - .into_iter() - .map(move |mut result| { - result.compaction_group_id = id_copy; - result - }) - .collect_vec() - }) + .map_ok(move |results| results.into_iter()) .instrument_await(format!("shared_buffer_compact_compaction_group {}", id)), ); } diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 9b007f629f74a..cacd49cba92a0 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -527,7 +527,7 @@ impl SstableBuilder { let writer_output = self.writer.finish(meta).await?; Ok(SstableBuilderOutput:: { - sst_info: LocalSstableInfo::with_stats(sst_info, self.table_stats), + sst_info: LocalSstableInfo::new(sst_info, self.table_stats), bloom_filter_size, writer_output, avg_key_size, diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index d20759f017a2b..bc33f434bf222 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -620,11 +620,9 @@ impl LocalBarrierWorker { .into_iter() .map( |LocalSstableInfo { - compaction_group_id, sst_info, table_stats, }| GroupedSstableInfo { - compaction_group_id, sst: Some(sst_info), table_stats_map: to_prost_table_stats_map(table_stats), }, From 5bbb5c83be73a4826cf4c64866576ac9ff2ee81f Mon Sep 17 00:00:00 2001 From: Dylan Date: Thu, 11 Jul 2024 15:30:25 +0800 Subject: [PATCH 20/20] fix(iceberg): add hive-exec jar (#17642) --- java/connector-node/risingwave-connector-test/pom.xml | 5 ----- java/connector-node/risingwave-sink-iceberg/pom.xml | 4 ++++ java/pom.xml | 5 +++++ 3 files changed, 9 insertions(+), 5 deletions(-) diff --git a/java/connector-node/risingwave-connector-test/pom.xml b/java/connector-node/risingwave-connector-test/pom.xml index d3d47b0bc4571..432a16ec1f348 100644 --- a/java/connector-node/risingwave-connector-test/pom.xml +++ b/java/connector-node/risingwave-connector-test/pom.xml @@ -28,11 +28,6 @@ risingwave-sink-deltalake test - - com.risingwave - risingwave-sink-iceberg - test - com.risingwave s3-common diff --git a/java/connector-node/risingwave-sink-iceberg/pom.xml b/java/connector-node/risingwave-sink-iceberg/pom.xml index d8ef3d6db384a..404a895e7506f 100644 --- a/java/connector-node/risingwave-sink-iceberg/pom.xml +++ b/java/connector-node/risingwave-sink-iceberg/pom.xml @@ -74,6 +74,10 @@ org.apache.hive hive-metastore + + org.apache.hive + hive-exec + org.apache.parquet parquet-avro diff --git a/java/pom.xml b/java/pom.xml index 644588c9d6b44..2afd5606a39f0 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -387,6 +387,11 @@ hive-metastore ${hive.version} + + org.apache.hive + hive-exec + ${hive.version} + org.apache.hadoop hadoop-mapreduce-client-core