From 826da0be747305e96df7ab6b4c429fd8b62ea582 Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 17 Sep 2024 20:34:51 +0800 Subject: [PATCH 01/27] feat(storage): notify frontend with more hummock version info --- proto/meta.proto | 2 - .../common_service/src/observer_manager.rs | 1 - src/frontend/src/observer/observer_manager.rs | 32 +-- src/frontend/src/scheduler/snapshot.rs | 127 +++++---- src/meta/service/src/notification_service.rs | 10 +- src/meta/src/barrier/mod.rs | 20 +- src/meta/src/hummock/manager/commit_epoch.rs | 16 +- src/meta/src/hummock/manager/transaction.rs | 19 +- src/storage/hummock_sdk/src/change_log.rs | 4 +- .../compaction_group/hummock_version_ext.rs | 37 ++- .../hummock_sdk/src/frontend_version.rs | 257 ++++++++++++++++++ src/storage/hummock_sdk/src/lib.rs | 2 + 12 files changed, 402 insertions(+), 125 deletions(-) create mode 100644 src/storage/hummock_sdk/src/frontend_version.rs diff --git a/proto/meta.proto b/proto/meta.proto index 98a7f267c0124..8a979a17aa45c 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -433,7 +433,6 @@ message MetaSnapshot { GetSessionParamsResponse session_params = 20; repeated catalog.Secret secrets = 23; repeated common.WorkerNode nodes = 10; - hummock.HummockSnapshot hummock_snapshot = 11; hummock.HummockVersion hummock_version = 12; backup_service.MetaBackupManifestId meta_backup_manifest_id = 14; hummock.WriteLimits hummock_write_limits = 16; @@ -485,7 +484,6 @@ message SubscribeResponse { user.UserInfo user = 11; SetSessionParamRequest session_param = 26; common.WorkerNode node = 13; - hummock.HummockSnapshot hummock_snapshot = 14; hummock.HummockVersionDeltas hummock_version_deltas = 15; MetaSnapshot snapshot = 16; backup_service.MetaBackupManifestId meta_backup_manifest_id = 17; diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index da61118f1ad49..8648644e3a3e2 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -117,7 +117,6 @@ where Info::HummockVersionDeltas(version_delta) => { version_delta.version_deltas[0].id > info.hummock_version.as_ref().unwrap().id } - Info::HummockSnapshot(_) => true, Info::MetaBackupManifestId(_) => true, Info::SystemParams(_) | Info::SessionParam(_) => true, Info::Snapshot(_) | Info::HummockWriteLimits(_) => unreachable!(), diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index b49abcb023429..b1ecf4182d1d0 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -24,8 +24,9 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; +use risingwave_hummock_sdk::FrontendHummockVersion; use risingwave_pb::common::WorkerNode; -use risingwave_pb::hummock::HummockVersionStats; +use risingwave_pb::hummock::{HummockVersionDeltas, HummockVersionStats}; use risingwave_pb::meta::relation::RelationInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::{FragmentWorkerSlotMapping, MetaSnapshot, SubscribeResponse}; @@ -85,11 +86,8 @@ impl ObserverState for FrontendObserverNode { resp ) } - Info::HummockSnapshot(_) => { - self.handle_hummock_snapshot_notification(resp); - } - Info::HummockVersionDeltas(_) => { - panic!("frontend node should not receive HummockVersionDeltas"); + Info::HummockVersionDeltas(deltas) => { + self.handle_hummock_snapshot_notification(deltas); } Info::MetaBackupManifestId(_) => { panic!("frontend node should not receive MetaBackupManifestId"); @@ -141,8 +139,7 @@ impl ObserverState for FrontendObserverNode { connections, users, nodes, - hummock_snapshot, - hummock_version: _, + hummock_version, meta_backup_manifest_id: _, hummock_write_limits: _, streaming_worker_slot_mappings, @@ -195,7 +192,9 @@ impl ObserverState for FrontendObserverNode { convert_worker_slot_mapping(&serving_worker_slot_mappings), ); self.hummock_snapshot_manager - .update(hummock_snapshot.unwrap()); + .init(FrontendHummockVersion::from_protobuf( + hummock_version.unwrap(), + )); let snapshot_version = version.unwrap(); catalog_guard.set_version(snapshot_version.catalog_version); @@ -465,19 +464,8 @@ impl FrontendObserverNode { } /// Update max committed epoch in `HummockSnapshotManager`. - fn handle_hummock_snapshot_notification(&self, resp: SubscribeResponse) { - let Some(info) = resp.info.as_ref() else { - return; - }; - match info { - Info::HummockSnapshot(hummock_snapshot) => match resp.operation() { - Operation::Update => { - self.hummock_snapshot_manager.update(*hummock_snapshot); - } - _ => panic!("receive an unsupported notify {:?}", resp), - }, - _ => unreachable!(), - } + fn handle_hummock_snapshot_notification(&self, deltas: HummockVersionDeltas) { + self.hummock_snapshot_manager.update(deltas); } fn handle_secret_notification(&mut self, resp: SubscribeResponse) { diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 4eb23e97dd5f7..c4df076826ab2 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -14,16 +14,22 @@ use std::assert_matches::assert_matches; use std::collections::btree_map::Entry; -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use std::time::Duration; -use risingwave_common::util::epoch::{Epoch, INVALID_EPOCH}; +use risingwave_common::must_match; +use risingwave_common::util::epoch::Epoch; +use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; +use risingwave_hummock_sdk::{ + FrontendHummockVersion, FrontendHummockVersionDelta, HummockVersionId, INVALID_VERSION_ID, +}; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; -use risingwave_pb::hummock::PbHummockSnapshot; +use risingwave_pb::hummock::{HummockVersionDeltas, PbHummockSnapshot}; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::watch; +use tracing::warn; use crate::expr::InlineNowProcTime; use crate::meta_client::FrontendMetaClient; @@ -85,7 +91,7 @@ impl ReadSnapshot { // DO NOT implement `Clone` for `PinnedSnapshot` because it's a "resource" that should always be a // singleton for each snapshot. Use `PinnedSnapshotRef` instead. pub struct PinnedSnapshot { - value: PbHummockSnapshot, + value: FrontendHummockVersion, unpin_sender: UnboundedSender, } @@ -103,26 +109,29 @@ impl PinnedSnapshot { let epoch = if is_barrier_read { batch_query_epoch::Epoch::Current(u64::MAX) } else { - batch_query_epoch::Epoch::Committed(self.value.committed_epoch) + batch_query_epoch::Epoch::Committed(self.value.max_committed_epoch) }; BatchQueryEpoch { epoch: Some(epoch) } } pub fn committed_epoch(&self) -> u64 { - self.value.committed_epoch + self.value.max_committed_epoch } } impl Drop for PinnedSnapshot { fn drop(&mut self) { - let _ = self.unpin_sender.send(Operation::Unpin(self.value)); + let _ = self.unpin_sender.send(Operation::Unpin(self.value.id)); } } /// Returns an invalid snapshot, used for initial values. -fn invalid_snapshot() -> PbHummockSnapshot { - PbHummockSnapshot { - committed_epoch: INVALID_EPOCH, +fn invalid_snapshot() -> FrontendHummockVersion { + FrontendHummockVersion { + id: INVALID_VERSION_ID, + max_committed_epoch: 0, + state_table_info: HummockVersionStateTableInfo::from_protobuf(&HashMap::new()), + table_change_log: Default::default(), } } @@ -170,43 +179,54 @@ impl HummockSnapshotManager { self.latest_snapshot.borrow().clone() } + pub fn init(&self, version: FrontendHummockVersion) { + self.worker_sender + .send(Operation::Pin(version.id, version.max_committed_epoch)) + .unwrap(); + // Then set the latest snapshot. + let snapshot = Arc::new(PinnedSnapshot { + value: version, + unpin_sender: self.worker_sender.clone(), + }); + if self.latest_snapshot.send(snapshot).is_err() { + warn!("fail to set init version"); + } + } + /// Update the latest snapshot. /// /// Should only be called by the observer manager. - pub fn update(&self, snapshot: PbHummockSnapshot) { + pub fn update(&self, deltas: HummockVersionDeltas) { self.latest_snapshot.send_if_modified(move |old_snapshot| { - // Note(bugen): theoretically, the snapshots from the observer should always be - // monotonically increasing, so there's no need to `max` them or check whether they are - // the same. But we still do it here to be safe. - // TODO: turn this into an assertion. - let snapshot = PbHummockSnapshot { - committed_epoch: std::cmp::max( - old_snapshot.value.committed_epoch, - snapshot.committed_epoch, - ), + if deltas.version_deltas.is_empty() { + return false; + } + let snapshot = { + let mut snapshot = old_snapshot.value.clone(); + for delta in deltas.version_deltas { + snapshot.apply_delta(FrontendHummockVersionDelta::from_protobuf(delta)); + } + snapshot }; - if old_snapshot.value == snapshot { - // Ignore the same snapshot - false - } else { - // First tell the worker that a new snapshot is going to be pinned. - self.worker_sender.send(Operation::Pin(snapshot)).unwrap(); - // Then set the latest snapshot. - *old_snapshot = Arc::new(PinnedSnapshot { - value: snapshot, - unpin_sender: self.worker_sender.clone(), - }); - - true - } + // First tell the worker that a new snapshot is going to be pinned. + self.worker_sender + .send(Operation::Pin(snapshot.id, snapshot.max_committed_epoch)) + .unwrap(); + // Then set the latest snapshot. + *old_snapshot = Arc::new(PinnedSnapshot { + value: snapshot, + unpin_sender: self.worker_sender.clone(), + }); + + true }); } /// Wait until the latest snapshot is newer than the given one. pub async fn wait(&self, snapshot: PbHummockSnapshot) { let mut rx = self.latest_snapshot.subscribe(); - while rx.borrow_and_update().value.committed_epoch < snapshot.committed_epoch { + while rx.borrow_and_update().value.max_committed_epoch < snapshot.committed_epoch { rx.changed().await.unwrap(); } } @@ -216,7 +236,7 @@ impl HummockSnapshotManager { #[derive(Debug)] enum PinState { /// The snapshot is currently pinned by some sessions in this frontend. - Pinned, + Pinned(u64), /// The snapshot is no longer pinned by any session in this frontend, but it's still considered /// to be pinned by the meta service. It will be unpinned by the [`UnpinWorker`] in the next @@ -228,20 +248,18 @@ enum PinState { #[derive(Debug)] enum Operation { /// Mark the snapshot as pinned, sent when a new snapshot is pinned with `update`. - Pin(PbHummockSnapshot), + Pin(HummockVersionId, u64), /// Mark the snapshot as unpinned, sent when all references to a [`PinnedSnapshot`] is dropped. - Unpin(PbHummockSnapshot), + Unpin(HummockVersionId), } impl Operation { /// Returns whether the operation is for an invalid snapshot, which should be ignored. fn is_invalid(&self) -> bool { - match self { - Operation::Pin(s) | Operation::Unpin(s) => s, - } - .committed_epoch - == INVALID_EPOCH + *match self { + Operation::Pin(id, _) | Operation::Unpin(id) => id, + } == INVALID_VERSION_ID } } @@ -249,13 +267,13 @@ impl Operation { /// /// The snapshot will be first sorted by `committed_epoch`, then by `current_epoch`. #[derive(Debug, PartialEq, Clone)] -struct SnapshotKey(PbHummockSnapshot); +struct SnapshotKey(HummockVersionId); impl Eq for SnapshotKey {} impl Ord for SnapshotKey { fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.0.committed_epoch.cmp(&other.0.committed_epoch) + self.0.to_u64().cmp(&other.0.to_u64()) } } @@ -319,15 +337,15 @@ impl UnpinWorker { } match operation { - Operation::Pin(snapshot) => { + Operation::Pin(version_id, committed_epoch) => { self.states - .try_insert(SnapshotKey(snapshot), PinState::Pinned) + .try_insert(SnapshotKey(version_id), PinState::Pinned(committed_epoch)) .unwrap(); } Operation::Unpin(snapshot) => match self.states.entry(SnapshotKey(snapshot)) { Entry::Vacant(_v) => unreachable!("unpin a snapshot that is not pinned"), Entry::Occupied(o) => { - assert_matches!(o.get(), PinState::Pinned); + assert_matches!(o.get(), PinState::Pinned(_)); *o.into_mut() = PinState::Unpinned; } }, @@ -338,18 +356,23 @@ impl UnpinWorker { /// and clean up their entries. async fn unpin_batch(&mut self) { // Find the minimum snapshot that is pinned. Unpin all snapshots before it. - if let Some(min_snapshot) = self + if let Some((min_snapshot, min_committed_epoch)) = self .states .iter() - .find(|(_, s)| matches!(s, PinState::Pinned)) - .map(|(k, _)| k.clone()) + .find(|(_, s)| matches!(s, PinState::Pinned(_))) + .map(|(k, s)| { + ( + k.clone(), + must_match!(s, PinState::Pinned(committed_epoch) => *committed_epoch), + ) + }) { if &min_snapshot == self.states.first_key_value().unwrap().0 { // Nothing to unpin. return; } - let min_epoch = min_snapshot.0.committed_epoch; + let min_epoch = min_committed_epoch; match self.meta_client.unpin_snapshot_before(min_epoch).await { Ok(()) => { diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index aada6c6876afe..6e993f2067f0a 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -15,6 +15,7 @@ use anyhow::{anyhow, Context}; use itertools::Itertools; use risingwave_common::secret::{LocalSecretManager, SecretEncryption}; +use risingwave_hummock_sdk::FrontendHummockVersion; use risingwave_meta::manager::{MetadataManager, SessionParamsManagerImpl}; use risingwave_meta::MetaResult; use risingwave_pb::backup_service::MetaBackupManifestId; @@ -305,7 +306,12 @@ impl NotificationServiceImpl { let (nodes, worker_node_version) = self.get_worker_node_snapshot().await?; - let hummock_snapshot = Some(self.hummock_manager.latest_snapshot()); + let hummock_version = self + .hummock_manager + .on_current_version(|version| { + FrontendHummockVersion::from_version(version).to_protobuf() + }) + .await; let session_params = match self.env.session_params_manager_impl_ref() { SessionParamsManagerImpl::Kv(manager) => manager.get_params().await, @@ -331,7 +337,7 @@ impl NotificationServiceImpl { secrets: decrypted_secrets, users, nodes, - hummock_snapshot, + hummock_version: Some(hummock_version), version: Some(SnapshotVersion { catalog_version, worker_node_version, diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 0772bac6699e1..125a45d583ce3 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -41,7 +41,6 @@ use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo}; use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; -use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::{PausedReason, PbRecoveryStatus}; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; use risingwave_pb::stream_service::BarrierCompleteResponse; @@ -1281,13 +1280,6 @@ impl GlobalBarrierManagerContext { ) -> MetaResult> { { { - // We must ensure all epochs are committed in ascending order, - // because the storage engine will query from new to old in the order in which - // the L0 layer files are generated. - // See https://github.com/risingwave-labs/risingwave/issues/1251 - // hummock_manager commit epoch. - let mut new_snapshot = None; - match &command_ctx.kind { BarrierKind::Initial => {} BarrierKind::Checkpoint(epochs) => { @@ -1298,7 +1290,7 @@ impl GlobalBarrierManagerContext { backfill_pinned_log_epoch, tables_to_commit, ); - new_snapshot = self.hummock_manager.commit_epoch(commit_info).await?; + self.hummock_manager.commit_epoch(commit_info).await?; } BarrierKind::Barrier => { // if we collect a barrier(checkpoint = false), @@ -1309,16 +1301,6 @@ impl GlobalBarrierManagerContext { } command_ctx.post_collect().await?; - // Notify new snapshot after fragment_mapping changes have been notified in - // `post_collect`. - if let Some(snapshot) = new_snapshot { - self.env - .notification_manager() - .notify_frontend_without_version( - Operation::Update, // Frontends don't care about operation. - Info::HummockSnapshot(snapshot), - ); - } Ok(if command_ctx.kind.is_checkpoint() { Some(self.hummock_manager.get_version_stats().await) } else { diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 8c021509dcbb2..f586999e55f94 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -71,10 +71,7 @@ pub struct CommitEpochInfo { impl HummockManager { /// Caller should ensure `epoch` > `max_committed_epoch` - pub async fn commit_epoch( - &self, - commit_info: CommitEpochInfo, - ) -> Result> { + pub async fn commit_epoch(&self, commit_info: CommitEpochInfo) -> Result<()> { let CommitEpochInfo { mut sstables, new_table_watermarks, @@ -89,7 +86,7 @@ impl HummockManager { let _timer = start_measure_real_process_timer!(self, "commit_epoch"); // Prevent commit new epochs if this flag is set if versioning_guard.disable_commit_epochs { - return Ok(None); + return Ok(()); } let versioning: &mut Versioning = &mut versioning_guard; @@ -291,14 +288,11 @@ impl HummockManager { )?; } - let snapshot = if is_visible_table_committed_epoch { + if is_visible_table_committed_epoch { let snapshot = HummockSnapshot { committed_epoch }; let prev_snapshot = self.latest_snapshot.swap(snapshot.into()); assert!(prev_snapshot.committed_epoch < committed_epoch); - Some(snapshot) - } else { - None - }; + } for compaction_group_id in &modified_compaction_groups { trigger_sst_stat( @@ -329,7 +323,7 @@ impl HummockManager { { self.check_state_consistency().await; } - Ok(snapshot) + Ok(()) } fn collect_table_write_throughput(&self, table_stats: PbTableStatsMap) { diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 9a795608f7e1a..14ea961d82bef 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -23,10 +23,12 @@ use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{ GroupDelta, HummockVersion, HummockVersionDelta, IntraLevelDelta, }; -use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, HummockVersionId}; +use risingwave_hummock_sdk::{ + CompactionGroupId, FrontendHummockVersionDelta, HummockEpoch, HummockVersionId, +}; use risingwave_pb::hummock::{ - CompactionConfig, CompatibilityVersion, GroupConstruct, HummockVersionStats, - StateTableInfoDelta, + CompactionConfig, CompatibilityVersion, GroupConstruct, HummockVersionDeltas, + HummockVersionStats, StateTableInfoDelta, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -230,6 +232,17 @@ impl<'a> InMemValTransaction for HummockVersionTransaction<'a> { version_deltas: pb_deltas, }), ); + self.notification_manager.notify_frontend_without_version( + Operation::Update, + Info::HummockVersionDeltas(HummockVersionDeltas { + version_deltas: deltas + .iter() + .map(|delta| { + FrontendHummockVersionDelta::from_delta(delta).to_protobuf() + }) + .collect(), + }), + ); } for delta in deltas { assert!(self.orig_deltas.insert(delta.id, delta.clone()).is_none()); diff --git a/src/storage/hummock_sdk/src/change_log.rs b/src/storage/hummock_sdk/src/change_log.rs index c231b0eb6b7b5..4402c63b2831c 100644 --- a/src/storage/hummock_sdk/src/change_log.rs +++ b/src/storage/hummock_sdk/src/change_log.rs @@ -87,8 +87,8 @@ where } } -impl TableChangeLog { - pub fn filter_epoch(&self, (min_epoch, max_epoch): (u64, u64)) -> &[EpochNewChangeLog] { +impl TableChangeLogCommon { + pub fn filter_epoch(&self, (min_epoch, max_epoch): (u64, u64)) -> &[EpochNewChangeLogCommon] { let start = self.0.partition_point(|epoch_change_log| { epoch_change_log.epochs.last().expect("non-empty") < &min_epoch }); 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 376626e844242..881527882672b 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 @@ -23,13 +23,13 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::VnodeBitmapExt; use risingwave_pb::hummock::{ CompactionConfig, CompatibilityVersion, GroupConstruct, GroupMerge, GroupMetaChange, - GroupTableChange, PbLevelType, + GroupTableChange, PbLevelType, StateTableInfo, StateTableInfoDelta, }; use tracing::warn; use super::group_split::get_sub_level_insert_hint; use super::{group_split, StateTableId}; -use crate::change_log::TableChangeLogCommon; +use crate::change_log::{ChangeLogDeltaCommon, TableChangeLogCommon}; use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; use crate::level::{Level, Levels, OverlappingLevel}; @@ -781,9 +781,25 @@ impl HummockVersion { } // apply to table change log - for (table_id, change_log_delta) in &version_delta.change_log_delta { + Self::apply_change_log_delta( + &mut self.table_change_log, + &version_delta.change_log_delta, + &version_delta.removed_table_ids, + &version_delta.state_table_info_delta, + &changed_table_info, + ); + } + + pub fn apply_change_log_delta( + table_change_log: &mut HashMap>, + change_log_delta: &HashMap>, + removed_table_ids: &HashSet, + state_table_info_delta: &HashMap, + changed_table_info: &HashMap>, + ) { + for (table_id, change_log_delta) in change_log_delta { let new_change_log = change_log_delta.new_log.as_ref().unwrap(); - match self.table_change_log.entry(*table_id) { + match table_change_log.entry(*table_id) { Entry::Occupied(entry) => { let change_log = entry.into_mut(); if let Some(prev_log) = change_log.0.last() { @@ -803,22 +819,21 @@ impl HummockVersion { // If a table has no new change log entry (even an empty one), it means we have stopped maintained // the change log for the table, and then we will remove the table change log. // The table change log will also be removed when the table id is removed. - self.table_change_log.retain(|table_id, _| { - if version_delta.removed_table_ids.contains(table_id) { + table_change_log.retain(|table_id, _| { + if removed_table_ids.contains(table_id) { return false; } - if let Some(table_info_delta) = version_delta.state_table_info_delta.get(table_id) + if let Some(table_info_delta) = state_table_info_delta.get(table_id) && let Some(Some(prev_table_info)) = changed_table_info.get(table_id) && table_info_delta.committed_epoch > prev_table_info.committed_epoch { // the table exists previously, and its committed epoch has progressed. } else { // otherwise, the table change log should be kept anyway return true; } - let contains = version_delta.change_log_delta.contains_key(table_id); + let contains = change_log_delta.contains_key(table_id); if !contains { warn!( ?table_id, - max_committed_epoch = version_delta.visible_table_committed_epoch(), "table change log dropped due to no further change log at newly committed epoch", ); } @@ -826,8 +841,8 @@ impl HummockVersion { }); // truncate the remaining table change log - for (table_id, change_log_delta) in &version_delta.change_log_delta { - if let Some(change_log) = self.table_change_log.get_mut(table_id) { + for (table_id, change_log_delta) in change_log_delta { + if let Some(change_log) = table_change_log.get_mut(table_id) { change_log.truncate(change_log_delta.truncate_epoch); } } diff --git a/src/storage/hummock_sdk/src/frontend_version.rs b/src/storage/hummock_sdk/src/frontend_version.rs new file mode 100644 index 0000000000000..549688054a8ad --- /dev/null +++ b/src/storage/hummock_sdk/src/frontend_version.rs @@ -0,0 +1,257 @@ +// 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, HashSet}; + +use risingwave_common::catalog::TableId; +use risingwave_pb::hummock::hummock_version_delta::PbChangeLogDelta; +use risingwave_pb::hummock::{ + PbEpochNewChangeLog, PbHummockVersion, PbHummockVersionDelta, PbTableChangeLog, + StateTableInfoDelta, +}; + +use crate::change_log::{ChangeLogDeltaCommon, EpochNewChangeLogCommon, TableChangeLogCommon}; +use crate::version::{HummockVersion, HummockVersionDelta, HummockVersionStateTableInfo}; +use crate::{HummockVersionId, INVALID_VERSION_ID}; + +#[derive(Clone, Debug)] +pub struct FrontendHummockVersion { + pub id: HummockVersionId, + pub max_committed_epoch: u64, + pub state_table_info: HummockVersionStateTableInfo, + pub table_change_log: HashMap>, +} + +impl FrontendHummockVersion { + pub fn from_version(version: &HummockVersion) -> Self { + Self { + id: version.id, + max_committed_epoch: version.max_committed_epoch, + state_table_info: version.state_table_info.clone(), + table_change_log: version + .table_change_log + .iter() + .map(|(table_id, change_log)| { + ( + *table_id, + TableChangeLogCommon( + change_log + .0 + .iter() + .map(|change_log| EpochNewChangeLogCommon { + new_value: vec![], + old_value: vec![], + epochs: change_log.epochs.clone(), + }) + .collect(), + ), + ) + }) + .collect(), + } + } + + pub fn to_protobuf(&self) -> PbHummockVersion { + PbHummockVersion { + id: self.id.0, + levels: Default::default(), + max_committed_epoch: self.max_committed_epoch, + safe_epoch: 0, + table_watermarks: Default::default(), + table_change_logs: self + .table_change_log + .iter() + .map(|(table_id, change_log)| { + ( + table_id.table_id, + PbTableChangeLog { + change_logs: change_log + .0 + .iter() + .map(|change_log| PbEpochNewChangeLog { + old_value: vec![], + new_value: vec![], + epochs: change_log.epochs.clone(), + }) + .collect(), + }, + ) + }) + .collect(), + state_table_info: self.state_table_info.to_protobuf(), + } + } + + pub fn from_protobuf(value: PbHummockVersion) -> Self { + Self { + id: HummockVersionId(value.id), + max_committed_epoch: value.max_committed_epoch, + state_table_info: HummockVersionStateTableInfo::from_protobuf(&value.state_table_info), + table_change_log: value + .table_change_logs + .into_iter() + .map(|(table_id, change_log)| { + ( + TableId::new(table_id), + TableChangeLogCommon( + change_log + .change_logs + .into_iter() + .map(|change_log| EpochNewChangeLogCommon { + new_value: vec![], + old_value: vec![], + epochs: change_log.epochs, + }) + .collect(), + ), + ) + }) + .collect(), + } + } + + pub fn apply_delta(&mut self, delta: FrontendHummockVersionDelta) { + if self.id != INVALID_VERSION_ID { + assert_eq!(self.id, delta.prev_id); + } + self.id = delta.id; + self.max_committed_epoch = delta.max_committed_epoch; + let (changed_table_info, _) = self + .state_table_info + .apply_delta(&delta.state_table_info_delta, &delta.removed_table_id); + HummockVersion::apply_change_log_delta( + &mut self.table_change_log, + &delta.change_log_delta, + &delta.removed_table_id, + &delta.state_table_info_delta, + &changed_table_info, + ); + } +} + +pub struct FrontendHummockVersionDelta { + pub prev_id: HummockVersionId, + pub id: HummockVersionId, + pub max_committed_epoch: u64, + pub removed_table_id: HashSet, + pub state_table_info_delta: HashMap, + pub change_log_delta: HashMap>, +} + +impl FrontendHummockVersionDelta { + pub fn from_delta(delta: &HummockVersionDelta) -> Self { + Self { + prev_id: delta.prev_id, + id: delta.id, + max_committed_epoch: delta.max_committed_epoch, + removed_table_id: delta.removed_table_ids.clone(), + state_table_info_delta: delta.state_table_info_delta.clone(), + change_log_delta: delta + .change_log_delta + .iter() + .map(|(table_id, change_log_delta)| { + ( + *table_id, + ChangeLogDeltaCommon { + truncate_epoch: change_log_delta.truncate_epoch, + new_log: change_log_delta.new_log.as_ref().map(|new_log| { + EpochNewChangeLogCommon { + new_value: vec![], + old_value: vec![], + epochs: new_log.epochs.clone(), + } + }), + }, + ) + }) + .collect(), + } + } + + pub fn to_protobuf(&self) -> PbHummockVersionDelta { + PbHummockVersionDelta { + id: self.id.to_u64(), + prev_id: self.prev_id.to_u64(), + group_deltas: Default::default(), + max_committed_epoch: self.max_committed_epoch, + safe_epoch: 0, + trivial_move: false, + new_table_watermarks: Default::default(), + removed_table_ids: self + .removed_table_id + .iter() + .map(|table_id| table_id.table_id) + .collect(), + change_log_delta: self + .change_log_delta + .iter() + .map(|(table_id, delta)| { + ( + table_id.table_id, + PbChangeLogDelta { + new_log: delta.new_log.as_ref().map(|new_log| PbEpochNewChangeLog { + old_value: vec![], + new_value: vec![], + epochs: new_log.epochs.clone(), + }), + truncate_epoch: delta.truncate_epoch, + }, + ) + }) + .collect(), + state_table_info_delta: self + .state_table_info_delta + .iter() + .map(|(table_id, delta)| (table_id.table_id, *delta)) + .collect(), + } + } + + pub fn from_protobuf(delta: PbHummockVersionDelta) -> Self { + Self { + prev_id: HummockVersionId::new(delta.prev_id), + id: HummockVersionId::new(delta.id), + max_committed_epoch: delta.max_committed_epoch, + removed_table_id: delta + .removed_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + state_table_info_delta: delta + .state_table_info_delta + .into_iter() + .map(|(table_id, delta)| (TableId::new(table_id), delta)) + .collect(), + change_log_delta: delta + .change_log_delta + .iter() + .map(|(table_id, change_log_delta)| { + ( + TableId::new(*table_id), + ChangeLogDeltaCommon { + truncate_epoch: change_log_delta.truncate_epoch, + new_log: change_log_delta.new_log.as_ref().map(|new_log| { + EpochNewChangeLogCommon { + new_value: vec![], + old_value: vec![], + epochs: new_log.epochs.clone(), + } + }), + }, + ) + }) + .collect(), + } + } +} diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 921ab18fcf7cd..b84d2751a50ea 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -51,6 +51,8 @@ pub mod table_stats; pub mod table_watermark; pub mod time_travel; pub mod version; +pub use frontend_version::{FrontendHummockVersion, FrontendHummockVersionDelta}; +mod frontend_version; pub use compact::*; use risingwave_common::catalog::TableId; From 14fdb689268a5e5f0cabb079b8dee3b17b74fdfc Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 18 Sep 2024 13:28:14 +0800 Subject: [PATCH 02/27] temp save --- src/batch/src/execution/grpc_exchange.rs | 2 +- src/frontend/src/handler/query.rs | 14 ++++++++++++-- src/frontend/src/scheduler/local.rs | 21 ++++++++++++--------- src/rpc_client/src/compute_client.rs | 3 ++- 4 files changed, 27 insertions(+), 13 deletions(-) diff --git a/src/batch/src/execution/grpc_exchange.rs b/src/batch/src/execution/grpc_exchange.rs index 27f64b1d5ed48..35eedca8b6da7 100644 --- a/src/batch/src/execution/grpc_exchange.rs +++ b/src/batch/src/execution/grpc_exchange.rs @@ -54,7 +54,7 @@ impl GrpcExchangeSource { tracing_context: plan.tracing_context, expr_context: Some(capture_expr_context()?), }; - client.execute(execute_request).await? + client.execute1(execute_request).await? } None => client.get_data(task_output_id.clone()).await?, }; diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 480bb1c7f6563..68e6c4645ebe4 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -515,9 +515,19 @@ pub async fn local_execute( // TODO: if there's no table scan, we don't need to acquire snapshot. let snapshot = session.pinned_snapshot(); + let epoch = snapshot.batch_query_epoch(); + let temp = 0; + // TODO: Passing sql here - let execution = - LocalQueryExecution::new(query, front_env.clone(), "", snapshot, session, timeout); + let execution = LocalQueryExecution::new( + query, + front_env.clone(), + "", + snapshot, + session, + timeout, + epoch, + ); Ok(execution.stream_rows()) } diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index fcd15368bb5fc..3980727afd06e 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -41,7 +41,7 @@ use risingwave_pb::batch_plan::{ ExchangeInfo, ExchangeSource, LocalExecutePlan, PbTaskId, PlanFragment, PlanNode as PbPlanNode, TaskOutputId, }; -use risingwave_pb::common::WorkerNode; +use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; use tracing::debug; @@ -61,9 +61,10 @@ pub struct LocalQueryExecution { sql: String, query: Query, front_env: FrontendEnv, + epoch: BatchQueryEpoch, // The snapshot will be released when LocalQueryExecution is dropped. // TODO - snapshot: ReadSnapshot, + _snapshot: ReadSnapshot, session: Arc, worker_node_manager: WorkerNodeSelector, timeout: Option, @@ -77,6 +78,7 @@ impl LocalQueryExecution { snapshot: ReadSnapshot, session: Arc, timeout: Option, + epoch: BatchQueryEpoch, ) -> Self { let sql = sql.into(); let worker_node_manager = WorkerNodeSelector::new( @@ -88,7 +90,8 @@ impl LocalQueryExecution { sql, query, front_env, - snapshot, + epoch, + _snapshot: snapshot, session, worker_node_manager, timeout, @@ -118,7 +121,7 @@ impl LocalQueryExecution { &plan_node, &task_id, context, - self.snapshot.batch_query_epoch(), + self.epoch, self.shutdown_rx().clone(), ); let executor = executor.build().await?; @@ -133,7 +136,7 @@ impl LocalQueryExecution { let span = tracing::info_span!( "local_execute", query_id = self.query.query_id.id, - epoch = ?self.snapshot.batch_query_epoch(), + epoch = ?self.epoch, ); Box::pin(self.run_inner().instrument(span)) } @@ -339,7 +342,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()), + epoch: Some(self.epoch), tracing_context: tracing_context.clone(), }; let exchange_source = ExchangeSource { @@ -383,7 +386,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()), + epoch: Some(self.epoch), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -422,7 +425,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()), + epoch: Some(self.epoch), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -458,7 +461,7 @@ impl LocalQueryExecution { let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()), + epoch: Some(self.epoch), tracing_context, }; diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index c065bb6935954..e00f5af4f3b5d 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -170,6 +170,7 @@ impl ComputeClient { epoch: BatchQueryEpoch, expr_context: ExprContext, ) -> Result> { + let temp = 0; Ok(self .task_client .to_owned() @@ -185,7 +186,7 @@ impl ComputeClient { .into_inner()) } - pub async fn execute(&self, req: ExecuteRequest) -> Result> { + pub async fn execute1(&self, req: ExecuteRequest) -> Result> { Ok(self .task_client .to_owned() From e4b390cdc74fc18b743f5f11a036abdfaea9e762 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 18 Sep 2024 13:43:10 +0800 Subject: [PATCH 03/27] fix test --- src/frontend/src/scheduler/snapshot.rs | 47 +++++++++++++---------- src/storage/hummock_sdk/src/change_log.rs | 5 ++- 2 files changed, 30 insertions(+), 22 deletions(-) diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index c4df076826ab2..5d1ad6d69d0b9 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -29,7 +29,6 @@ use risingwave_pb::hummock::{HummockVersionDeltas, PbHummockSnapshot}; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::watch; -use tracing::warn; use crate::expr::InlineNowProcTime; use crate::meta_client::FrontendMetaClient; @@ -180,35 +179,41 @@ impl HummockSnapshotManager { } pub fn init(&self, version: FrontendHummockVersion) { - self.worker_sender - .send(Operation::Pin(version.id, version.max_committed_epoch)) - .unwrap(); - // Then set the latest snapshot. - let snapshot = Arc::new(PinnedSnapshot { - value: version, - unpin_sender: self.worker_sender.clone(), - }); - if self.latest_snapshot.send(snapshot).is_err() { - warn!("fail to set init version"); - } + self.update_inner(|_| Some(version)); } /// Update the latest snapshot. /// /// Should only be called by the observer manager. pub fn update(&self, deltas: HummockVersionDeltas) { - self.latest_snapshot.send_if_modified(move |old_snapshot| { + self.update_inner(|old_snapshot| { if deltas.version_deltas.is_empty() { - return false; + return None; } - let snapshot = { - let mut snapshot = old_snapshot.value.clone(); - for delta in deltas.version_deltas { - snapshot.apply_delta(FrontendHummockVersionDelta::from_protobuf(delta)); - } - snapshot - }; + let mut snapshot = old_snapshot.clone(); + for delta in deltas.version_deltas { + snapshot.apply_delta(FrontendHummockVersionDelta::from_protobuf(delta)); + } + Some(snapshot) + }) + } + fn update_inner( + &self, + get_new_snapshot: impl FnOnce(&FrontendHummockVersion) -> Option, + ) { + self.latest_snapshot.send_if_modified(move |old_snapshot| { + let new_snapshot = get_new_snapshot(&old_snapshot.value); + let Some(snapshot) = new_snapshot else { + return false; + }; + if snapshot.id <= old_snapshot.value.id { + assert_eq!( + snapshot.id, old_snapshot.value.id, + "receive stale frontend version" + ); + return false; + } // First tell the worker that a new snapshot is going to be pinned. self.worker_sender .send(Operation::Pin(snapshot.id, snapshot.max_committed_epoch)) diff --git a/src/storage/hummock_sdk/src/change_log.rs b/src/storage/hummock_sdk/src/change_log.rs index 4402c63b2831c..cf3ded58b946e 100644 --- a/src/storage/hummock_sdk/src/change_log.rs +++ b/src/storage/hummock_sdk/src/change_log.rs @@ -88,7 +88,10 @@ where } impl TableChangeLogCommon { - pub fn filter_epoch(&self, (min_epoch, max_epoch): (u64, u64)) -> &[EpochNewChangeLogCommon] { + pub fn filter_epoch( + &self, + (min_epoch, max_epoch): (u64, u64), + ) -> &[EpochNewChangeLogCommon] { let start = self.0.partition_point(|epoch_change_log| { epoch_change_log.epochs.last().expect("non-empty") < &min_epoch }); From 73674f36de11ebce415cdb5c2378beb3ee38ade1 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 18 Sep 2024 19:09:40 +0800 Subject: [PATCH 04/27] feat(frontend): generate query epoch by committed epoch of involved tables --- src/frontend/planner_test/src/lib.rs | 10 +- src/frontend/src/binder/relation/mod.rs | 22 +++++ src/frontend/src/binder/set_expr.rs | 20 +++- src/frontend/src/binder/statement.rs | 16 +++- src/frontend/src/handler/create_table.rs | 9 +- src/frontend/src/handler/declare_cursor.rs | 5 +- src/frontend/src/handler/query.rs | 34 +++---- .../src/optimizer/logical_optimization.rs | 26 +++-- src/frontend/src/optimizer/mod.rs | 14 ++- .../src/scheduler/distributed/query.rs | 29 ++---- .../scheduler/distributed/query_manager.rs | 6 +- src/frontend/src/scheduler/local.rs | 25 +++-- src/frontend/src/scheduler/snapshot.rs | 96 +++++++++++++------ src/frontend/src/session/cursor_manager.rs | 25 +++-- src/frontend/src/session/transaction.rs | 16 ++-- src/rpc_client/src/compute_client.rs | 1 - src/tests/sqlsmith/tests/frontend/mod.rs | 3 +- 17 files changed, 232 insertions(+), 125 deletions(-) diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 675b99ad0e145..3239bdfa32fa7 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -603,6 +603,8 @@ impl TestCase { } }; + let scan_tables = bound.scan_tables(); + let mut planner = Planner::new(context.clone()); let plan_root = match planner.plan(bound) { @@ -626,7 +628,7 @@ impl TestCase { { let mut plan_root = plan_root.clone(); let optimized_logical_plan_for_batch = - match plan_root.gen_optimized_logical_plan_for_batch() { + match plan_root.gen_optimized_logical_plan_for_batch(&scan_tables) { Ok(optimized_logical_plan_for_batch) => optimized_logical_plan_for_batch, Err(err) => { ret.optimizer_error = Some(err.to_report_string_pretty()); @@ -675,7 +677,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan() { + let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { Ok(_batch_plan) => match plan_root.gen_batch_distributed_plan() { Ok(batch_plan) => batch_plan, Err(err) => { @@ -708,7 +710,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan() { + let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { Ok(_batch_plan) => match plan_root.gen_batch_local_plan() { Ok(batch_plan) => batch_plan, Err(err) => { @@ -736,7 +738,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan() { + let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { Ok(_batch_plan) => match plan_root.gen_batch_distributed_plan() { Ok(batch_plan) => batch_plan, Err(err) => { diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 2cdf3ea07db4e..1844d9161e247 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -137,6 +137,28 @@ impl Relation { _ => vec![], } } + + pub fn visit_all_scan_table(&self, mut visiter: impl FnMut(TableId)) { + match self { + Relation::Source(_) => {} + Relation::BaseTable(table) => { + visiter(table.table_id); + } + Relation::SystemTable(_) => {} + Relation::Subquery(subquery) => { + subquery.query.body.visit_all_scan_table_id(&mut visiter); + } + Relation::Join(join) | Relation::Apply(join) => { + join.left.visit_all_scan_table(&mut visiter); + join.right.visit_all_scan_table(&mut visiter); + } + Relation::WindowTableFunction(_) => {} + Relation::TableFunction { .. } => {} + Relation::Watermark(_) => {} + Relation::Share(_) => {} + Relation::BackCteRef(_) => {} + } + } } #[derive(Debug)] diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index 68af5845bf7a4..fed540d8e928c 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -16,7 +16,7 @@ use std::borrow::Cow; use std::collections::HashMap; use risingwave_common::bail_not_implemented; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableId}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Corresponding, SetExpr, SetOperator}; @@ -143,6 +143,24 @@ impl BoundSetExpr { } } } + + pub fn visit_all_scan_table_id(&self, mut visiter: impl FnMut(TableId)) { + match self { + BoundSetExpr::Select(select) => { + if let Some(relation) = &select.from { + relation.visit_all_scan_table(&mut visiter); + } + } + BoundSetExpr::Query(query) => { + query.body.visit_all_scan_table_id(&mut visiter); + } + BoundSetExpr::Values(_) => {} + BoundSetExpr::SetOperation { left, right, .. } => { + left.visit_all_scan_table_id(&mut visiter); + right.visit_all_scan_table_id(&mut visiter); + } + } + } } impl Binder { diff --git a/src/frontend/src/binder/statement.rs b/src/frontend/src/binder/statement.rs index b73fab90aed9a..addff64c7e145 100644 --- a/src/frontend/src/binder/statement.rs +++ b/src/frontend/src/binder/statement.rs @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; + use risingwave_common::bail_not_implemented; -use risingwave_common::catalog::Field; +use risingwave_common::catalog::{Field, TableId}; use risingwave_sqlparser::ast::Statement; use super::delete::BoundDelete; @@ -57,6 +59,18 @@ impl BoundStatement { BoundStatement::CreateView(_) => vec![], } } + + pub fn scan_tables(&self) -> HashSet { + if let BoundStatement::Query(query) = self { + let mut tables = HashSet::new(); + query.body.visit_all_scan_table_id(|table_id| { + tables.insert(table_id); + }); + tables + } else { + HashSet::new() + } + } } impl Binder { diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index e7b2b44226657..0a27f2b7a4e25 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -27,6 +27,7 @@ use risingwave_common::catalog::{ INITIAL_TABLE_VERSION_ID, }; use risingwave_common::license::Feature; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; @@ -56,7 +57,7 @@ use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, ColumnId, DatabaseId, SchemaId}; use crate::error::{ErrorCode, Result, RwError}; -use crate::expr::{Expr, ExprImpl, ExprRewriter}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; use crate::handler::create_source::{ bind_columns_from_source, bind_connector_props, bind_create_source_or_table_with_connector, bind_source_watermark, handle_addition_columns, UPSTREAM_SOURCE_KEY, @@ -329,10 +330,8 @@ pub fn bind_sql_column_constraints( // so the rewritten expression should almost always be pure and we directly call `fold_const` // here. Actually we do not require purity of the expression here since we're only to get a // snapshot value. - let rewritten_expr_impl = session - .pinned_snapshot() - .inline_now_proc_time() - .rewrite_expr(expr_impl.clone()); + let rewritten_expr_impl = + InlineNowProcTime::new(Epoch::now()).rewrite_expr(expr_impl.clone()); if let Some(snapshot_value) = rewritten_expr_impl.try_fold_const() { let snapshot_value = snapshot_value?; diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index 4c13886581706..d0bb1e0f0496e 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -149,6 +149,7 @@ pub async fn create_chunk_stream_for_cursor( plan_fragmenter, query_mode, schema, + scan_tables, .. } = plan_fragmenter_result; @@ -161,10 +162,10 @@ pub async fn create_chunk_stream_for_cursor( match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => CursorDataChunkStream::LocalDataChunk(Some( - local_execute(session.clone(), query, can_timeout_cancel).await?, + local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, )), QueryMode::Distributed => CursorDataChunkStream::DistributedDataChunk(Some( - distribute_execute(session.clone(), query, can_timeout_cancel).await?, + distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, )), }, schema.fields.clone(), diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 68e6c4645ebe4..37f40ed53ded8 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -206,6 +206,7 @@ pub struct BatchQueryPlanResult { // subset of the final one. i.e. the final one may contain more implicit dependencies on // indices. pub(crate) dependent_relations: Vec, + pub(crate) scan_tables: HashSet, } fn gen_batch_query_plan( @@ -223,9 +224,11 @@ fn gen_batch_query_plan( let mut planner = Planner::new(context); + let scan_tables = bound.scan_tables(); + let mut logical = planner.plan(bound)?; let schema = logical.schema(); - let batch_plan = logical.gen_batch_plan()?; + let batch_plan = logical.gen_batch_plan(&scan_tables)?; let dependent_relations = RelationCollectorVisitor::collect_with(dependent_relations, batch_plan.clone()); @@ -260,6 +263,7 @@ fn gen_batch_query_plan( schema, stmt_type, dependent_relations: dependent_relations.into_iter().collect_vec(), + scan_tables, }) } @@ -312,6 +316,7 @@ pub struct BatchPlanFragmenterResult { pub(crate) schema: Schema, pub(crate) stmt_type: StatementType, pub(crate) _dependent_relations: Vec, + pub(crate) scan_tables: HashSet, } pub fn gen_batch_plan_fragmenter( @@ -324,6 +329,7 @@ pub fn gen_batch_plan_fragmenter( schema, stmt_type, dependent_relations, + scan_tables, } = plan_result; tracing::trace!( @@ -348,6 +354,7 @@ pub fn gen_batch_plan_fragmenter( schema, stmt_type, _dependent_relations: dependent_relations, + scan_tables, }) } @@ -361,6 +368,7 @@ pub async fn create_stream( query_mode, schema, stmt_type, + scan_tables, .. } = plan_fragmenter_result; @@ -392,7 +400,7 @@ pub async fn create_stream( let row_stream = match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new( - local_execute(session.clone(), query, can_timeout_cancel).await?, + local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, column_types, formats, session.clone(), @@ -400,7 +408,7 @@ pub async fn create_stream( // Local mode do not support cancel tasks. QueryMode::Distributed => { PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new( - distribute_execute(session.clone(), query, can_timeout_cancel).await?, + distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, column_types, formats, session.clone(), @@ -480,6 +488,7 @@ pub async fn distribute_execute( session: Arc, query: Query, can_timeout_cancel: bool, + scan_tables: HashSet, ) -> Result { let timeout = if cfg!(madsim) { None @@ -493,7 +502,7 @@ pub async fn distribute_execute( let query_manager = session.env().query_manager().clone(); query_manager - .schedule(execution_context, query) + .schedule(execution_context, query, scan_tables) .await .map_err(|err| err.into()) } @@ -502,6 +511,7 @@ pub async fn local_execute( session: Arc, query: Query, can_timeout_cancel: bool, + scan_tables: HashSet, ) -> Result { let timeout = if cfg!(madsim) { None @@ -513,21 +523,11 @@ pub async fn local_execute( let front_env = session.env(); // TODO: if there's no table scan, we don't need to acquire snapshot. - let snapshot = session.pinned_snapshot(); - - let epoch = snapshot.batch_query_epoch(); - let temp = 0; + let snapshot = session.pinned_snapshot(scan_tables); // TODO: Passing sql here - let execution = LocalQueryExecution::new( - query, - front_env.clone(), - "", - snapshot, - session, - timeout, - epoch, - ); + let execution = + LocalQueryExecution::new(query, front_env.clone(), "", snapshot, session, timeout); Ok(execution.stream_rows()) } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 93da63cf70a42..35b92b98c7d28 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; + use itertools::Itertools; use risingwave_common::bail; @@ -108,6 +110,8 @@ impl OptimizationStage { use std::sync::LazyLock; +use crate::catalog::TableId; + pub struct LogicalOptimizer {} static DAG_TO_TREE: LazyLock = LazyLock::new(|| { @@ -532,15 +536,22 @@ impl LogicalOptimizer { plan } - pub fn inline_now_proc_time(plan: PlanRef, ctx: &OptimizerContextRef) -> PlanRef { + pub fn inline_now_proc_time( + plan: PlanRef, + ctx: &OptimizerContextRef, + scan_tables: &HashSet, + ) -> Result { // If now() and proctime() are not found, bail out. let mut v = NowProcTimeFinder::default(); plan.visit_exprs_recursive(&mut v); if !v.has() { - return plan; + return Ok(plan); } - let mut v = ctx.session_ctx().pinned_snapshot().inline_now_proc_time(); + let mut v = ctx + .session_ctx() + .pinned_snapshot(scan_tables.clone()) + .inline_now_proc_time()?; let plan = plan.rewrite_exprs_recursive(&mut v); @@ -548,7 +559,7 @@ impl LogicalOptimizer { ctx.trace("Inline Now and ProcTime:"); ctx.trace(plan.explain_to_string()); } - plan + Ok(plan) } pub fn gen_optimized_logical_plan_for_stream(mut plan: PlanRef) -> Result { @@ -679,7 +690,10 @@ impl LogicalOptimizer { Ok(plan) } - pub fn gen_optimized_logical_plan_for_batch(mut plan: PlanRef) -> Result { + pub fn gen_optimized_logical_plan_for_batch( + mut plan: PlanRef, + scan_tables: &HashSet, + ) -> Result { let ctx = plan.ctx(); let explain_trace = ctx.is_explain_trace(); @@ -689,7 +703,7 @@ impl LogicalOptimizer { } // Inline `NOW()` and `PROCTIME()`, only for batch queries. - plan = Self::inline_now_proc_time(plan, &ctx); + plan = Self::inline_now_proc_time(plan, &ctx, scan_tables)?; // Convert the dag back to the tree, because we don't support DAG plan for batch. plan = plan.optimize_by_rules(&DAG_TO_TREE); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index de5c3deaf0d6b..fc6786ff6057c 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -40,7 +40,7 @@ mod plan_expr_visitor; mod rule; use std::assert_matches::assert_matches; -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use fixedbitset::FixedBitSet; use itertools::Itertools as _; @@ -279,22 +279,26 @@ impl PlanRoot { } /// Apply logical optimization to the plan for batch. - pub fn gen_optimized_logical_plan_for_batch(&mut self) -> Result { + pub fn gen_optimized_logical_plan_for_batch( + &mut self, + scan_tables: &HashSet, + ) -> Result { assert_eq!(self.phase, PlanPhase::Logical); assert_eq!(self.plan.convention(), Convention::Logical); - self.plan = LogicalOptimizer::gen_optimized_logical_plan_for_batch(self.plan.clone())?; + self.plan = + LogicalOptimizer::gen_optimized_logical_plan_for_batch(self.plan.clone(), scan_tables)?; self.phase = PlanPhase::OptimizedLogicalForBatch; assert_eq!(self.plan.convention(), Convention::Logical); Ok(self.plan.clone()) } /// Optimize and generate a singleton batch physical plan without exchange nodes. - pub fn gen_batch_plan(&mut self) -> Result { + pub fn gen_batch_plan(&mut self, scan_tables: &HashSet) -> Result { assert_eq!(self.plan.convention(), Convention::Logical); let mut plan = match self.phase { PlanPhase::Logical => { // Logical optimization - self.gen_optimized_logical_plan_for_batch()? + self.gen_optimized_logical_plan_for_batch(scan_tables)? } PlanPhase::OptimizedLogicalForBatch => self.plan.clone(), PlanPhase::Batch | PlanPhase::OptimizedLogicalForStream | PlanPhase::Stream => { diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index b991d86eca2b7..61996a4d50e29 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -25,7 +25,7 @@ use pgwire::pg_server::SessionId; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::array::DataChunk; use risingwave_pb::batch_plan::{TaskId as PbTaskId, TaskOutputId as PbTaskOutputId}; -use risingwave_pb::common::HostAddress; +use risingwave_pb::common::{BatchQueryEpoch, HostAddress}; use risingwave_rpc_client::ComputeClientPoolRef; use thiserror_ext::AsReport; use tokio::sync::mpsc::{channel, Receiver, Sender}; @@ -40,7 +40,7 @@ use crate::scheduler::distributed::stage::StageEvent::ScheduledRoot; use crate::scheduler::distributed::StageEvent::Scheduled; use crate::scheduler::distributed::StageExecution; use crate::scheduler::plan_fragmenter::{Query, StageId, ROOT_TASK_ID, ROOT_TASK_OUTPUT_ID}; -use crate::scheduler::{ExecutionContextRef, ReadSnapshot, SchedulerError, SchedulerResult}; +use crate::scheduler::{ExecutionContextRef, QuerySnapshot, SchedulerError, SchedulerResult}; /// Message sent to a `QueryRunner` to control its execution. #[derive(Debug)] @@ -124,7 +124,7 @@ impl QueryExecution { self: Arc, context: ExecutionContextRef, worker_node_manager: WorkerNodeSelector, - pinned_snapshot: ReadSnapshot, + pinned_snapshot: QuerySnapshot, compute_client_pool: ComputeClientPoolRef, catalog_reader: CatalogReader, query_execution_info: QueryExecutionInfoRef, @@ -137,7 +137,7 @@ impl QueryExecution { // reference of `pinned_snapshot`. Its ownership will be moved into `QueryRunner` so that it // can control when to release the snapshot. let stage_executions = self.gen_stage_executions( - &pinned_snapshot, + pinned_snapshot.batch_query_epoch()?, context.clone(), worker_node_manager, compute_client_pool.clone(), @@ -225,7 +225,7 @@ impl QueryExecution { fn gen_stage_executions( &self, - pinned_snapshot: &ReadSnapshot, + epoch: BatchQueryEpoch, context: ExecutionContextRef, worker_node_manager: WorkerNodeSelector, compute_client_pool: ComputeClientPoolRef, @@ -244,7 +244,7 @@ impl QueryExecution { .collect::>>(); let stage_exec = Arc::new(StageExecution::new( - pinned_snapshot.batch_query_epoch(), + epoch, self.query.stage_graph.stages[&stage_id].clone(), worker_node_manager.clone(), self.shutdown_tx.clone(), @@ -296,7 +296,7 @@ impl Debug for QueryRunner { } impl QueryRunner { - async fn run(mut self, pinned_snapshot: ReadSnapshot) { + async fn run(mut self, pinned_snapshot: QuerySnapshot) { self.query_metrics.running_query_num.inc(); // Start leaf stages. let leaf_stages = self.query.leaf_stages(); @@ -465,7 +465,7 @@ impl QueryRunner { #[cfg(test)] pub(crate) mod tests { - use std::collections::HashMap; + use std::collections::{HashMap, HashSet}; use std::sync::{Arc, RwLock}; use fixedbitset::FixedBitSet; @@ -495,11 +495,9 @@ pub(crate) mod tests { use crate::scheduler::distributed::QueryExecution; use crate::scheduler::plan_fragmenter::{BatchPlanFragmenter, Query}; use crate::scheduler::{ - DistributedQueryMetrics, ExecutionContext, HummockSnapshotManager, QueryExecutionInfo, - ReadSnapshot, + DistributedQueryMetrics, ExecutionContext, QueryExecutionInfo, QuerySnapshot, ReadSnapshot, }; use crate::session::SessionImpl; - use crate::test_utils::MockFrontendMetaClient; use crate::utils::Condition; use crate::TableCatalog; @@ -508,14 +506,10 @@ pub(crate) mod tests { let worker_node_manager = Arc::new(WorkerNodeManager::mock(vec![])); let worker_node_selector = WorkerNodeSelector::new(worker_node_manager.clone(), false); let compute_client_pool = Arc::new(ComputeClientPool::for_test()); - let hummock_snapshot_manager = Arc::new(HummockSnapshotManager::new(Arc::new( - MockFrontendMetaClient {}, - ))); let catalog_reader = CatalogReader::new(Arc::new(parking_lot::RwLock::new(Catalog::default()))); let query = create_query().await; let query_id = query.query_id().clone(); - let pinned_snapshot = hummock_snapshot_manager.acquire(); let query_execution = Arc::new(QueryExecution::new(query, (0, 0), None)); let query_execution_info = Arc::new(RwLock::new(QueryExecutionInfo::new_from_map( HashMap::from([(query_id, query_execution.clone())]), @@ -525,10 +519,7 @@ pub(crate) mod tests { .start( ExecutionContext::new(SessionImpl::mock().into(), None).into(), worker_node_selector, - ReadSnapshot::FrontendPinned { - snapshot: pinned_snapshot, - is_barrier_read: true - }, + QuerySnapshot::new(ReadSnapshot::BarrierRead, HashSet::from_iter([0.into()])), compute_client_pool, catalog_reader, query_execution_info, diff --git a/src/frontend/src/scheduler/distributed/query_manager.rs b/src/frontend/src/scheduler/distributed/query_manager.rs index 2d977cfb675e6..6464759ebbd18 100644 --- a/src/frontend/src/scheduler/distributed/query_manager.rs +++ b/src/frontend/src/scheduler/distributed/query_manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use std::fmt::{Debug, Formatter}; use std::pin::Pin; use std::sync::{Arc, RwLock}; @@ -33,6 +33,7 @@ use tokio::sync::OwnedSemaphorePermit; use super::stats::DistributedQueryMetrics; use super::QueryExecution; use crate::catalog::catalog_service::CatalogReader; +use crate::catalog::TableId; use crate::scheduler::plan_fragmenter::{Query, QueryId}; use crate::scheduler::{ExecutionContextRef, SchedulerResult}; @@ -190,6 +191,7 @@ impl QueryManager { &self, context: ExecutionContextRef, query: Query, + scan_tables: HashSet, ) -> SchedulerResult { if let Some(query_limit) = self.disrtibuted_query_limit && self.query_metrics.running_query_num.get() as u64 == query_limit @@ -212,7 +214,7 @@ impl QueryManager { .add_query(query_id.clone(), query_execution.clone()); // TODO: if there's no table scan, we don't need to acquire snapshot. - let pinned_snapshot = context.session().pinned_snapshot(); + let pinned_snapshot = context.session().pinned_snapshot(scan_tables); let worker_node_manager_reader = WorkerNodeSelector::new( self.worker_node_manager.clone(), diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 3980727afd06e..0dd7b77fc0823 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -41,7 +41,7 @@ use risingwave_pb::batch_plan::{ ExchangeInfo, ExchangeSource, LocalExecutePlan, PbTaskId, PlanFragment, PlanNode as PbPlanNode, TaskOutputId, }; -use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; +use risingwave_pb::common::WorkerNode; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; use tracing::debug; @@ -52,7 +52,7 @@ use crate::error::RwError; use crate::optimizer::plan_node::PlanNodeType; use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; use crate::scheduler::task_context::FrontendBatchTaskContext; -use crate::scheduler::{ReadSnapshot, SchedulerError, SchedulerResult}; +use crate::scheduler::{QuerySnapshot, SchedulerError, SchedulerResult}; use crate::session::{FrontendEnv, SessionImpl}; // TODO(error-handling): use a concrete error type. @@ -61,10 +61,9 @@ pub struct LocalQueryExecution { sql: String, query: Query, front_env: FrontendEnv, - epoch: BatchQueryEpoch, // The snapshot will be released when LocalQueryExecution is dropped. // TODO - _snapshot: ReadSnapshot, + snapshot: QuerySnapshot, session: Arc, worker_node_manager: WorkerNodeSelector, timeout: Option, @@ -75,10 +74,9 @@ impl LocalQueryExecution { query: Query, front_env: FrontendEnv, sql: S, - snapshot: ReadSnapshot, + snapshot: QuerySnapshot, session: Arc, timeout: Option, - epoch: BatchQueryEpoch, ) -> Self { let sql = sql.into(); let worker_node_manager = WorkerNodeSelector::new( @@ -90,8 +88,7 @@ impl LocalQueryExecution { sql, query, front_env, - epoch, - _snapshot: snapshot, + snapshot, session, worker_node_manager, timeout, @@ -121,7 +118,7 @@ impl LocalQueryExecution { &plan_node, &task_id, context, - self.epoch, + self.snapshot.batch_query_epoch()?, self.shutdown_rx().clone(), ); let executor = executor.build().await?; @@ -136,7 +133,7 @@ impl LocalQueryExecution { let span = tracing::info_span!( "local_execute", query_id = self.query.query_id.id, - epoch = ?self.epoch, + epoch = ?self.snapshot.batch_query_epoch(), ); Box::pin(self.run_inner().instrument(span)) } @@ -342,7 +339,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.epoch), + epoch: Some(self.snapshot.batch_query_epoch()?), tracing_context: tracing_context.clone(), }; let exchange_source = ExchangeSource { @@ -386,7 +383,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.epoch), + epoch: Some(self.snapshot.batch_query_epoch()?), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -425,7 +422,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.epoch), + epoch: Some(self.snapshot.batch_query_epoch()?), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -461,7 +458,7 @@ impl LocalQueryExecution { let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.epoch), + epoch: Some(self.snapshot.batch_query_epoch()?), tracing_context, }; diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 5d1ad6d69d0b9..c01499671eca9 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -14,10 +14,12 @@ use std::assert_matches::assert_matches; use std::collections::btree_map::Entry; -use std::collections::{BTreeMap, HashMap}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; use std::time::Duration; +use anyhow::anyhow; +use risingwave_common::catalog::TableId; use risingwave_common::must_match; use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; @@ -32,6 +34,7 @@ use tokio::sync::watch; use crate::expr::InlineNowProcTime; use crate::meta_client::FrontendMetaClient; +use crate::scheduler::SchedulerError; /// The interval between two unpin batches. const UNPIN_INTERVAL_SECS: u64 = 10; @@ -42,47 +45,60 @@ pub enum ReadSnapshot { /// A frontend-pinned snapshot. FrontendPinned { snapshot: PinnedSnapshotRef, - // It's embedded here because we always use it together with snapshot. - is_barrier_read: bool, }, + BarrierRead, + /// Other arbitrary epoch, e.g. user specified. /// Availability and consistency of underlying data should be guaranteed accordingly. /// Currently it's only used for querying meta snapshot backup. Other(Epoch), } -impl ReadSnapshot { +pub struct QuerySnapshot { + snapshot: ReadSnapshot, + scan_tables: HashSet, +} + +impl QuerySnapshot { + pub fn new(snapshot: ReadSnapshot, scan_tables: HashSet) -> Self { + Self { + snapshot, + scan_tables, + } + } + /// Get the [`BatchQueryEpoch`] for this snapshot. - pub fn batch_query_epoch(&self) -> BatchQueryEpoch { - match self { - ReadSnapshot::FrontendPinned { - snapshot, - is_barrier_read, - } => snapshot.batch_query_epoch(*is_barrier_read), + pub fn batch_query_epoch(&self) -> Result { + Ok(match &self.snapshot { + ReadSnapshot::FrontendPinned { snapshot } => BatchQueryEpoch { + epoch: Some(batch_query_epoch::Epoch::Committed( + snapshot.batch_query_epoch(&self.scan_tables)?.0, + )), + }, + ReadSnapshot::BarrierRead => BatchQueryEpoch { + epoch: Some(batch_query_epoch::Epoch::Current(u64::MAX)), + }, ReadSnapshot::Other(e) => BatchQueryEpoch { epoch: Some(batch_query_epoch::Epoch::Backup(e.0)), }, - } + }) } - pub fn inline_now_proc_time(&self) -> InlineNowProcTime { - let epoch = match self { - ReadSnapshot::FrontendPinned { snapshot, .. } => Epoch(snapshot.committed_epoch()), + pub fn inline_now_proc_time(&self) -> Result { + let epoch = match &self.snapshot { + ReadSnapshot::FrontendPinned { snapshot, .. } => { + snapshot.batch_query_epoch(&self.scan_tables)? + } ReadSnapshot::Other(epoch) => *epoch, + ReadSnapshot::BarrierRead => Epoch::now(), }; - InlineNowProcTime::new(epoch) + Ok(InlineNowProcTime::new(epoch)) } /// Returns true if this snapshot is a barrier read. pub fn support_barrier_read(&self) -> bool { - match self { - ReadSnapshot::FrontendPinned { - snapshot: _, - is_barrier_read, - } => *is_barrier_read, - ReadSnapshot::Other(_) => false, - } + matches!(&self.snapshot, ReadSnapshot::BarrierRead) } } @@ -104,17 +120,35 @@ impl std::fmt::Debug for PinnedSnapshot { pub type PinnedSnapshotRef = Arc; impl PinnedSnapshot { - fn batch_query_epoch(&self, is_barrier_read: bool) -> BatchQueryEpoch { - let epoch = if is_barrier_read { - batch_query_epoch::Epoch::Current(u64::MAX) - } else { - batch_query_epoch::Epoch::Committed(self.value.max_committed_epoch) - }; - BatchQueryEpoch { epoch: Some(epoch) } + fn batch_query_epoch(&self, scan_tables: &HashSet) -> Result { + // use the min committed epoch of tables involved in the scan + let epoch = scan_tables + .iter() + .map(|table_id| { + self.value + .state_table_info + .info() + .get(table_id) + .map(|info| Epoch(info.committed_epoch)) + .ok_or_else(|| anyhow!("table id {table_id} may have been dropped")) + }) + .try_fold(None, |prev_min_committed_epoch, committed_epoch| { + committed_epoch.map(|committed_epoch| { + if let Some(prev_min_committed_epoch) = prev_min_committed_epoch + && prev_min_committed_epoch >= committed_epoch + { + Some(prev_min_committed_epoch) + } else { + Some(committed_epoch) + } + }) + })? + .unwrap_or_else(Epoch::now); // When no table is involved, use current timestamp as epoch + Ok(epoch) } - pub fn committed_epoch(&self) -> u64 { - self.value.max_committed_epoch + pub fn version(&self) -> &FrontendHummockVersion { + &self.value } } diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index 2a17c3ee0dc2e..3ef7e10406900 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -14,11 +14,12 @@ use core::mem; use core::time::Duration; -use std::collections::{HashMap, VecDeque}; +use std::collections::{HashMap, HashSet, VecDeque}; use std::rc::Rc; use std::sync::Arc; use std::time::Instant; +use anyhow::anyhow; use bytes::Bytes; use fixedbitset::FixedBitSet; use futures::StreamExt; @@ -47,7 +48,7 @@ use crate::handler::HandlerArgs; use crate::optimizer::plan_node::{generic, BatchLogSeqScan}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::PlanRoot; -use crate::scheduler::{DistributedQueryStream, LocalQueryStream, ReadSnapshot}; +use crate::scheduler::{DistributedQueryStream, LocalQueryStream}; use crate::{OptimizerContext, OptimizerContextRef, PgResponseStream, PlanRef, TableCatalog}; pub enum CursorDataChunkStream { @@ -247,14 +248,17 @@ impl SubscriptionCursor { // TODO: is this the right behavior? Should we delay the query stream initiation till the first fetch? let (chunk_stream, fields) = Self::initiate_query(None, &dependent_table_id, handle_args.clone()).await?; - let pinned_epoch = match handle_args.session.get_pinned_snapshot().ok_or_else(|| { - ErrorCode::InternalError("Fetch Cursor can't find snapshot epoch".to_string()) - })? { - ReadSnapshot::FrontendPinned { snapshot, .. } => snapshot.committed_epoch(), - ReadSnapshot::Other(_) => { - return Err(ErrorCode::InternalError("Fetch Cursor can't start from specified query epoch. May run `set query_epoch = 0;`".to_string()).into()); - } - }; + let pinned_epoch = handle_args + .session + .env + .hummock_snapshot_manager + .acquire() + .version() + .state_table_info + .info() + .get(&dependent_table_id) + .ok_or_else(|| anyhow!("dependent_table_id {dependent_table_id} not exists"))? + .committed_epoch; let start_timestamp = pinned_epoch; ( @@ -620,6 +624,7 @@ impl SubscriptionCursor { schema, stmt_type: StatementType::SELECT, dependent_relations: table_catalog.dependent_relations.clone(), + scan_tables: HashSet::from_iter([table_catalog.id]), }) } diff --git a/src/frontend/src/session/transaction.rs b/src/frontend/src/session/transaction.rs index d14df7f4900a2..871421ad31c7d 100644 --- a/src/frontend/src/session/transaction.rs +++ b/src/frontend/src/session/transaction.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::{Arc, Weak}; @@ -21,8 +22,9 @@ use risingwave_hummock_sdk::EpochWithGap; use super::SessionImpl; use crate::catalog::catalog_service::CatalogWriter; +use crate::catalog::TableId; use crate::error::{ErrorCode, Result}; -use crate::scheduler::ReadSnapshot; +use crate::scheduler::{QuerySnapshot, ReadSnapshot}; use crate::user::user_service::UserInfoWriter; /// Globally unique transaction id in this frontend instance. @@ -213,8 +215,9 @@ impl SessionImpl { /// Acquires and pins a snapshot for the current transaction. /// /// If a snapshot is already acquired, returns it directly. - pub fn pinned_snapshot(&self) -> ReadSnapshot { - self.txn_ctx() + pub fn pinned_snapshot(&self, scan_tables: HashSet) -> QuerySnapshot { + let snapshot = self + .txn_ctx() .snapshot .get_or_insert_with(|| { // query_epoch must be pure epoch @@ -225,19 +228,20 @@ impl SessionImpl { if let Some(query_epoch) = query_epoch { ReadSnapshot::Other(query_epoch) + } else if self.is_barrier_read() { + ReadSnapshot::BarrierRead } else { // Acquire hummock snapshot for execution. - let is_barrier_read = self.is_barrier_read(); let hummock_snapshot_manager = self.env().hummock_snapshot_manager(); let pinned_snapshot = hummock_snapshot_manager.acquire(); ReadSnapshot::FrontendPinned { snapshot: pinned_snapshot, - is_barrier_read, } } }) - .clone() + .clone(); + QuerySnapshot::new(snapshot, scan_tables) } } diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index e00f5af4f3b5d..b4d0ba4e77546 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -170,7 +170,6 @@ impl ComputeClient { epoch: BatchQueryEpoch, expr_context: ExprContext, ) -> Result> { - let temp = 0; Ok(self .task_client .to_owned() diff --git a/src/tests/sqlsmith/tests/frontend/mod.rs b/src/tests/sqlsmith/tests/frontend/mod.rs index 6eedb4e41876b..bf884d8a83955 100644 --- a/src/tests/sqlsmith/tests/frontend/mod.rs +++ b/src/tests/sqlsmith/tests/frontend/mod.rs @@ -185,6 +185,7 @@ fn run_batch_query( let bound = binder .bind(stmt) .map_err(|e| Failed::from(format!("Failed to bind:\nReason:\n{}", e.as_report())))?; + let scan_tables = bound.scan_tables(); let mut planner = Planner::new(context); let mut plan_root = planner.plan(bound).map_err(|e| { Failed::from(format!( @@ -192,7 +193,7 @@ fn run_batch_query( e.as_report() )) })?; - plan_root.gen_batch_plan().map_err(|e| { + plan_root.gen_batch_plan(&scan_tables).map_err(|e| { Failed::from(format!( "Failed to generate batch plan:\nReason:\n{}", e.as_report() From ab43fbd0cb4db06ca25e5cd65cfd1c073a307eb8 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 18 Sep 2024 19:11:32 +0800 Subject: [PATCH 05/27] rename --- src/batch/src/execution/grpc_exchange.rs | 2 +- src/rpc_client/src/compute_client.rs | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/batch/src/execution/grpc_exchange.rs b/src/batch/src/execution/grpc_exchange.rs index 35eedca8b6da7..27f64b1d5ed48 100644 --- a/src/batch/src/execution/grpc_exchange.rs +++ b/src/batch/src/execution/grpc_exchange.rs @@ -54,7 +54,7 @@ impl GrpcExchangeSource { tracing_context: plan.tracing_context, expr_context: Some(capture_expr_context()?), }; - client.execute1(execute_request).await? + client.execute(execute_request).await? } None => client.get_data(task_output_id.clone()).await?, }; diff --git a/src/rpc_client/src/compute_client.rs b/src/rpc_client/src/compute_client.rs index b4d0ba4e77546..c065bb6935954 100644 --- a/src/rpc_client/src/compute_client.rs +++ b/src/rpc_client/src/compute_client.rs @@ -185,7 +185,7 @@ impl ComputeClient { .into_inner()) } - pub async fn execute1(&self, req: ExecuteRequest) -> Result> { + pub async fn execute(&self, req: ExecuteRequest) -> Result> { Ok(self .task_client .to_owned() From d74fe42c4bfb9aa1ca63e376c42a43f8d764d09b Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 18 Sep 2024 19:28:10 +0800 Subject: [PATCH 06/27] fix recursive --- src/frontend/src/binder/relation/mod.rs | 8 ++++---- src/frontend/src/binder/set_expr.rs | 10 +++++----- src/frontend/src/binder/statement.rs | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 1844d9161e247..1b7b46de7deba 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -138,7 +138,7 @@ impl Relation { } } - pub fn visit_all_scan_table(&self, mut visiter: impl FnMut(TableId)) { + pub fn visit_all_scan_table(&self, visiter: &mut impl FnMut(TableId)) { match self { Relation::Source(_) => {} Relation::BaseTable(table) => { @@ -146,11 +146,11 @@ impl Relation { } Relation::SystemTable(_) => {} Relation::Subquery(subquery) => { - subquery.query.body.visit_all_scan_table_id(&mut visiter); + subquery.query.body.visit_all_scan_table_id(visiter); } Relation::Join(join) | Relation::Apply(join) => { - join.left.visit_all_scan_table(&mut visiter); - join.right.visit_all_scan_table(&mut visiter); + join.left.visit_all_scan_table(visiter); + join.right.visit_all_scan_table(visiter); } Relation::WindowTableFunction(_) => {} Relation::TableFunction { .. } => {} diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index fed540d8e928c..c4919ef8560f5 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -144,20 +144,20 @@ impl BoundSetExpr { } } - pub fn visit_all_scan_table_id(&self, mut visiter: impl FnMut(TableId)) { + pub fn visit_all_scan_table_id(&self, visiter: &mut impl FnMut(TableId)) { match self { BoundSetExpr::Select(select) => { if let Some(relation) = &select.from { - relation.visit_all_scan_table(&mut visiter); + relation.visit_all_scan_table(visiter); } } BoundSetExpr::Query(query) => { - query.body.visit_all_scan_table_id(&mut visiter); + query.body.visit_all_scan_table_id(visiter); } BoundSetExpr::Values(_) => {} BoundSetExpr::SetOperation { left, right, .. } => { - left.visit_all_scan_table_id(&mut visiter); - right.visit_all_scan_table_id(&mut visiter); + left.visit_all_scan_table_id(visiter); + right.visit_all_scan_table_id(visiter); } } } diff --git a/src/frontend/src/binder/statement.rs b/src/frontend/src/binder/statement.rs index addff64c7e145..102f9625bd326 100644 --- a/src/frontend/src/binder/statement.rs +++ b/src/frontend/src/binder/statement.rs @@ -63,7 +63,7 @@ impl BoundStatement { pub fn scan_tables(&self) -> HashSet { if let BoundStatement::Query(query) = self { let mut tables = HashSet::new(); - query.body.visit_all_scan_table_id(|table_id| { + query.body.visit_all_scan_table_id(&mut |table_id| { tables.insert(table_id); }); tables From 04b112c93ec93cb2ef1bac4cfa45334116c3aad5 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 19 Sep 2024 18:08:03 +0800 Subject: [PATCH 07/27] fix test --- .../tests/testdata/input/share.yaml | 2 +- .../tests/testdata/output/share.yaml | 24 +++++++------------ src/frontend/src/scheduler/snapshot.rs | 23 ++++++++++++++++-- src/frontend/src/session.rs | 9 ++++--- src/frontend/src/test_utils.rs | 10 +++++++- 5 files changed, 46 insertions(+), 22 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/share.yaml b/src/frontend/planner_test/tests/testdata/input/share.yaml index efadb03e2d439..510c32febba4e 100644 --- a/src/frontend/planner_test/tests/testdata/input/share.yaml +++ b/src/frontend/planner_test/tests/testdata/input/share.yaml @@ -90,7 +90,7 @@ before: - create_sources sql: | - select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > now() - INTERVAL '1' SECOND; + select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > '2021-04-01 00:00:00+00:00'::Timestamptz - INTERVAL '1' SECOND; expected_outputs: - batch_plan - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2cf3aee9fe043..2cfafbeca08bb 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -291,7 +291,7 @@ before: - create_sources sql: | - select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > now() - INTERVAL '1' SECOND; + select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > '2021-04-01 00:00:00+00:00'::Timestamptz - INTERVAL '1' SECOND; batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time] } @@ -306,24 +306,18 @@ stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } - └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } ├─StreamExchange { dist: HashShard(id) } - │ └─StreamProject { exprs: [id, date_time, _row_id] } - │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } - │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } - │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } - │ │ └─StreamShare { id: 4 } - │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - │ └─StreamExchange { dist: Broadcast } - │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } - │ └─StreamNow { output: [now] } + │ └─StreamFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } + │ └─StreamShare { id: 4 } + │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + │ └─StreamFilter { predicate: (((initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz)) OR (initial_bid = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } └─StreamExchange { dist: HashShard(id) } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamShare { id: 4 } └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + └─StreamFilter { predicate: (((initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz)) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index c01499671eca9..47b619282c7fe 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -21,13 +21,13 @@ use std::time::Duration; use anyhow::anyhow; use risingwave_common::catalog::TableId; use risingwave_common::must_match; -use risingwave_common::util::epoch::Epoch; +use risingwave_common::util::epoch::{Epoch, INVALID_EPOCH}; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_hummock_sdk::{ FrontendHummockVersion, FrontendHummockVersionDelta, HummockVersionId, INVALID_VERSION_ID, }; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; -use risingwave_pb::hummock::{HummockVersionDeltas, PbHummockSnapshot}; +use risingwave_pb::hummock::{HummockVersionDeltas, PbHummockSnapshot, StateTableInfoDelta}; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::watch; @@ -232,6 +232,25 @@ impl HummockSnapshotManager { }) } + pub fn add_table_for_test(&self, table_id: TableId) { + self.update_inner(|version| { + let mut version = version.clone(); + version.id = version.id.next(); + version.state_table_info.apply_delta( + &HashMap::from_iter([( + table_id, + StateTableInfoDelta { + committed_epoch: INVALID_EPOCH, + safe_epoch: INVALID_EPOCH, + compaction_group_id: 0, + }, + )]), + &HashSet::new(), + ); + Some(version) + }); + } + fn update_inner( &self, get_new_snapshot: impl FnOnce(&FrontendHummockVersion) -> Option, diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index a1150798951cb..88d1ae2e92239 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -184,14 +184,17 @@ impl FrontendEnv { use crate::test_utils::{MockCatalogWriter, MockFrontendMetaClient, MockUserInfoWriter}; let catalog = Arc::new(RwLock::new(Catalog::default())); - let catalog_writer = Arc::new(MockCatalogWriter::new(catalog.clone())); + let meta_client = Arc::new(MockFrontendMetaClient {}); + let hummock_snapshot_manager = Arc::new(HummockSnapshotManager::new(meta_client.clone())); + let catalog_writer = Arc::new(MockCatalogWriter::new( + catalog.clone(), + hummock_snapshot_manager.clone(), + )); let catalog_reader = CatalogReader::new(catalog); let user_info_manager = Arc::new(RwLock::new(UserInfoManager::default())); let user_info_writer = Arc::new(MockUserInfoWriter::new(user_info_manager.clone())); let user_info_reader = UserInfoReader::new(user_info_manager); let worker_node_manager = Arc::new(WorkerNodeManager::mock(vec![])); - let meta_client = Arc::new(MockFrontendMetaClient {}); - let hummock_snapshot_manager = Arc::new(HummockSnapshotManager::new(meta_client.clone())); let system_params_manager = Arc::new(LocalSystemParamsManager::for_test()); let compute_client_pool = Arc::new(ComputeClientPool::for_test()); let query_manager = QueryManager::new( diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 6123889262155..aaec1919e943c 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -71,6 +71,7 @@ use crate::catalog::{ConnectionId, DatabaseId, SchemaId, SecretId}; use crate::error::{ErrorCode, Result}; use crate::handler::RwPgResponse; use crate::meta_client::FrontendMetaClient; +use crate::scheduler::HummockSnapshotManagerRef; use crate::session::{AuthContext, FrontendEnv, SessionImpl}; use crate::user::user_manager::UserInfoManager; use crate::user::user_service::UserInfoWriter; @@ -233,6 +234,7 @@ pub struct MockCatalogWriter { id: AtomicU32, table_id_to_schema_id: RwLock>, schema_id_to_database_id: RwLock>, + hummock_snapshot_manager: HummockSnapshotManagerRef, } #[async_trait::async_trait] @@ -279,6 +281,8 @@ impl CatalogWriter for MockCatalogWriter { table.stream_job_status = PbStreamJobStatus::Created as _; self.catalog.write().create_table(&table); self.add_table_or_source_id(table.id, table.schema_id, table.database_id); + self.hummock_snapshot_manager + .add_table_for_test(TableId::new(table.id)); Ok(()) } @@ -668,7 +672,10 @@ impl CatalogWriter for MockCatalogWriter { } impl MockCatalogWriter { - pub fn new(catalog: Arc>) -> Self { + pub fn new( + catalog: Arc>, + hummock_snapshot_manager: HummockSnapshotManagerRef, + ) -> Self { catalog.write().create_database(&PbDatabase { id: 0, name: DEFAULT_DATABASE_NAME.to_string(), @@ -701,6 +708,7 @@ impl MockCatalogWriter { id: AtomicU32::new(3), table_id_to_schema_id: Default::default(), schema_id_to_database_id: RwLock::new(map), + hummock_snapshot_manager, } } From 9808588e5fe811bd9378273f133b7948927d9485 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 20 Sep 2024 13:03:12 +0800 Subject: [PATCH 08/27] avoid recalculate --- .../tests/testdata/input/share.yaml | 2 +- .../tests/testdata/output/share.yaml | 24 ++++++++++------ src/frontend/src/handler/query.rs | 2 +- src/frontend/src/scheduler/local.rs | 28 ++++++++++--------- src/frontend/src/scheduler/snapshot.rs | 2 +- 5 files changed, 33 insertions(+), 25 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/share.yaml b/src/frontend/planner_test/tests/testdata/input/share.yaml index 510c32febba4e..efadb03e2d439 100644 --- a/src/frontend/planner_test/tests/testdata/input/share.yaml +++ b/src/frontend/planner_test/tests/testdata/input/share.yaml @@ -90,7 +90,7 @@ before: - create_sources sql: | - select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > '2021-04-01 00:00:00+00:00'::Timestamptz - INTERVAL '1' SECOND; + select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > now() - INTERVAL '1' SECOND; expected_outputs: - batch_plan - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2cfafbeca08bb..2cf3aee9fe043 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -291,7 +291,7 @@ before: - create_sources sql: | - select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > '2021-04-01 00:00:00+00:00'::Timestamptz - INTERVAL '1' SECOND; + select A.id as a_id, B.id as b_id, A.date_time as A_ts, B.date_time as B_ts from auction A join auction B on A.id = B.id where A.initial_bid = 1 and B.initial_bid = 2 and A.date_time > now() - INTERVAL '1' SECOND; batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time] } @@ -306,18 +306,24 @@ stream_plan: |- StreamMaterialize { columns: [a_id, b_id, a_ts, b_ts, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, a_id], pk_columns: [_row_id, _row_id#1, a_id], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(id, _row_id, _row_id) } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } + └─StreamHashJoin { type: Inner, predicate: id = id, output: [id, id, date_time, date_time, _row_id, _row_id] } ├─StreamExchange { dist: HashShard(id) } - │ └─StreamFilter { predicate: (initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz) } - │ └─StreamShare { id: 4 } - │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - │ └─StreamFilter { predicate: (((initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz)) OR (initial_bid = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + │ └─StreamProject { exprs: [id, date_time, _row_id] } + │ └─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [id, date_time, $expr1, _row_id], cleaned_by_watermark: true } + │ ├─StreamProject { exprs: [id, date_time, AtTimeZone(date_time, 'UTC':Varchar) as $expr1, _row_id] } + │ │ └─StreamFilter { predicate: (initial_bid = 1:Int32) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } + │ │ └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:00:01':Interval, 'UTC':Varchar) as $expr2], output_watermarks: [$expr2] } + │ └─StreamNow { output: [now] } └─StreamExchange { dist: HashShard(id) } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamShare { id: 4 } └─StreamProject { exprs: [id, initial_bid, date_time, _row_id] } - └─StreamFilter { predicate: (((initial_bid = 1:Int32) AND (AtTimeZone(date_time, 'UTC':Varchar) > '2021-03-31 23:59:59+00:00':Timestamptz)) OR (initial_bid = 2:Int32)) } + └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 37f40ed53ded8..24d4b8ba61209 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -527,7 +527,7 @@ pub async fn local_execute( // TODO: Passing sql here let execution = - LocalQueryExecution::new(query, front_env.clone(), "", snapshot, session, timeout); + LocalQueryExecution::new(query, front_env.clone(), "", snapshot, session, timeout)?; Ok(execution.stream_rows()) } diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index 0dd7b77fc0823..b423dcce03f2f 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -41,7 +41,7 @@ use risingwave_pb::batch_plan::{ ExchangeInfo, ExchangeSource, LocalExecutePlan, PbTaskId, PlanFragment, PlanNode as PbPlanNode, TaskOutputId, }; -use risingwave_pb::common::WorkerNode; +use risingwave_pb::common::{BatchQueryEpoch, WorkerNode}; use tokio::sync::mpsc; use tokio_stream::wrappers::ReceiverStream; use tracing::debug; @@ -62,8 +62,8 @@ pub struct LocalQueryExecution { query: Query, front_env: FrontendEnv, // The snapshot will be released when LocalQueryExecution is dropped. - // TODO - snapshot: QuerySnapshot, + _snapshot: QuerySnapshot, + batch_query_epoch: BatchQueryEpoch, session: Arc, worker_node_manager: WorkerNodeSelector, timeout: Option, @@ -77,22 +77,24 @@ impl LocalQueryExecution { snapshot: QuerySnapshot, session: Arc, timeout: Option, - ) -> Self { + ) -> SchedulerResult { let sql = sql.into(); let worker_node_manager = WorkerNodeSelector::new( front_env.worker_node_manager_ref(), snapshot.support_barrier_read(), ); + let batch_query_epoch = snapshot.batch_query_epoch()?; - Self { + Ok(Self { sql, query, front_env, - snapshot, + _snapshot: snapshot, + batch_query_epoch, session, worker_node_manager, timeout, - } + }) } fn shutdown_rx(&self) -> ShutdownToken { @@ -118,7 +120,7 @@ impl LocalQueryExecution { &plan_node, &task_id, context, - self.snapshot.batch_query_epoch()?, + self.batch_query_epoch, self.shutdown_rx().clone(), ); let executor = executor.build().await?; @@ -133,7 +135,7 @@ impl LocalQueryExecution { let span = tracing::info_span!( "local_execute", query_id = self.query.query_id.id, - epoch = ?self.snapshot.batch_query_epoch(), + epoch = ?self.batch_query_epoch, ); Box::pin(self.run_inner().instrument(span)) } @@ -339,7 +341,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()?), + epoch: Some(self.batch_query_epoch), tracing_context: tracing_context.clone(), }; let exchange_source = ExchangeSource { @@ -383,7 +385,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()?), + epoch: Some(self.batch_query_epoch), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -422,7 +424,7 @@ impl LocalQueryExecution { }; let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()?), + epoch: Some(self.batch_query_epoch), tracing_context: tracing_context.clone(), }; // NOTE: select a random work node here. @@ -458,7 +460,7 @@ impl LocalQueryExecution { let local_execute_plan = LocalExecutePlan { plan: Some(second_stage_plan_fragment), - epoch: Some(self.snapshot.batch_query_epoch()?), + epoch: Some(self.batch_query_epoch), tracing_context, }; diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 47b619282c7fe..3a14bbc92b6a8 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -143,7 +143,7 @@ impl PinnedSnapshot { } }) })? - .unwrap_or_else(Epoch::now); // When no table is involved, use current timestamp as epoch + .unwrap_or(Epoch(self.value.max_committed_epoch)); Ok(epoch) } From 201dde3ee1eadf968cca4d957028e71b7ab88923 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 20 Sep 2024 13:06:20 +0800 Subject: [PATCH 09/27] fix typo --- src/frontend/src/binder/relation/mod.rs | 10 +++++----- src/frontend/src/binder/set_expr.rs | 10 +++++----- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 1b7b46de7deba..2442717abbcb5 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -138,19 +138,19 @@ impl Relation { } } - pub fn visit_all_scan_table(&self, visiter: &mut impl FnMut(TableId)) { + pub fn visit_all_scan_table(&self, visitor: &mut impl FnMut(TableId)) { match self { Relation::Source(_) => {} Relation::BaseTable(table) => { - visiter(table.table_id); + visitor(table.table_id); } Relation::SystemTable(_) => {} Relation::Subquery(subquery) => { - subquery.query.body.visit_all_scan_table_id(visiter); + subquery.query.body.visit_all_scan_table_id(visitor); } Relation::Join(join) | Relation::Apply(join) => { - join.left.visit_all_scan_table(visiter); - join.right.visit_all_scan_table(visiter); + join.left.visit_all_scan_table(visitor); + join.right.visit_all_scan_table(visitor); } Relation::WindowTableFunction(_) => {} Relation::TableFunction { .. } => {} diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index c4919ef8560f5..6d0707a52241b 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -144,20 +144,20 @@ impl BoundSetExpr { } } - pub fn visit_all_scan_table_id(&self, visiter: &mut impl FnMut(TableId)) { + pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { match self { BoundSetExpr::Select(select) => { if let Some(relation) = &select.from { - relation.visit_all_scan_table(visiter); + relation.visit_all_scan_table(visitor); } } BoundSetExpr::Query(query) => { - query.body.visit_all_scan_table_id(visiter); + query.body.visit_all_scan_table_id(visitor); } BoundSetExpr::Values(_) => {} BoundSetExpr::SetOperation { left, right, .. } => { - left.visit_all_scan_table_id(visiter); - right.visit_all_scan_table_id(visiter); + left.visit_all_scan_table_id(visitor); + right.visit_all_scan_table_id(visitor); } } } From f1cf8ec08052b2c3c315188c24009c9843d84690 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 20 Sep 2024 16:17:08 +0800 Subject: [PATCH 10/27] feat(storage): per table try wait epoch --- proto/stream_service.proto | 1 + src/compute/src/rpc/service/stream_service.rs | 11 +++- src/meta/src/barrier/command.rs | 21 ++++++-- src/meta/src/barrier/mod.rs | 5 +- src/storage/hummock_sdk/src/version.rs | 7 +++ .../src/bin/replay/replay_impl.rs | 10 ++-- .../hummock_test/src/failpoint_tests.rs | 13 +++-- .../hummock_test/src/hummock_storage_tests.rs | 24 ++++----- .../hummock_test/src/snapshot_tests.rs | 22 ++++++-- .../hummock_test/src/state_store_tests.rs | 28 +++++++--- src/storage/hummock_test/src/test_utils.rs | 7 ++- src/storage/hummock_trace/src/collector.rs | 12 +++-- src/storage/hummock_trace/src/opts.rs | 5 ++ src/storage/hummock_trace/src/record.rs | 4 +- src/storage/hummock_trace/src/replay/mod.rs | 10 ++-- .../hummock_trace/src/replay/worker.rs | 4 +- .../event_handler/hummock_event_handler.rs | 19 +++---- .../src/hummock/event_handler/uploader/mod.rs | 5 +- .../hummock/local_version/recent_versions.rs | 7 +-- .../src/hummock/store/hummock_storage.rs | 17 +++---- .../hummock/store/local_hummock_storage.rs | 18 +++++-- src/storage/src/hummock/store/version.rs | 5 +- src/storage/src/hummock/utils.rs | 51 +++++++++++++++---- src/storage/src/memory.rs | 6 ++- src/storage/src/monitor/monitored_store.rs | 3 +- src/storage/src/monitor/traced_store.rs | 10 ++-- src/storage/src/panic_store.rs | 6 ++- src/storage/src/store.rs | 32 +++++++++++- src/storage/src/store_impl.rs | 20 ++++++-- .../src/table/batch_table/storage_table.rs | 26 ++++++++-- .../common/log_store_impl/kv_log_store/mod.rs | 21 ++++++-- .../source/source_backfill_executor.rs | 7 ++- .../src/executor/source/source_executor.rs | 11 +++- 33 files changed, 327 insertions(+), 121 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index c13ee8875b43f..ab56c9f7e4050 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -49,6 +49,7 @@ message BarrierCompleteResponse { message WaitEpochCommitRequest { uint64 epoch = 1; + uint32 table_id = 2; } message WaitEpochCommitResponse { diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index 6253cfe74c730..e8d403a9693cc 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -17,6 +17,7 @@ use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_pb::stream_service::stream_service_server::StreamService; use risingwave_pb::stream_service::*; use risingwave_storage::dispatch_state_store; +use risingwave_storage::store::TryWaitEpochOptions; use risingwave_stream::error::StreamError; use risingwave_stream::task::{LocalStreamManager, StreamEnvironment}; use tokio::sync::mpsc::unbounded_channel; @@ -45,14 +46,20 @@ impl StreamService for StreamServiceImpl { &self, request: Request, ) -> Result, Status> { - let epoch = request.into_inner().epoch; + let request = request.into_inner(); + let epoch = request.epoch; dispatch_state_store!(self.env.state_store(), store, { use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_storage::StateStore; store - .try_wait_epoch(HummockReadEpoch::Committed(epoch)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch), + TryWaitEpochOptions { + table_id: request.table_id.into(), + }, + ) .instrument_await(format!("wait_epoch_commit (epoch {})", epoch)) .await .map_err(StreamError::from)?; diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index c18ad5d0f2b3b..1eff171d019b2 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -22,7 +22,6 @@ use risingwave_common::hash::ActorMapping; use risingwave_common::types::Timestamptz; use risingwave_common::util::epoch::Epoch; use risingwave_connector::source::SplitImpl; -use risingwave_hummock_sdk::HummockEpoch; use risingwave_pb::catalog::{CreateType, Table}; use risingwave_pb::common::PbWorkerNode; use risingwave_pb::meta::table_fragments::PbActorStatus; @@ -442,6 +441,8 @@ pub struct CommandContext { pub prev_epoch: TracedEpoch, pub curr_epoch: TracedEpoch, + pub table_ids_to_commit: HashSet, + pub current_paused_reason: Option, pub command: Command, @@ -470,12 +471,12 @@ impl std::fmt::Debug for CommandContext { } impl CommandContext { - #[allow(clippy::too_many_arguments)] pub(super) fn new( node_map: HashMap, subscription_info: InflightSubscriptionInfo, prev_epoch: TracedEpoch, curr_epoch: TracedEpoch, + table_ids_to_commit: HashSet, current_paused_reason: Option, command: Command, kind: BarrierKind, @@ -487,6 +488,7 @@ impl CommandContext { subscription_info, prev_epoch, curr_epoch, + table_ids_to_commit, current_paused_reason, command, kind, @@ -945,7 +947,13 @@ impl Command { } impl CommandContext { - pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { + pub async fn wait_epoch_commit(&self) -> MetaResult<()> { + let table_id = self.table_ids_to_commit.iter().next().cloned(); + // try wait epoch on an existing random table id + let Some(table_id) = table_id else { + // no need to wait epoch when there is no table id + return Ok(()); + }; let futures = self.node_map.values().map(|worker_node| async { let client = self .barrier_manager_context @@ -953,7 +961,10 @@ impl CommandContext { .stream_client_pool() .get(worker_node) .await?; - let request = WaitEpochCommitRequest { epoch }; + let request = WaitEpochCommitRequest { + epoch: self.prev_epoch.value().0, + table_id: table_id.table_id, + }; client.wait_epoch_commit(request).await }); @@ -976,7 +987,7 @@ impl CommandContext { // storage version with this epoch is synced to all compute nodes before the // execution of the next command of `Update`, as some newly created operators // may immediately initialize their states on that barrier. - self.wait_epoch_commit(self.prev_epoch.value().0).await?; + self.wait_epoch_commit().await?; } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 5bb76ee46133c..e05719f41dde4 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -1029,11 +1029,14 @@ impl GlobalBarrierManager { }); span.record("epoch", curr_epoch.value().0); + let table_ids_to_commit: HashSet<_> = pre_applied_graph_info.existing_table_ids().collect(); + let command_ctx = Arc::new(CommandContext::new( self.active_streaming_nodes.current().clone(), pre_applied_subscription_info, prev_epoch.clone(), curr_epoch.clone(), + table_ids_to_commit.clone(), self.state.paused_reason(), command, kind, @@ -1043,8 +1046,6 @@ impl GlobalBarrierManager { send_latency_timer.observe_duration(); - let table_ids_to_commit: HashSet<_> = pre_applied_graph_info.existing_table_ids().collect(); - let mut jobs_to_wait = HashSet::new(); for (table_id, creating_job) in &mut self.checkpoint_control.creating_streaming_job_controls diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index fe2825cc8ad0c..954426384c818 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -428,6 +428,13 @@ impl HummockVersion { self.max_committed_epoch } + pub fn table_committed_epoch(&self, table_id: TableId) -> Option { + self.state_table_info + .info() + .get(&table_id) + .map(|info| info.committed_epoch) + } + pub fn visible_table_committed_epoch(&self) -> u64 { self.max_committed_epoch } 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 282409f394476..6653db94e0543 100644 --- a/src/storage/hummock_test/src/bin/replay/replay_impl.rs +++ b/src/storage/hummock_test/src/bin/replay/replay_impl.rs @@ -25,7 +25,7 @@ use risingwave_hummock_sdk::{HummockReadEpoch, HummockVersionId, SyncResult}; use risingwave_hummock_trace::{ GlobalReplay, LocalReplay, LocalReplayRead, ReplayItem, ReplayRead, ReplayStateStore, ReplayWrite, Result, TraceError, TracedBytes, TracedInitOptions, TracedNewLocalOptions, - TracedReadOptions, TracedSealCurrentEpochOptions, TracedSubResp, + TracedReadOptions, TracedSealCurrentEpochOptions, TracedSubResp, TracedTryWaitEpochOptions, }; use risingwave_meta::manager::{MessageStatus, MetaSrvEnv, NotificationManagerRef, WorkerKey}; use risingwave_pb::common::WorkerNode; @@ -170,9 +170,13 @@ impl ReplayStateStore for GlobalReplayImpl { Box::new(LocalReplayImpl(local_storage)) } - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> Result<()> { + async fn try_wait_epoch( + &self, + epoch: HummockReadEpoch, + options: TracedTryWaitEpochOptions, + ) -> Result<()> { self.store - .try_wait_epoch(epoch) + .try_wait_epoch(epoch, options.into()) .await .map_err(|_| TraceError::TryWaitEpochFailed)?; Ok(()) diff --git a/src/storage/hummock_test/src/failpoint_tests.rs b/src/storage/hummock_test/src/failpoint_tests.rs index 27072abba08f2..cba7851c8cf88 100644 --- a/src/storage/hummock_test/src/failpoint_tests.rs +++ b/src/storage/hummock_test/src/failpoint_tests.rs @@ -30,7 +30,8 @@ use risingwave_storage::hummock::test_utils::{count_stream, default_opts_for_tes use risingwave_storage::hummock::{CachePolicy, HummockStorage}; use risingwave_storage::storage_value::StorageValue; use risingwave_storage::store::{ - LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreRead, WriteOptions, + LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, StateStoreRead, + TryWaitEpochOptions, WriteOptions, }; use risingwave_storage::StateStore; @@ -148,7 +149,10 @@ async fn test_failpoints_state_store_read_upload() { .unwrap(); meta_client.commit_epoch(1, res, false).await.unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(1)) + .try_wait_epoch( + HummockReadEpoch::Committed(1), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); // clear block cache @@ -225,7 +229,10 @@ async fn test_failpoints_state_store_read_upload() { .unwrap(); meta_client.commit_epoch(3, res, false).await.unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(3)) + .try_wait_epoch( + HummockReadEpoch::Committed(3), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 18bad67a62570..b938a7d6e036f 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -568,7 +568,7 @@ async fn test_state_store_sync() { .commit_epoch(epoch1, res, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch1).await; + test_env.wait_sync_committed_version().await; { // after sync 1 epoch let read_version = hummock_storage.read_version(); @@ -614,7 +614,7 @@ async fn test_state_store_sync() { .commit_epoch(epoch2, res, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; { // after sync all epoch let read_version = hummock_storage.read_version(); @@ -910,7 +910,7 @@ async fn test_delete_get() { .commit_epoch(epoch2, res, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; assert!(test_env .storage .get( @@ -1096,7 +1096,7 @@ async fn test_multiple_epoch_sync() { .commit_epoch(epoch3, sync_result3, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch3).await; + test_env.wait_sync_committed_version().await; test_get().await; } @@ -1270,7 +1270,7 @@ async fn test_iter_with_min_epoch() { .commit_epoch(epoch2, sync_result2, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; { let iter = test_env @@ -1561,7 +1561,7 @@ async fn test_hummock_version_reader() { .commit_epoch(epoch1, sync_result1, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch1).await; + test_env.wait_sync_committed_version().await; let sync_result2 = test_env .storage @@ -1573,7 +1573,7 @@ async fn test_hummock_version_reader() { .commit_epoch(epoch2, sync_result2, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; let sync_result3 = test_env .storage @@ -1585,7 +1585,7 @@ async fn test_hummock_version_reader() { .commit_epoch(epoch3, sync_result3, false) .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch3).await; + test_env.wait_sync_committed_version().await; { let (_, read_snapshot) = read_filter_for_version( epoch1, @@ -1979,7 +1979,7 @@ async fn test_get_with_min_epoch() { .await .unwrap(); - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; let k = gen_key(0); let prefix_hint = { let mut ret = Vec::with_capacity(TABLE_PREFIX_LEN + k.len()); @@ -2409,7 +2409,7 @@ async fn test_table_watermark() { }; test_env.commit_epoch(epoch1).await; - test_env.storage.try_wait_epoch_for_test(epoch1).await; + test_env.wait_sync_committed_version().await; let (local1, local2) = test_after_epoch2(local1, local2).await; @@ -2498,7 +2498,7 @@ async fn test_table_watermark() { let (local1, local2) = test_after_epoch2(local1, local2).await; test_env.commit_epoch(epoch2).await; - test_env.storage.try_wait_epoch_for_test(epoch2).await; + test_env.wait_sync_committed_version().await; test_global_read(test_env.storage.clone(), epoch2).await; @@ -2533,7 +2533,7 @@ async fn test_table_watermark() { let (local1, local2) = test_after_epoch2(local1, local2).await; test_env.commit_epoch(epoch3).await; - test_env.storage.try_wait_epoch_for_test(epoch3).await; + test_env.wait_sync_committed_version().await; check_version_table_watermark(test_env.storage.get_pinned_version()); diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index b15e8a3fa372c..503dd38c57d7e 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -27,7 +27,7 @@ use risingwave_storage::hummock::{CachePolicy, HummockStorage}; use risingwave_storage::storage_value::StorageValue; use risingwave_storage::store::{ LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, SealCurrentEpochOptions, - StateStoreRead, WriteOptions, + StateStoreRead, TryWaitEpochOptions, WriteOptions, }; use risingwave_storage::StateStore; @@ -149,7 +149,10 @@ async fn test_snapshot_inner( .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch1)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch1), + TryWaitEpochOptions::for_test(Default::default()), + ) .await .unwrap(); } @@ -193,7 +196,10 @@ async fn test_snapshot_inner( .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch2), + TryWaitEpochOptions::for_test(Default::default()), + ) .await .unwrap(); } @@ -236,7 +242,10 @@ async fn test_snapshot_inner( .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch3)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch3), + TryWaitEpochOptions::for_test(Default::default()), + ) .await .unwrap(); } @@ -298,7 +307,10 @@ async fn test_snapshot_range_scan_inner( .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); } diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index ab1e84aca2a66..828347f401236 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -385,7 +385,10 @@ async fn test_basic_v2() { .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch1)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch1), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); let value = hummock_storage @@ -1102,7 +1105,10 @@ async fn test_delete_get_v2() { .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch2), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); assert!(hummock_storage @@ -1263,7 +1269,10 @@ async fn test_multiple_epoch_sync_v2() { .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch3)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch3), + TryWaitEpochOptions::for_test(local.table_id()), + ) .await .unwrap(); test_get().await; @@ -1365,7 +1374,10 @@ async fn test_gc_watermark_and_clear_shared_buffer() { .await .unwrap(); hummock_storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch1)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch1), + TryWaitEpochOptions::for_test(local_hummock_storage.table_id()), + ) .await .unwrap(); @@ -1639,8 +1651,12 @@ async fn test_iter_log() { } hummock_storage - .try_wait_epoch_for_test(test_log_data.last().unwrap().0) - .await; + .try_wait_epoch( + HummockReadEpoch::Committed(test_log_data.last().unwrap().0), + TryWaitEpochOptions { table_id }, + ) + .await + .unwrap(); let verify_state_store = VerifyStateStore { actual: hummock_storage, diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index da861ff92810c..5a613c1d54eff 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -262,7 +262,12 @@ impl HummockTestEnv { .await .unwrap(); - self.storage.try_wait_epoch_for_test(epoch).await; + self.wait_sync_committed_version().await; + } + + pub async fn wait_sync_committed_version(&self) { + let version = self.manager.get_current_version().await; + self.storage.wait_version(version).await; } } diff --git a/src/storage/hummock_trace/src/collector.rs b/src/storage/hummock_trace/src/collector.rs index 979bf067db861..068cbdcee45ed 100644 --- a/src/storage/hummock_trace/src/collector.rs +++ b/src/storage/hummock_trace/src/collector.rs @@ -37,7 +37,7 @@ use crate::write::{TraceWriter, TraceWriterImpl}; use crate::{ ConcurrentIdGenerator, Operation, OperationResult, Record, RecordId, RecordIdGenerator, TracedInitOptions, TracedNewLocalOptions, TracedReadOptions, TracedSealCurrentEpochOptions, - TracedSubResp, UniqueIdGenerator, + TracedSubResp, TracedTryWaitEpochOptions, UniqueIdGenerator, }; // Global collector instance used for trace collection @@ -216,8 +216,14 @@ impl TraceSpan { Self::new_global_op(Operation::SealCurrentEpoch { epoch, opts }, storage_type) } - pub fn new_try_wait_epoch_span(epoch: HummockReadEpoch) -> MayTraceSpan { - Self::new_global_op(Operation::TryWaitEpoch(epoch.into()), StorageType::Global) + pub fn new_try_wait_epoch_span( + epoch: HummockReadEpoch, + options: TracedTryWaitEpochOptions, + ) -> MayTraceSpan { + Self::new_global_op( + Operation::TryWaitEpoch(epoch.into(), options), + StorageType::Global, + ) } pub fn new_get_span( diff --git a/src/storage/hummock_trace/src/opts.rs b/src/storage/hummock_trace/src/opts.rs index 562e989051395..c3c981a612064 100644 --- a/src/storage/hummock_trace/src/opts.rs +++ b/src/storage/hummock_trace/src/opts.rs @@ -170,6 +170,11 @@ pub struct TracedNewLocalOptions { pub vnodes: TracedBitmap, } +#[derive(Encode, Decode, PartialEq, Debug, Clone)] +pub struct TracedTryWaitEpochOptions { + pub table_id: TracedTableId, +} + #[cfg(test)] impl TracedNewLocalOptions { pub(crate) fn for_test(table_id: u32) -> Self { diff --git a/src/storage/hummock_trace/src/record.rs b/src/storage/hummock_trace/src/record.rs index 2995b7f13c8a7..a9ae562f02b4c 100644 --- a/src/storage/hummock_trace/src/record.rs +++ b/src/storage/hummock_trace/src/record.rs @@ -23,7 +23,7 @@ use risingwave_pb::meta::SubscribeResponse; use crate::{ LocalStorageId, StorageType, TracedHummockReadEpoch, TracedInitOptions, TracedNewLocalOptions, - TracedReadOptions, TracedSealCurrentEpochOptions, + TracedReadOptions, TracedSealCurrentEpochOptions, TracedTryWaitEpochOptions, }; pub type RecordId = u64; @@ -164,7 +164,7 @@ pub enum Operation { LocalStorageInit(TracedInitOptions), /// Try wait epoch - TryWaitEpoch(TracedHummockReadEpoch), + TryWaitEpoch(TracedHummockReadEpoch, TracedTryWaitEpochOptions), /// Seal current epoch SealCurrentEpoch { diff --git a/src/storage/hummock_trace/src/replay/mod.rs b/src/storage/hummock_trace/src/replay/mod.rs index 91e80cad1c4b6..347ef30704570 100644 --- a/src/storage/hummock_trace/src/replay/mod.rs +++ b/src/storage/hummock_trace/src/replay/mod.rs @@ -33,7 +33,7 @@ use crate::error::Result; use crate::TraceError; use crate::{ LocalStorageId, Record, TracedBytes, TracedInitOptions, TracedNewLocalOptions, - TracedReadOptions, TracedSealCurrentEpochOptions, + TracedReadOptions, TracedSealCurrentEpochOptions, TracedTryWaitEpochOptions, }; pub type ReplayItem = (TracedBytes, TracedBytes); @@ -118,7 +118,11 @@ pub trait ReplayStateStore { async fn sync(&self, id: u64, table_ids: Vec) -> Result; async fn notify_hummock(&self, info: Info, op: RespOperation, version: u64) -> Result; async fn new_local(&self, opts: TracedNewLocalOptions) -> Box; - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> Result<()>; + async fn try_wait_epoch( + &self, + epoch: HummockReadEpoch, + options: TracedTryWaitEpochOptions, + ) -> Result<()>; } // define mock trait for replay interfaces @@ -147,7 +151,7 @@ mock! { async fn notify_hummock(&self, info: Info, op: RespOperation, version: u64, ) -> Result; async fn new_local(&self, opts: TracedNewLocalOptions) -> Box; - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> Result<()>; + async fn try_wait_epoch(&self, epoch: HummockReadEpoch,options: TracedTryWaitEpochOptions) -> Result<()>; } impl GlobalReplay for GlobalReplayInterface{} } diff --git a/src/storage/hummock_trace/src/replay/worker.rs b/src/storage/hummock_trace/src/replay/worker.rs index 8a02c3efde196..08d877cadf3ab 100644 --- a/src/storage/hummock_trace/src/replay/worker.rs +++ b/src/storage/hummock_trace/src/replay/worker.rs @@ -310,11 +310,11 @@ impl ReplayWorker { let local_storage = local_storages.get_mut(&storage_type).unwrap(); local_storage.init(options).await.unwrap(); } - Operation::TryWaitEpoch(epoch) => { + Operation::TryWaitEpoch(epoch, options) => { assert_eq!(storage_type, StorageType::Global); let res = res_rx.recv().await.expect("recv result failed"); if let OperationResult::TryWaitEpoch(expected) = res { - let actual = replay.try_wait_epoch(epoch.into()).await; + let actual = replay.try_wait_epoch(epoch.into(), options).await; assert_eq!(TraceResult::from(actual), expected, "try_wait_epoch wrong"); } else { panic!( 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 1c8abc78ddffc..82cdc14838946 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -197,7 +197,7 @@ pub struct HummockEventHandler { /// A copy of `read_version_mapping` but owned by event handler local_read_version_mapping: HashMap, - version_update_notifier_tx: Arc>, + version_update_notifier_tx: Arc>, recent_versions: Arc>, write_conflict_detector: Option>, @@ -316,8 +316,7 @@ impl HummockEventHandler { ) -> Self { let (hummock_event_tx, hummock_event_rx) = event_channel(state_store_metrics.event_handler_pending_event.clone()); - let (version_update_notifier_tx, _) = - tokio::sync::watch::channel(pinned_version.visible_table_committed_epoch()); + let (version_update_notifier_tx, _) = tokio::sync::watch::channel(pinned_version.clone()); let version_update_notifier_tx = Arc::new(version_update_notifier_tx); let read_version_mapping = Arc::new(RwLock::new(HashMap::default())); let buffer_tracker = BufferTracker::from_storage_opts( @@ -371,7 +370,7 @@ impl HummockEventHandler { } } - pub fn version_update_notifier_tx(&self) -> Arc> { + pub fn version_update_notifier_tx(&self) -> Arc> { self.version_update_notifier_tx.clone() } @@ -648,18 +647,16 @@ impl HummockEventHandler { ); } - let prev_max_committed_epoch = pinned_version.visible_table_committed_epoch(); let max_committed_epoch = new_pinned_version.visible_table_committed_epoch(); // only notify local_version_manager when MCE change self.version_update_notifier_tx.send_if_modified(|state| { - assert_eq!(prev_max_committed_epoch, *state); - if max_committed_epoch > *state { - *state = max_committed_epoch; - true - } else { - false + if state.id() == new_pinned_version.id() { + return false; } + assert!(pinned_version.id() > state.id()); + *state = new_pinned_version.clone(); + true }); if let Some(conflict_detector) = self.write_conflict_detector.as_ref() { diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 90e6a9306930a..3557136c0b96c 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -1195,10 +1195,7 @@ impl HummockUploader { self.context .pinned_version .version() - .state_table_info - .info() - .get(table_id) - .map(|info| info.committed_epoch), + .table_committed_epoch(*table_id), ) }); table_data.new_epoch(epoch); diff --git a/src/storage/src/hummock/local_version/recent_versions.rs b/src/storage/src/hummock/local_version/recent_versions.rs index 8d3f1a015ad6a..c88a15ecdb526 100644 --- a/src/storage/src/hummock/local_version/recent_versions.rs +++ b/src/storage/src/hummock/local_version/recent_versions.rs @@ -140,12 +140,7 @@ impl RecentVersions { ); } let result = self.recent_versions.binary_search_by(|version| { - let committed_epoch = version - .version() - .state_table_info - .info() - .get(&table_id) - .map(|info| info.committed_epoch); + let committed_epoch = version.version().table_committed_epoch(table_id); if let Some(committed_epoch) = committed_epoch { committed_epoch.cmp(&epoch) } else { diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 888de0db1af1c..ae018df24ec8a 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -96,7 +96,7 @@ pub struct HummockStorage { buffer_tracker: BufferTracker, - version_update_notifier_tx: Arc>, + version_update_notifier_tx: Arc>, recent_versions: Arc>, @@ -626,7 +626,11 @@ impl StateStore for HummockStorage { /// Waits until the local hummock version contains the epoch. If `wait_epoch` is `Current`, /// we will only check whether it is le `sealed_epoch` and won't wait. - async fn try_wait_epoch(&self, wait_epoch: HummockReadEpoch) -> StorageResult<()> { + async fn try_wait_epoch( + &self, + wait_epoch: HummockReadEpoch, + options: TryWaitEpochOptions, + ) -> StorageResult<()> { let wait_epoch = match wait_epoch { HummockReadEpoch::Committed(epoch) => { assert!(!is_max_epoch(epoch), "epoch should not be MAX EPOCH"); @@ -634,7 +638,7 @@ impl StateStore for HummockStorage { } _ => return Ok(()), }; - wait_for_epoch(&self.version_update_notifier_tx, wait_epoch).await + wait_for_epoch(&self.version_update_notifier_tx, wait_epoch, options).await } fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture { @@ -698,13 +702,6 @@ impl HummockStorage { self.buffer_tracker.get_buffer_size() } - pub async fn try_wait_epoch_for_test(&self, wait_epoch: u64) { - let mut rx = self.version_update_notifier_tx.subscribe(); - while *(rx.borrow_and_update()) < wait_epoch { - rx.changed().await.unwrap(); - } - } - /// Creates a [`HummockStorage`] with default stats. Should only be used by tests. pub async fn for_test( options: Arc, diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 2221b4350ebc9..ae3815ca551a5 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -25,7 +25,7 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::MAX_SPILL_TIMES; use risingwave_hummock_sdk::key::{is_empty_key_range, vnode_range, TableKey, TableKeyRange}; use risingwave_hummock_sdk::sstable_info::SstableInfo; -use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch}; +use risingwave_hummock_sdk::EpochWithGap; use tracing::{warn, Instrument}; use super::version::{StagingData, VersionUpdate}; @@ -37,6 +37,7 @@ use crate::hummock::iterator::{ Backward, BackwardUserIterator, ConcatIteratorInner, Forward, HummockIteratorUnion, IteratorFactory, MergeIterator, UserIterator, }; +use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::shared_buffer::shared_buffer_batch::{ SharedBufferBatch, SharedBufferBatchIterator, SharedBufferBatchOldValues, SharedBufferItem, SharedBufferValue, @@ -96,7 +97,7 @@ pub struct LocalHummockStorage { write_limiter: WriteLimiterRef, - version_update_notifier_tx: Arc>, + version_update_notifier_tx: Arc>, mem_table_spill_threshold: usize, } @@ -134,8 +135,15 @@ impl LocalHummockStorage { .await } - pub async fn wait_for_epoch(&self, wait_epoch: u64) -> StorageResult<()> { - wait_for_epoch(&self.version_update_notifier_tx, wait_epoch).await + async fn wait_for_epoch(&self, wait_epoch: u64) -> StorageResult<()> { + wait_for_epoch( + &self.version_update_notifier_tx, + wait_epoch, + TryWaitEpochOptions { + table_id: self.table_id, + }, + ) + .await } pub async fn iter_flushed( @@ -658,7 +666,7 @@ impl LocalHummockStorage { memory_limiter: Arc, write_limiter: WriteLimiterRef, option: NewLocalOptions, - version_update_notifier_tx: Arc>, + version_update_notifier_tx: Arc>, mem_table_spill_threshold: usize, ) -> Self { let stats = hummock_version_reader.stats().clone(); diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 7d0dc49847398..23631ba2d81ea 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -428,10 +428,7 @@ impl HummockReadVersion { vnode_watermarks, self.committed .version() - .state_table_info - .info() - .get(&self.table_id) - .map(|info| info.committed_epoch), + .table_committed_epoch(self.table_id), )); } } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index c2f6cbafed79b..b8761da23143e 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -19,26 +19,28 @@ use std::ops::Bound::{Excluded, Included, Unbounded}; use std::ops::{Bound, RangeBounds}; use std::sync::atomic::{AtomicBool, AtomicU64, Ordering as AtomicOrdering}; use std::sync::Arc; -use std::time::Duration; +use std::time::{Duration, Instant}; use bytes::Bytes; use foyer::CacheContext; use parking_lot::Mutex; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::config::StorageMemoryConfig; +use risingwave_hummock_sdk::can_concat; use risingwave_hummock_sdk::key::{ bound_table_key_range, EmptySliceRef, FullKey, TableKey, UserKey, }; use risingwave_hummock_sdk::sstable_info::SstableInfo; -use risingwave_hummock_sdk::{can_concat, HummockEpoch}; use tokio::sync::oneshot::{channel, Receiver, Sender}; +use tracing::warn; use super::{HummockError, SstableStoreRef}; use crate::error::StorageResult; +use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::CachePolicy; use crate::mem_table::{KeyOp, MemTableError}; use crate::monitor::MemoryCollector; -use crate::store::{OpConsistencyLevel, ReadOptions, StateStoreRead}; +use crate::store::{OpConsistencyLevel, ReadOptions, StateStoreRead, TryWaitEpochOptions}; pub fn range_overlap( search_key_range: &R, @@ -575,15 +577,29 @@ pub(crate) fn filter_with_delete_range<'a>( } pub(crate) async fn wait_for_epoch( - notifier: &tokio::sync::watch::Sender, + notifier: &tokio::sync::watch::Sender, wait_epoch: u64, + options: TryWaitEpochOptions, ) -> StorageResult<()> { let mut receiver = notifier.subscribe(); - // avoid unnecessary check in the loop if the value does not change - let max_committed_epoch = *receiver.borrow_and_update(); - if max_committed_epoch >= wait_epoch { - return Ok(()); + { + // avoid unnecessary check in the loop if the value does not change + let committed_epoch = receiver + .borrow_and_update() + .version() + .table_committed_epoch(options.table_id); + if let Some(committed_epoch) = committed_epoch { + if committed_epoch >= wait_epoch { + return Ok(()); + } + } else { + warn!( + table_id = options.table_id.table_id, + "table id not exist yet. wait for table creation" + ); + } } + let start_time = Instant::now(); loop { match tokio::time::timeout(Duration::from_secs(30), receiver.changed()).await { Err(_) => { @@ -598,6 +614,7 @@ pub(crate) async fn wait_for_epoch( // See #3845 for more details. tracing::warn!( epoch = wait_epoch, + elapsed = ?start_time.elapsed(), "wait_epoch timeout when waiting for version update", ); continue; @@ -606,9 +623,21 @@ pub(crate) async fn wait_for_epoch( return Err(HummockError::wait_epoch("tx dropped").into()); } Ok(Ok(_)) => { - let max_committed_epoch = *receiver.borrow(); - if max_committed_epoch >= wait_epoch { - return Ok(()); + // TODO: should handle the corner case of drop table + let committed_epoch = receiver + .borrow() + .version() + .table_committed_epoch(options.table_id); + if let Some(committed_epoch) = committed_epoch { + if committed_epoch >= wait_epoch { + return Ok(()); + } + } else { + warn!( + table_id = options.table_id.table_id, + elapsed = ?start_time.elapsed(), + "table id not exist yet. wait for table creation" + ); } } } diff --git a/src/storage/src/memory.rs b/src/storage/src/memory.rs index 7416f54688a5f..9558811a2bdb0 100644 --- a/src/storage/src/memory.rs +++ b/src/storage/src/memory.rs @@ -735,7 +735,11 @@ impl StateStore for RangeKvStateStore { type Local = MemtableLocalStateStore; #[allow(clippy::unused_async)] - async fn try_wait_epoch(&self, _epoch: HummockReadEpoch) -> StorageResult<()> { + async fn try_wait_epoch( + &self, + _epoch: HummockReadEpoch, + _options: TryWaitEpochOptions, + ) -> StorageResult<()> { // memory backend doesn't need to wait for epoch, so this is a no-op. Ok(()) } diff --git a/src/storage/src/monitor/monitored_store.rs b/src/storage/src/monitor/monitored_store.rs index ec2785c354229..a7be71307ffec 100644 --- a/src/storage/src/monitor/monitored_store.rs +++ b/src/storage/src/monitor/monitored_store.rs @@ -298,9 +298,10 @@ impl StateStore for MonitoredStateStore { fn try_wait_epoch( &self, epoch: HummockReadEpoch, + options: TryWaitEpochOptions, ) -> impl Future> + Send + '_ { self.inner - .try_wait_epoch(epoch) + .try_wait_epoch(epoch, options) .verbose_instrument_await("store_wait_epoch") .inspect_err(|e| error!(error = %e.as_report(), "Failed in wait_epoch")) } diff --git a/src/storage/src/monitor/traced_store.rs b/src/storage/src/monitor/traced_store.rs index dac8712a924d7..8bd8013ba3810 100644 --- a/src/storage/src/monitor/traced_store.rs +++ b/src/storage/src/monitor/traced_store.rs @@ -242,10 +242,14 @@ impl LocalStateStore for TracedStateStore { impl StateStore for TracedStateStore { type Local = TracedStateStore; - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> StorageResult<()> { - let span = TraceSpan::new_try_wait_epoch_span(epoch); + async fn try_wait_epoch( + &self, + epoch: HummockReadEpoch, + options: TryWaitEpochOptions, + ) -> StorageResult<()> { + let span = TraceSpan::new_try_wait_epoch_span(epoch, options.clone().into()); - let res = self.inner.try_wait_epoch(epoch).await; + let res = self.inner.try_wait_epoch(epoch, options).await; span.may_send_result(OperationResult::TryWaitEpoch( res.as_ref().map(|o| *o).into(), )); diff --git a/src/storage/src/panic_store.rs b/src/storage/src/panic_store.rs index a7fb7c3643ed0..03b0471f90446 100644 --- a/src/storage/src/panic_store.rs +++ b/src/storage/src/panic_store.rs @@ -173,7 +173,11 @@ impl StateStore for PanicStateStore { type Local = Self; #[allow(clippy::unused_async)] - async fn try_wait_epoch(&self, _epoch: HummockReadEpoch) -> StorageResult<()> { + async fn try_wait_epoch( + &self, + _epoch: HummockReadEpoch, + _options: TryWaitEpochOptions, + ) -> StorageResult<()> { panic!("should not wait epoch from the panic state store!"); } diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index db21faa78c6cf..e958a93392dc9 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -35,7 +35,8 @@ use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection use risingwave_hummock_sdk::{HummockReadEpoch, SyncResult}; use risingwave_hummock_trace::{ TracedInitOptions, TracedNewLocalOptions, TracedOpConsistencyLevel, TracedPrefetchOptions, - TracedReadOptions, TracedSealCurrentEpochOptions, TracedWriteOptions, + TracedReadOptions, TracedSealCurrentEpochOptions, TracedTryWaitEpochOptions, + TracedWriteOptions, }; use risingwave_pb::hummock::PbVnodeWatermark; @@ -344,6 +345,34 @@ pub trait StateStoreWrite: StaticSendSync { pub trait SyncFuture = Future> + Send + 'static; +#[derive(Clone)] +pub struct TryWaitEpochOptions { + pub table_id: TableId, +} + +impl TryWaitEpochOptions { + #[cfg(any(test, feature = "test"))] + pub fn for_test(table_id: TableId) -> Self { + Self { table_id } + } +} + +impl From for TryWaitEpochOptions { + fn from(value: TracedTryWaitEpochOptions) -> Self { + Self { + table_id: value.table_id.into(), + } + } +} + +impl From for TracedTryWaitEpochOptions { + fn from(value: TryWaitEpochOptions) -> Self { + Self { + table_id: value.table_id.into(), + } + } +} + pub trait StateStore: StateStoreRead + StaticSendSync + Clone { type Local: LocalStateStore; @@ -352,6 +381,7 @@ pub trait StateStore: StateStoreRead + StaticSendSync + Clone { fn try_wait_epoch( &self, epoch: HummockReadEpoch, + options: TryWaitEpochOptions, ) -> impl Future> + Send + '_; fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture; diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index d9261d67e24bc..cfb0bfb3c8085 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -570,8 +570,9 @@ pub mod verify { fn try_wait_epoch( &self, epoch: HummockReadEpoch, + options: TryWaitEpochOptions, ) -> impl Future> + Send + '_ { - self.actual.try_wait_epoch(epoch) + self.actual.try_wait_epoch(epoch, options) } fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture { @@ -1154,7 +1155,11 @@ pub mod boxed_state_store { #[async_trait::async_trait] pub trait DynamicDispatchedStateStoreExt: StaticSendSync { - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> StorageResult<()>; + async fn try_wait_epoch( + &self, + epoch: HummockReadEpoch, + options: TryWaitEpochOptions, + ) -> StorageResult<()>; fn sync( &self, @@ -1167,8 +1172,12 @@ pub mod boxed_state_store { #[async_trait::async_trait] impl DynamicDispatchedStateStoreExt for S { - async fn try_wait_epoch(&self, epoch: HummockReadEpoch) -> StorageResult<()> { - self.try_wait_epoch(epoch).await + async fn try_wait_epoch( + &self, + epoch: HummockReadEpoch, + options: TryWaitEpochOptions, + ) -> StorageResult<()> { + self.try_wait_epoch(epoch, options).await } fn sync( @@ -1253,8 +1262,9 @@ pub mod boxed_state_store { fn try_wait_epoch( &self, epoch: HummockReadEpoch, + options: TryWaitEpochOptions, ) -> impl Future> + Send + '_ { - self.deref().try_wait_epoch(epoch) + self.deref().try_wait_epoch(epoch, options) } fn sync( diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 254e8e73095b1..d368542db4a06 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -49,6 +49,7 @@ use crate::row_serde::value_serde::{ValueRowSerde, ValueRowSerdeNew}; use crate::row_serde::{find_columns_by_ids, ColumnMapping}; use crate::store::{ PrefetchOptions, ReadLogOptions, ReadOptions, StateStoreIter, StateStoreIterExt, + TryWaitEpochOptions, }; use crate::table::merge_sort::merge_sort; use crate::table::{ChangeLogRow, KeyedRow, TableDistribution, TableIter}; @@ -365,7 +366,14 @@ impl StorageTableInner { wait_epoch, HummockReadEpoch::TimeTravel(_) | HummockReadEpoch::Committed(_) ); - self.store.try_wait_epoch(wait_epoch).await?; + self.store + .try_wait_epoch( + wait_epoch, + TryWaitEpochOptions { + table_id: self.table_id, + }, + ) + .await?; let serialized_pk = serialize_pk_with_vnode( &pk, &self.pk_serializer, @@ -872,7 +880,14 @@ impl StorageTableInnerIterInner { epoch: HummockReadEpoch, ) -> StorageResult { let raw_epoch = epoch.get_epoch(); - store.try_wait_epoch(epoch).await?; + store + .try_wait_epoch( + epoch, + TryWaitEpochOptions { + table_id: read_options.table_id, + }, + ) + .await?; let iter = store.iter(table_key_range, raw_epoch, read_options).await?; let iter = Self { iter, @@ -975,7 +990,12 @@ impl StorageTableInnerIterLogInner { end_epoch: u64, ) -> StorageResult { store - .try_wait_epoch(HummockReadEpoch::Committed(end_epoch)) + .try_wait_epoch( + HummockReadEpoch::Committed(end_epoch), + TryWaitEpochOptions { + table_id: read_options.table_id, + }, + ) .await?; let iter = store .iter_log((start_epoch, end_epoch), table_key_range, read_options) diff --git a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs index 440c7188d2fa1..3da42d67f2a0b 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs @@ -455,6 +455,7 @@ mod tests { use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_hummock_test::test_utils::prepare_hummock_test_env; use risingwave_storage::hummock::HummockStorage; + use risingwave_storage::store::TryWaitEpochOptions; use risingwave_storage::StateStore; use crate::common::log_store_impl::kv_log_store::reader::KvLogStoreReader; @@ -685,7 +686,10 @@ mod tests { .unwrap(); test_env .storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch2), + TryWaitEpochOptions::for_test(table.id.into()), + ) .await .unwrap(); @@ -901,7 +905,10 @@ mod tests { test_env.commit_epoch(epoch2).await; test_env .storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch2), + TryWaitEpochOptions::for_test(table.id.into()), + ) .await .unwrap(); @@ -1137,7 +1144,10 @@ mod tests { test_env.commit_epoch(epoch2).await; test_env .storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch2)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch2), + TryWaitEpochOptions::for_test(table.id.into()), + ) .await .unwrap(); @@ -1401,7 +1411,10 @@ mod tests { test_env .storage - .try_wait_epoch(HummockReadEpoch::Committed(epoch3)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch3), + TryWaitEpochOptions::for_test(table.id.into()), + ) .await .unwrap(); diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index 3f2cd83aca286..5efaaf5c3764d 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -32,6 +32,7 @@ use risingwave_connector::source::{ SplitMetaData, }; use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::store::TryWaitEpochOptions; use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; @@ -417,6 +418,7 @@ impl SourceBackfillExecutorInner { } let state_store = self.backfill_state_store.state_store.state_store().clone(); + let table_id = self.backfill_state_store.state_store.table_id().into(); static STATE_TABLE_INITIALIZED: Once = Once::new(); tokio::spawn(async move { // This is for self.backfill_finished() to be safe. @@ -424,7 +426,10 @@ impl SourceBackfillExecutorInner { let epoch = barrier.epoch.curr; tracing::info!("waiting for epoch: {}", epoch); state_store - .try_wait_epoch(HummockReadEpoch::Committed(epoch)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch), + TryWaitEpochOptions { table_id }, + ) .await .expect("failed to wait epoch"); STATE_TABLE_INITIALIZED.call_once(|| ()); diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index dd93ac85d1f1c..d4a02ce462441 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -20,6 +20,7 @@ use either::Either; use futures::TryStreamExt; use itertools::Itertools; use risingwave_common::array::ArrayRef; +use risingwave_common::catalog::TableId; use risingwave_common::metrics::{LabelGuardedIntCounter, GLOBAL_ERROR_METRICS}; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; @@ -32,6 +33,7 @@ use risingwave_connector::source::{ SplitMetaData, WaitCheckpointTask, }; use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::store::TryWaitEpochOptions; use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::{mpsc, oneshot}; @@ -104,6 +106,7 @@ impl SourceExecutor { let wait_checkpoint_worker = WaitCheckpointWorker { wait_checkpoint_rx, state_store: core.split_state_store.state_table.state_store().clone(), + table_id: core.split_state_store.state_table.table_id().into(), }; tokio::spawn(wait_checkpoint_worker.run()); Ok(Some(WaitCheckpointTaskBuilder { @@ -820,6 +823,7 @@ impl WaitCheckpointTaskBuilder { struct WaitCheckpointWorker { wait_checkpoint_rx: UnboundedReceiver<(Epoch, WaitCheckpointTask)>, state_store: S, + table_id: TableId, } impl WaitCheckpointWorker { @@ -832,7 +836,12 @@ impl WaitCheckpointWorker { tracing::debug!("start to wait epoch {}", epoch.0); let ret = self .state_store - .try_wait_epoch(HummockReadEpoch::Committed(epoch.0)) + .try_wait_epoch( + HummockReadEpoch::Committed(epoch.0), + TryWaitEpochOptions { + table_id: self.table_id, + }, + ) .await; match ret { From f5f78d4bf846ba8d1c8884a77805fe4a977aed20 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 20 Sep 2024 16:41:16 +0800 Subject: [PATCH 11/27] fix test --- .../src/hummock/event_handler/hummock_event_handler.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 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 82cdc14838946..61b75aecd4066 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -649,12 +649,12 @@ impl HummockEventHandler { let max_committed_epoch = new_pinned_version.visible_table_committed_epoch(); - // only notify local_version_manager when MCE change self.version_update_notifier_tx.send_if_modified(|state| { + assert_eq!(pinned_version.id(), state.id()); if state.id() == new_pinned_version.id() { return false; } - assert!(pinned_version.id() > state.id()); + assert!(new_pinned_version.id() > state.id()); *state = new_pinned_version.clone(); true }); From 902c065a7635f3c989651b4471c771d30c51c877 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 20 Sep 2024 18:20:24 +0800 Subject: [PATCH 12/27] fix test --- .../hummock_test/src/state_store_tests.rs | 41 ++++++++++++++----- src/storage/src/hummock/utils.rs | 38 +++++++---------- 2 files changed, 45 insertions(+), 34 deletions(-) diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 468bc91a13e3a..1018a72e7d274 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -31,6 +31,7 @@ use risingwave_hummock_sdk::{ HummockReadEpoch, HummockSstableObjectId, LocalSstableInfo, SyncResult, }; use risingwave_meta::hummock::test_utils::setup_compute_env; +use risingwave_meta::hummock::{CommitEpochInfo, NewTableFragmentInfo}; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::iterator::change_log::test_utils::{ apply_test_log_data, gen_test_data, @@ -1420,7 +1421,31 @@ async fn test_gc_watermark_and_clear_shared_buffer() { async fn test_replicated_local_hummock_storage() { const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; - let (hummock_storage, _meta_client) = with_hummock_storage_v2(Default::default()).await; + let (hummock_storage, meta_client) = with_hummock_storage_v2(Default::default()).await; + + let epoch0 = meta_client + .hummock_manager_ref() + .on_current_version(|version| version.visible_table_committed_epoch()) + .await; + + let epoch0 = epoch0.next_epoch(); + + meta_client + .hummock_manager_ref() + .commit_epoch(CommitEpochInfo { + sstables: vec![], + new_table_watermarks: Default::default(), + sst_to_context: Default::default(), + new_table_fragment_info: NewTableFragmentInfo::NewCompactionGroup { + table_ids: HashSet::from_iter([TEST_TABLE_ID]), + }, + change_log_delta: Default::default(), + committed_epoch: epoch0, + tables_to_commit: Default::default(), + is_visible_table_committed_epoch: true, + }) + .await + .unwrap(); let read_options = ReadOptions { table_id: TableId { @@ -1441,12 +1466,6 @@ async fn test_replicated_local_hummock_storage() { )) .await; - let epoch0 = local_hummock_storage - .read_version() - .read() - .committed() - .max_committed_epoch(); - let epoch1 = epoch0.next_epoch(); local_hummock_storage.init_for_test(epoch1).await.unwrap(); @@ -1496,13 +1515,13 @@ async fn test_replicated_local_hummock_storage() { [ Ok( ( - FullKey { UserKey { 233, TableKey { 000061616161 } }, epoch: 65536, epoch_with_gap: 65536, spill_offset: 0}, + FullKey { UserKey { 233, TableKey { 000061616161 } }, epoch: 131072, epoch_with_gap: 131072, spill_offset: 0}, b"1111", ), ), Ok( ( - FullKey { UserKey { 233, TableKey { 000062626262 } }, epoch: 65536, epoch_with_gap: 65536, spill_offset: 0}, + FullKey { UserKey { 233, TableKey { 000062626262 } }, epoch: 131072, epoch_with_gap: 131072, spill_offset: 0}, b"2222", ), ), @@ -1564,13 +1583,13 @@ async fn test_replicated_local_hummock_storage() { [ Ok( ( - FullKey { UserKey { 233, TableKey { 000063636363 } }, epoch: 131072, epoch_with_gap: 131072, spill_offset: 0}, + FullKey { UserKey { 233, TableKey { 000063636363 } }, epoch: 196608, epoch_with_gap: 196608, spill_offset: 0}, b"3333", ), ), Ok( ( - FullKey { UserKey { 233, TableKey { 000064646464 } }, epoch: 131072, epoch_with_gap: 131072, spill_offset: 0}, + FullKey { UserKey { 233, TableKey { 000064646464 } }, epoch: 196608, epoch_with_gap: 196608, spill_offset: 0}, b"4444", ), ), diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index b8761da23143e..235edc884ae5b 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -32,7 +32,6 @@ use risingwave_hummock_sdk::key::{ }; use risingwave_hummock_sdk::sstable_info::SstableInfo; use tokio::sync::oneshot::{channel, Receiver, Sender}; -use tracing::warn; use super::{HummockError, SstableStoreRef}; use crate::error::StorageResult; @@ -582,23 +581,19 @@ pub(crate) async fn wait_for_epoch( options: TryWaitEpochOptions, ) -> StorageResult<()> { let mut receiver = notifier.subscribe(); - { + let mut committed_epoch = { // avoid unnecessary check in the loop if the value does not change let committed_epoch = receiver .borrow_and_update() .version() .table_committed_epoch(options.table_id); - if let Some(committed_epoch) = committed_epoch { - if committed_epoch >= wait_epoch { - return Ok(()); - } - } else { - warn!( - table_id = options.table_id.table_id, - "table id not exist yet. wait for table creation" - ); + if let Some(committed_epoch) = committed_epoch + && committed_epoch >= wait_epoch + { + return Ok(()); } - } + committed_epoch + }; let start_time = Instant::now(); loop { match tokio::time::timeout(Duration::from_secs(30), receiver.changed()).await { @@ -614,6 +609,8 @@ pub(crate) async fn wait_for_epoch( // See #3845 for more details. tracing::warn!( epoch = wait_epoch, + ?committed_epoch, + table_id = options.table_id.table_id, elapsed = ?start_time.elapsed(), "wait_epoch timeout when waiting for version update", ); @@ -624,21 +621,16 @@ pub(crate) async fn wait_for_epoch( } Ok(Ok(_)) => { // TODO: should handle the corner case of drop table - let committed_epoch = receiver + let new_committed_epoch = receiver .borrow() .version() .table_committed_epoch(options.table_id); - if let Some(committed_epoch) = committed_epoch { - if committed_epoch >= wait_epoch { - return Ok(()); - } - } else { - warn!( - table_id = options.table_id.table_id, - elapsed = ?start_time.elapsed(), - "table id not exist yet. wait for table creation" - ); + if let Some(committed_epoch) = new_committed_epoch + && committed_epoch >= wait_epoch + { + return Ok(()); } + committed_epoch = new_committed_epoch; } } } From 84c8037d3b4e55596d6e5e9d26d370902aa783bd Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 13:20:09 +0800 Subject: [PATCH 13/27] fix comment --- src/frontend/src/binder/query.rs | 9 +++- src/frontend/src/binder/relation/mod.rs | 48 +++++++++++++++--- src/frontend/src/binder/set_expr.rs | 15 ++++-- src/frontend/src/expr/agg_call.rs | 4 ++ src/frontend/src/expr/mod.rs | 49 ++++++++++++++++++- .../src/scheduler/distributed/query.rs | 5 +- src/frontend/src/scheduler/snapshot.rs | 10 ++-- src/frontend/src/session/transaction.rs | 2 +- 8 files changed, 122 insertions(+), 20 deletions(-) diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index 7ad2091e6fb87..8622a569ed14c 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -16,7 +16,7 @@ use std::cell::RefCell; use std::collections::HashMap; use std::rc::Rc; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableId}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_sqlparser::ast::{ @@ -122,6 +122,13 @@ impl BoundQuery { extra_order_exprs: vec![], } } + + pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { + self.body.visit_all_scan_table_id(visitor); + self.extra_order_exprs + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } } impl RewriteExprsRecursive for BoundQuery { diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 2442717abbcb5..317426a8ee538 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -15,6 +15,7 @@ use std::collections::hash_map::Entry; use std::ops::Deref; +use either::Either; use itertools::{EitherOrBoth, Itertools}; use risingwave_common::bail; use risingwave_common::catalog::{Field, TableId, DEFAULT_SCHEMA_NAME}; @@ -138,9 +139,13 @@ impl Relation { } } - pub fn visit_all_scan_table(&self, visitor: &mut impl FnMut(TableId)) { + pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { match self { - Relation::Source(_) => {} + Relation::Source(source) => { + if let Some(table_id) = &source.catalog.associated_table_id { + visitor(*table_id); + } + } Relation::BaseTable(table) => { visitor(table.table_id); } @@ -149,13 +154,40 @@ impl Relation { subquery.query.body.visit_all_scan_table_id(visitor); } Relation::Join(join) | Relation::Apply(join) => { - join.left.visit_all_scan_table(visitor); - join.right.visit_all_scan_table(visitor); + join.left.visit_all_scan_table_id(visitor); + join.right.visit_all_scan_table_id(visitor); + join.cond.visit_all_scan_table_id(visitor); + } + Relation::WindowTableFunction(func) => { + func.input.visit_all_scan_table_id(visitor); + func.args + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + Relation::TableFunction { expr, .. } => { + expr.visit_all_scan_table_id(visitor); } - Relation::WindowTableFunction(_) => {} - Relation::TableFunction { .. } => {} - Relation::Watermark(_) => {} - Relation::Share(_) => {} + Relation::Watermark(watermark) => { + watermark.input.visit_all_scan_table_id(visitor); + watermark + .args + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + Relation::Share(share) => match &share.input { + BoundShareInput::Query(query) => match query { + Either::Left(query) => { + query.visit_all_scan_table_id(visitor); + } + Either::Right(recursive) => { + recursive.base.visit_all_scan_table_id(visitor); + recursive.recursive.visit_all_scan_table_id(visitor); + } + }, + BoundShareInput::ChangeLog(change_log) => { + change_log.visit_all_scan_table_id(visitor); + } + }, Relation::BackCteRef(_) => {} } } diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index 6d0707a52241b..791193c9d1464 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -148,13 +148,22 @@ impl BoundSetExpr { match self { BoundSetExpr::Select(select) => { if let Some(relation) = &select.from { - relation.visit_all_scan_table(visitor); + relation.visit_all_scan_table_id(visitor); } + select + .exprs() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); } BoundSetExpr::Query(query) => { - query.body.visit_all_scan_table_id(visitor); + query.visit_all_scan_table_id(visitor); + } + BoundSetExpr::Values(values) => { + values + .rows + .iter() + .flatten() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); } - BoundSetExpr::Values(_) => {} BoundSetExpr::SetOperation { left, right, .. } => { left.visit_all_scan_table_id(visitor); right.visit_all_scan_table_id(visitor); diff --git a/src/frontend/src/expr/agg_call.rs b/src/frontend/src/expr/agg_call.rs index 452d37652d341..33dcc3b5d3b7a 100644 --- a/src/frontend/src/expr/agg_call.rs +++ b/src/frontend/src/expr/agg_call.rs @@ -124,6 +124,10 @@ impl AggCall { pub fn filter_mut(&mut self) -> &mut Condition { &mut self.filter } + + pub fn iter_expr(&self) -> impl Iterator + '_ { + self.args.iter().chain(self.filter.conjunctions.iter()) + } } impl Expr for AggCall { diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index c7acdfa5c4a3c..e7ec8e2531220 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -404,6 +404,53 @@ impl ExprImpl { pub fn fold_const(&self) -> RwResult { self.try_fold_const().expect("expression is not constant") } + + pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { + match self { + ExprImpl::CorrelatedInputRef(_) => {} + ExprImpl::InputRef(_) => {} + ExprImpl::Literal(_) => {} + ExprImpl::FunctionCall(call) => { + call.inputs + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::FunctionCallWithLambda(call) => { + call.inputs_with_lambda_arg() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::AggCall(agg) => { + agg.iter_expr() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::Subquery(query) => { + query.query.body.visit_all_scan_table_id(visitor); + query + .query + .extra_order_exprs + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::TableFunction(func) => { + func.args + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::WindowFunction(func) => { + func.args + .iter() + .chain(func.partition_by.iter()) + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::UserDefinedFunction(func) => { + func.args + .iter() + .for_each(|expr| expr.visit_all_scan_table_id(visitor)); + } + ExprImpl::Parameter(_) => {} + ExprImpl::Now(_) => {} + } + } } /// Implement helper functions which recursively checks whether an variant is included in the @@ -1168,7 +1215,7 @@ macro_rules! assert_eq_input_ref { #[cfg(test)] pub(crate) use assert_eq_input_ref; use risingwave_common::bail; -use risingwave_common::catalog::Schema; +use risingwave_common::catalog::{Schema, TableId}; use risingwave_common::row::OwnedRow; use self::function_call::CastError; diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 61996a4d50e29..b0cee2beb5070 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -519,7 +519,10 @@ pub(crate) mod tests { .start( ExecutionContext::new(SessionImpl::mock().into(), None).into(), worker_node_selector, - QuerySnapshot::new(ReadSnapshot::BarrierRead, HashSet::from_iter([0.into()])), + QuerySnapshot::new( + ReadSnapshot::ReadUncommitted, + HashSet::from_iter([0.into()]) + ), compute_client_pool, catalog_reader, query_execution_info, diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index a8b78826f6cc3..16a518edd8b19 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -38,7 +38,7 @@ pub enum ReadSnapshot { snapshot: PinnedSnapshotRef, }, - BarrierRead, + ReadUncommitted, /// Other arbitrary epoch, e.g. user specified. /// Availability and consistency of underlying data should be guaranteed accordingly. @@ -67,7 +67,7 @@ impl QuerySnapshot { snapshot.batch_query_epoch(&self.scan_tables)?.0, )), }, - ReadSnapshot::BarrierRead => BatchQueryEpoch { + ReadSnapshot::ReadUncommitted => BatchQueryEpoch { epoch: Some(batch_query_epoch::Epoch::Current(u64::MAX)), }, ReadSnapshot::Other(e) => BatchQueryEpoch { @@ -82,14 +82,14 @@ impl QuerySnapshot { snapshot.batch_query_epoch(&self.scan_tables)? } ReadSnapshot::Other(epoch) => *epoch, - ReadSnapshot::BarrierRead => Epoch::now(), + ReadSnapshot::ReadUncommitted => Epoch::now(), }; Ok(InlineNowProcTime::new(epoch)) } /// Returns true if this snapshot is a barrier read. pub fn support_barrier_read(&self) -> bool { - matches!(&self.snapshot, ReadSnapshot::BarrierRead) + matches!(&self.snapshot, ReadSnapshot::ReadUncommitted) } } @@ -124,7 +124,7 @@ impl PinnedSnapshot { .try_fold(None, |prev_min_committed_epoch, committed_epoch| { committed_epoch.map(|committed_epoch| { if let Some(prev_min_committed_epoch) = prev_min_committed_epoch - && prev_min_committed_epoch >= committed_epoch + && prev_min_committed_epoch <= committed_epoch { Some(prev_min_committed_epoch) } else { diff --git a/src/frontend/src/session/transaction.rs b/src/frontend/src/session/transaction.rs index 871421ad31c7d..05fde4f91fd6f 100644 --- a/src/frontend/src/session/transaction.rs +++ b/src/frontend/src/session/transaction.rs @@ -229,7 +229,7 @@ impl SessionImpl { if let Some(query_epoch) = query_epoch { ReadSnapshot::Other(query_epoch) } else if self.is_barrier_read() { - ReadSnapshot::BarrierRead + ReadSnapshot::ReadUncommitted } else { // Acquire hummock snapshot for execution. let hummock_snapshot_manager = self.env().hummock_snapshot_manager(); From c88b9df1ace9b9de6eb055694d4d48e725d9e238 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 15:21:30 +0800 Subject: [PATCH 14/27] simplify now --- src/frontend/planner_test/src/lib.rs | 10 ++-- .../tests/testdata/input/share.yaml | 1 + .../tests/testdata/output/share.yaml | 1 + src/frontend/src/binder/query.rs | 9 +--- src/frontend/src/binder/relation/mod.rs | 54 ------------------- src/frontend/src/binder/set_expr.rs | 29 +--------- src/frontend/src/binder/statement.rs | 16 +----- src/frontend/src/expr/agg_call.rs | 4 -- src/frontend/src/expr/mod.rs | 49 +---------------- src/frontend/src/handler/declare_cursor.rs | 18 +++++-- src/frontend/src/handler/query.rs | 39 ++++++++------ .../src/optimizer/logical_optimization.rs | 37 ++++++------- src/frontend/src/optimizer/mod.rs | 14 ++--- src/frontend/src/scheduler/snapshot.rs | 2 +- src/frontend/src/session/cursor_manager.rs | 5 +- src/tests/sqlsmith/tests/frontend/mod.rs | 3 +- 16 files changed, 75 insertions(+), 216 deletions(-) diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 3239bdfa32fa7..675b99ad0e145 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -603,8 +603,6 @@ impl TestCase { } }; - let scan_tables = bound.scan_tables(); - let mut planner = Planner::new(context.clone()); let plan_root = match planner.plan(bound) { @@ -628,7 +626,7 @@ impl TestCase { { let mut plan_root = plan_root.clone(); let optimized_logical_plan_for_batch = - match plan_root.gen_optimized_logical_plan_for_batch(&scan_tables) { + match plan_root.gen_optimized_logical_plan_for_batch() { Ok(optimized_logical_plan_for_batch) => optimized_logical_plan_for_batch, Err(err) => { ret.optimizer_error = Some(err.to_report_string_pretty()); @@ -677,7 +675,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { + let batch_plan = match plan_root.gen_batch_plan() { Ok(_batch_plan) => match plan_root.gen_batch_distributed_plan() { Ok(batch_plan) => batch_plan, Err(err) => { @@ -710,7 +708,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { + let batch_plan = match plan_root.gen_batch_plan() { Ok(_batch_plan) => match plan_root.gen_batch_local_plan() { Ok(batch_plan) => batch_plan, Err(err) => { @@ -738,7 +736,7 @@ impl TestCase { || self.expected_outputs.contains(&TestType::BatchError) { let mut plan_root = plan_root.clone(); - let batch_plan = match plan_root.gen_batch_plan(&scan_tables) { + let batch_plan = match plan_root.gen_batch_plan() { Ok(_batch_plan) => match plan_root.gen_batch_distributed_plan() { Ok(batch_plan) => batch_plan, Err(err) => { diff --git a/src/frontend/planner_test/tests/testdata/input/share.yaml b/src/frontend/planner_test/tests/testdata/input/share.yaml index efadb03e2d439..91217f52d3851 100644 --- a/src/frontend/planner_test/tests/testdata/input/share.yaml +++ b/src/frontend/planner_test/tests/testdata/input/share.yaml @@ -14,6 +14,7 @@ nexmark.split.num = '4', nexmark.min.event.gap.in.ns = '1000' ); + create table table_for_fixed_now_timestamp; expected_outputs: [] - id: self_join before: diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 2cf3aee9fe043..b9a7fe05b6681 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -15,6 +15,7 @@ nexmark.split.num = '4', nexmark.min.event.gap.in.ns = '1000' ); + create table table_for_fixed_now_timestamp; - id: self_join before: - create_sources diff --git a/src/frontend/src/binder/query.rs b/src/frontend/src/binder/query.rs index 8622a569ed14c..7ad2091e6fb87 100644 --- a/src/frontend/src/binder/query.rs +++ b/src/frontend/src/binder/query.rs @@ -16,7 +16,7 @@ use std::cell::RefCell; use std::collections::HashMap; use std::rc::Rc; -use risingwave_common::catalog::{Schema, TableId}; +use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_sqlparser::ast::{ @@ -122,13 +122,6 @@ impl BoundQuery { extra_order_exprs: vec![], } } - - pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { - self.body.visit_all_scan_table_id(visitor); - self.extra_order_exprs - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } } impl RewriteExprsRecursive for BoundQuery { diff --git a/src/frontend/src/binder/relation/mod.rs b/src/frontend/src/binder/relation/mod.rs index 317426a8ee538..2cdf3ea07db4e 100644 --- a/src/frontend/src/binder/relation/mod.rs +++ b/src/frontend/src/binder/relation/mod.rs @@ -15,7 +15,6 @@ use std::collections::hash_map::Entry; use std::ops::Deref; -use either::Either; use itertools::{EitherOrBoth, Itertools}; use risingwave_common::bail; use risingwave_common::catalog::{Field, TableId, DEFAULT_SCHEMA_NAME}; @@ -138,59 +137,6 @@ impl Relation { _ => vec![], } } - - pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { - match self { - Relation::Source(source) => { - if let Some(table_id) = &source.catalog.associated_table_id { - visitor(*table_id); - } - } - Relation::BaseTable(table) => { - visitor(table.table_id); - } - Relation::SystemTable(_) => {} - Relation::Subquery(subquery) => { - subquery.query.body.visit_all_scan_table_id(visitor); - } - Relation::Join(join) | Relation::Apply(join) => { - join.left.visit_all_scan_table_id(visitor); - join.right.visit_all_scan_table_id(visitor); - join.cond.visit_all_scan_table_id(visitor); - } - Relation::WindowTableFunction(func) => { - func.input.visit_all_scan_table_id(visitor); - func.args - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - Relation::TableFunction { expr, .. } => { - expr.visit_all_scan_table_id(visitor); - } - Relation::Watermark(watermark) => { - watermark.input.visit_all_scan_table_id(visitor); - watermark - .args - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - Relation::Share(share) => match &share.input { - BoundShareInput::Query(query) => match query { - Either::Left(query) => { - query.visit_all_scan_table_id(visitor); - } - Either::Right(recursive) => { - recursive.base.visit_all_scan_table_id(visitor); - recursive.recursive.visit_all_scan_table_id(visitor); - } - }, - BoundShareInput::ChangeLog(change_log) => { - change_log.visit_all_scan_table_id(visitor); - } - }, - Relation::BackCteRef(_) => {} - } - } } #[derive(Debug)] diff --git a/src/frontend/src/binder/set_expr.rs b/src/frontend/src/binder/set_expr.rs index 791193c9d1464..68af5845bf7a4 100644 --- a/src/frontend/src/binder/set_expr.rs +++ b/src/frontend/src/binder/set_expr.rs @@ -16,7 +16,7 @@ use std::borrow::Cow; use std::collections::HashMap; use risingwave_common::bail_not_implemented; -use risingwave_common::catalog::{Schema, TableId}; +use risingwave_common::catalog::Schema; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Corresponding, SetExpr, SetOperator}; @@ -143,33 +143,6 @@ impl BoundSetExpr { } } } - - pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { - match self { - BoundSetExpr::Select(select) => { - if let Some(relation) = &select.from { - relation.visit_all_scan_table_id(visitor); - } - select - .exprs() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - BoundSetExpr::Query(query) => { - query.visit_all_scan_table_id(visitor); - } - BoundSetExpr::Values(values) => { - values - .rows - .iter() - .flatten() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - BoundSetExpr::SetOperation { left, right, .. } => { - left.visit_all_scan_table_id(visitor); - right.visit_all_scan_table_id(visitor); - } - } - } } impl Binder { diff --git a/src/frontend/src/binder/statement.rs b/src/frontend/src/binder/statement.rs index 102f9625bd326..b73fab90aed9a 100644 --- a/src/frontend/src/binder/statement.rs +++ b/src/frontend/src/binder/statement.rs @@ -12,10 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; - use risingwave_common::bail_not_implemented; -use risingwave_common::catalog::{Field, TableId}; +use risingwave_common::catalog::Field; use risingwave_sqlparser::ast::Statement; use super::delete::BoundDelete; @@ -59,18 +57,6 @@ impl BoundStatement { BoundStatement::CreateView(_) => vec![], } } - - pub fn scan_tables(&self) -> HashSet { - if let BoundStatement::Query(query) = self { - let mut tables = HashSet::new(); - query.body.visit_all_scan_table_id(&mut |table_id| { - tables.insert(table_id); - }); - tables - } else { - HashSet::new() - } - } } impl Binder { diff --git a/src/frontend/src/expr/agg_call.rs b/src/frontend/src/expr/agg_call.rs index 33dcc3b5d3b7a..452d37652d341 100644 --- a/src/frontend/src/expr/agg_call.rs +++ b/src/frontend/src/expr/agg_call.rs @@ -124,10 +124,6 @@ impl AggCall { pub fn filter_mut(&mut self) -> &mut Condition { &mut self.filter } - - pub fn iter_expr(&self) -> impl Iterator + '_ { - self.args.iter().chain(self.filter.conjunctions.iter()) - } } impl Expr for AggCall { diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index e7ec8e2531220..c7acdfa5c4a3c 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -404,53 +404,6 @@ impl ExprImpl { pub fn fold_const(&self) -> RwResult { self.try_fold_const().expect("expression is not constant") } - - pub fn visit_all_scan_table_id(&self, visitor: &mut impl FnMut(TableId)) { - match self { - ExprImpl::CorrelatedInputRef(_) => {} - ExprImpl::InputRef(_) => {} - ExprImpl::Literal(_) => {} - ExprImpl::FunctionCall(call) => { - call.inputs - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::FunctionCallWithLambda(call) => { - call.inputs_with_lambda_arg() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::AggCall(agg) => { - agg.iter_expr() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::Subquery(query) => { - query.query.body.visit_all_scan_table_id(visitor); - query - .query - .extra_order_exprs - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::TableFunction(func) => { - func.args - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::WindowFunction(func) => { - func.args - .iter() - .chain(func.partition_by.iter()) - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::UserDefinedFunction(func) => { - func.args - .iter() - .for_each(|expr| expr.visit_all_scan_table_id(visitor)); - } - ExprImpl::Parameter(_) => {} - ExprImpl::Now(_) => {} - } - } } /// Implement helper functions which recursively checks whether an variant is included in the @@ -1215,7 +1168,7 @@ macro_rules! assert_eq_input_ref { #[cfg(test)] pub(crate) use assert_eq_input_ref; use risingwave_common::bail; -use risingwave_common::catalog::{Schema, TableId}; +use risingwave_common::catalog::Schema; use risingwave_common::row::OwnedRow; use self::function_call::CastError; diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index f4ae44f8c2b3f..3ea8fb60b3ef0 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -157,7 +157,7 @@ pub async fn create_chunk_stream_for_cursor( plan_fragmenter, query_mode, schema, - scan_tables, + dependent_relations, .. } = plan_fragmenter_result; @@ -170,10 +170,22 @@ pub async fn create_chunk_stream_for_cursor( match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => CursorDataChunkStream::LocalDataChunk(Some( - local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + local_execute( + session.clone(), + query, + can_timeout_cancel, + dependent_relations, + ) + .await?, )), QueryMode::Distributed => CursorDataChunkStream::DistributedDataChunk(Some( - distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + distribute_execute( + session.clone(), + query, + can_timeout_cancel, + dependent_relations, + ) + .await?, )), }, schema.fields.clone(), diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 24d4b8ba61209..510924d59d6bb 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -205,8 +205,7 @@ pub struct BatchQueryPlanResult { // Note that these relations are only resolved in the binding phase, and it may only be a // subset of the final one. i.e. the final one may contain more implicit dependencies on // indices. - pub(crate) dependent_relations: Vec, - pub(crate) scan_tables: HashSet, + pub(crate) dependent_relations: HashSet, } fn gen_batch_query_plan( @@ -224,11 +223,9 @@ fn gen_batch_query_plan( let mut planner = Planner::new(context); - let scan_tables = bound.scan_tables(); - let mut logical = planner.plan(bound)?; let schema = logical.schema(); - let batch_plan = logical.gen_batch_plan(&scan_tables)?; + let batch_plan = logical.gen_batch_plan()?; let dependent_relations = RelationCollectorVisitor::collect_with(dependent_relations, batch_plan.clone()); @@ -262,8 +259,7 @@ fn gen_batch_query_plan( query_mode, schema, stmt_type, - dependent_relations: dependent_relations.into_iter().collect_vec(), - scan_tables, + dependent_relations: dependent_relations.into_iter().collect(), }) } @@ -315,8 +311,7 @@ pub struct BatchPlanFragmenterResult { pub(crate) query_mode: QueryMode, pub(crate) schema: Schema, pub(crate) stmt_type: StatementType, - pub(crate) _dependent_relations: Vec, - pub(crate) scan_tables: HashSet, + pub(crate) dependent_relations: HashSet, } pub fn gen_batch_plan_fragmenter( @@ -329,7 +324,6 @@ pub fn gen_batch_plan_fragmenter( schema, stmt_type, dependent_relations, - scan_tables, } = plan_result; tracing::trace!( @@ -353,8 +347,7 @@ pub fn gen_batch_plan_fragmenter( query_mode, schema, stmt_type, - _dependent_relations: dependent_relations, - scan_tables, + dependent_relations, }) } @@ -368,7 +361,7 @@ pub async fn create_stream( query_mode, schema, stmt_type, - scan_tables, + dependent_relations, .. } = plan_fragmenter_result; @@ -400,7 +393,13 @@ pub async fn create_stream( let row_stream = match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new( - local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + local_execute( + session.clone(), + query, + can_timeout_cancel, + dependent_relations, + ) + .await?, column_types, formats, session.clone(), @@ -408,7 +407,13 @@ pub async fn create_stream( // Local mode do not support cancel tasks. QueryMode::Distributed => { PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new( - distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + distribute_execute( + session.clone(), + query, + can_timeout_cancel, + dependent_relations, + ) + .await?, column_types, formats, session.clone(), @@ -511,7 +516,7 @@ pub async fn local_execute( session: Arc, query: Query, can_timeout_cancel: bool, - scan_tables: HashSet, + dependent_relations: HashSet, ) -> Result { let timeout = if cfg!(madsim) { None @@ -523,7 +528,7 @@ pub async fn local_execute( let front_env = session.env(); // TODO: if there's no table scan, we don't need to acquire snapshot. - let snapshot = session.pinned_snapshot(scan_tables); + let snapshot = session.pinned_snapshot(dependent_relations); // TODO: Passing sql here let execution = diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 35b92b98c7d28..ab68b9cbb361d 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -12,15 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; - use itertools::Itertools; use risingwave_common::bail; use super::plan_node::RewriteExprsRecursive; use super::plan_visitor::has_logical_max_one_row; use crate::error::Result; -use crate::expr::NowProcTimeFinder; +use crate::expr::{InlineNowProcTime, NowProcTimeFinder}; use crate::optimizer::heuristic_optimizer::{ApplyOrder, HeuristicOptimizer}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, VisitExprsRecursive, @@ -110,7 +108,7 @@ impl OptimizationStage { use std::sync::LazyLock; -use crate::catalog::TableId; +use risingwave_common::util::epoch::Epoch; pub struct LogicalOptimizer {} @@ -536,11 +534,7 @@ impl LogicalOptimizer { plan } - pub fn inline_now_proc_time( - plan: PlanRef, - ctx: &OptimizerContextRef, - scan_tables: &HashSet, - ) -> Result { + pub fn inline_now_proc_time(plan: PlanRef, ctx: &OptimizerContextRef) -> Result { // If now() and proctime() are not found, bail out. let mut v = NowProcTimeFinder::default(); plan.visit_exprs_recursive(&mut v); @@ -548,10 +542,20 @@ impl LogicalOptimizer { return Ok(plan); } - let mut v = ctx - .session_ctx() - .pinned_snapshot(scan_tables.clone()) - .inline_now_proc_time()?; + // use the maximun committed_epoch over all tables as the now value, or current timestamp when there is no table + let mut v = InlineNowProcTime::new( + ctx.session_ctx() + .env() + .hummock_snapshot_manager() + .acquire() + .version() + .state_table_info + .info() + .values() + .map(|info| Epoch(info.committed_epoch)) + .max() + .unwrap_or_else(Epoch::now), + ); let plan = plan.rewrite_exprs_recursive(&mut v); @@ -690,10 +694,7 @@ impl LogicalOptimizer { Ok(plan) } - pub fn gen_optimized_logical_plan_for_batch( - mut plan: PlanRef, - scan_tables: &HashSet, - ) -> Result { + pub fn gen_optimized_logical_plan_for_batch(mut plan: PlanRef) -> Result { let ctx = plan.ctx(); let explain_trace = ctx.is_explain_trace(); @@ -703,7 +704,7 @@ impl LogicalOptimizer { } // Inline `NOW()` and `PROCTIME()`, only for batch queries. - plan = Self::inline_now_proc_time(plan, &ctx, scan_tables)?; + plan = Self::inline_now_proc_time(plan, &ctx)?; // Convert the dag back to the tree, because we don't support DAG plan for batch. plan = plan.optimize_by_rules(&DAG_TO_TREE); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index fc6786ff6057c..de5c3deaf0d6b 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -40,7 +40,7 @@ mod plan_expr_visitor; mod rule; use std::assert_matches::assert_matches; -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use fixedbitset::FixedBitSet; use itertools::Itertools as _; @@ -279,26 +279,22 @@ impl PlanRoot { } /// Apply logical optimization to the plan for batch. - pub fn gen_optimized_logical_plan_for_batch( - &mut self, - scan_tables: &HashSet, - ) -> Result { + pub fn gen_optimized_logical_plan_for_batch(&mut self) -> Result { assert_eq!(self.phase, PlanPhase::Logical); assert_eq!(self.plan.convention(), Convention::Logical); - self.plan = - LogicalOptimizer::gen_optimized_logical_plan_for_batch(self.plan.clone(), scan_tables)?; + self.plan = LogicalOptimizer::gen_optimized_logical_plan_for_batch(self.plan.clone())?; self.phase = PlanPhase::OptimizedLogicalForBatch; assert_eq!(self.plan.convention(), Convention::Logical); Ok(self.plan.clone()) } /// Optimize and generate a singleton batch physical plan without exchange nodes. - pub fn gen_batch_plan(&mut self, scan_tables: &HashSet) -> Result { + pub fn gen_batch_plan(&mut self) -> Result { assert_eq!(self.plan.convention(), Convention::Logical); let mut plan = match self.phase { PlanPhase::Logical => { // Logical optimization - self.gen_optimized_logical_plan_for_batch(scan_tables)? + self.gen_optimized_logical_plan_for_batch()? } PlanPhase::OptimizedLogicalForBatch => self.plan.clone(), PlanPhase::Batch | PlanPhase::OptimizedLogicalForStream | PlanPhase::Stream => { diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 16a518edd8b19..1e8506ad949ab 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -132,7 +132,7 @@ impl PinnedSnapshot { } }) })? - .unwrap_or(Epoch(self.value.max_committed_epoch)); + .unwrap_or_else(Epoch::now); Ok(epoch) } diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index f26f9354d455b..9f14aae5a1931 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -14,7 +14,7 @@ use core::mem; use core::time::Duration; -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, VecDeque}; use std::rc::Rc; use std::sync::Arc; use std::time::Instant; @@ -650,8 +650,7 @@ impl SubscriptionCursor { query_mode, schema, stmt_type: StatementType::SELECT, - dependent_relations: table_catalog.dependent_relations.clone(), - scan_tables: HashSet::from_iter([table_catalog.id]), + dependent_relations: table_catalog.dependent_relations.iter().cloned().collect(), }) } diff --git a/src/tests/sqlsmith/tests/frontend/mod.rs b/src/tests/sqlsmith/tests/frontend/mod.rs index bf884d8a83955..6eedb4e41876b 100644 --- a/src/tests/sqlsmith/tests/frontend/mod.rs +++ b/src/tests/sqlsmith/tests/frontend/mod.rs @@ -185,7 +185,6 @@ fn run_batch_query( let bound = binder .bind(stmt) .map_err(|e| Failed::from(format!("Failed to bind:\nReason:\n{}", e.as_report())))?; - let scan_tables = bound.scan_tables(); let mut planner = Planner::new(context); let mut plan_root = planner.plan(bound).map_err(|e| { Failed::from(format!( @@ -193,7 +192,7 @@ fn run_batch_query( e.as_report() )) })?; - plan_root.gen_batch_plan(&scan_tables).map_err(|e| { + plan_root.gen_batch_plan().map_err(|e| { Failed::from(format!( "Failed to generate batch plan:\nReason:\n{}", e.as_report() From 40f8ac68bc5f88b4cdaed3f9510ea34d0a100226 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 15:57:23 +0800 Subject: [PATCH 15/27] refine --- src/frontend/src/handler/create_table.rs | 8 +++--- .../src/optimizer/logical_optimization.rs | 27 +++++-------------- src/frontend/src/session.rs | 18 +++++++++++++ 3 files changed, 28 insertions(+), 25 deletions(-) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 0a27f2b7a4e25..d91785bd76dbd 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -27,7 +27,6 @@ use risingwave_common::catalog::{ INITIAL_TABLE_VERSION_ID, }; use risingwave_common::license::Feature; -use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; @@ -57,7 +56,7 @@ use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, ColumnId, DatabaseId, SchemaId}; use crate::error::{ErrorCode, Result, RwError}; -use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::handler::create_source::{ bind_columns_from_source, bind_connector_props, bind_create_source_or_table_with_connector, bind_source_watermark, handle_addition_columns, UPSTREAM_SOURCE_KEY, @@ -330,8 +329,9 @@ pub fn bind_sql_column_constraints( // so the rewritten expression should almost always be pure and we directly call `fold_const` // here. Actually we do not require purity of the expression here since we're only to get a // snapshot value. - let rewritten_expr_impl = - InlineNowProcTime::new(Epoch::now()).rewrite_expr(expr_impl.clone()); + let rewritten_expr_impl = session + .inline_now_proc_time() + .rewrite_expr(expr_impl.clone()); if let Some(snapshot_value) = rewritten_expr_impl.try_fold_const() { let snapshot_value = snapshot_value?; diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index ab68b9cbb361d..cdc05610910fe 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -18,7 +18,7 @@ use risingwave_common::bail; use super::plan_node::RewriteExprsRecursive; use super::plan_visitor::has_logical_max_one_row; use crate::error::Result; -use crate::expr::{InlineNowProcTime, NowProcTimeFinder}; +use crate::expr::NowProcTimeFinder; use crate::optimizer::heuristic_optimizer::{ApplyOrder, HeuristicOptimizer}; use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, VisitExprsRecursive, @@ -108,8 +108,6 @@ impl OptimizationStage { use std::sync::LazyLock; -use risingwave_common::util::epoch::Epoch; - pub struct LogicalOptimizer {} static DAG_TO_TREE: LazyLock = LazyLock::new(|| { @@ -534,28 +532,15 @@ impl LogicalOptimizer { plan } - pub fn inline_now_proc_time(plan: PlanRef, ctx: &OptimizerContextRef) -> Result { + pub fn inline_now_proc_time(plan: PlanRef, ctx: &OptimizerContextRef) -> PlanRef { // If now() and proctime() are not found, bail out. let mut v = NowProcTimeFinder::default(); plan.visit_exprs_recursive(&mut v); if !v.has() { - return Ok(plan); + return plan; } - // use the maximun committed_epoch over all tables as the now value, or current timestamp when there is no table - let mut v = InlineNowProcTime::new( - ctx.session_ctx() - .env() - .hummock_snapshot_manager() - .acquire() - .version() - .state_table_info - .info() - .values() - .map(|info| Epoch(info.committed_epoch)) - .max() - .unwrap_or_else(Epoch::now), - ); + let mut v = ctx.session_ctx().inline_now_proc_time(); let plan = plan.rewrite_exprs_recursive(&mut v); @@ -563,7 +548,7 @@ impl LogicalOptimizer { ctx.trace("Inline Now and ProcTime:"); ctx.trace(plan.explain_to_string()); } - Ok(plan) + plan } pub fn gen_optimized_logical_plan_for_stream(mut plan: PlanRef) -> Result { @@ -704,7 +689,7 @@ impl LogicalOptimizer { } // Inline `NOW()` and `PROCTIME()`, only for batch queries. - plan = Self::inline_now_proc_time(plan, &ctx)?; + plan = Self::inline_now_proc_time(plan, &ctx); // Convert the dag back to the tree, because we don't support DAG plan for batch. plan = plan.optimize_by_rules(&DAG_TO_TREE); diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index e78a339bc1499..5e5c6c93211f6 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -60,6 +60,7 @@ use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::types::DataType; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::cluster_limit::ActorCountPerParallelism; +use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_common::util::{cluster_limit, resource_util}; @@ -96,6 +97,7 @@ use crate::catalog::{ check_schema_writable, CatalogError, DatabaseId, OwnedByUserCatalog, SchemaId, TableId, }; use crate::error::{ErrorCode, Result, RwError}; +use crate::expr::InlineNowProcTime; use crate::handler::describe::infer_describe; use crate::handler::extended_handle::{ handle_bind, handle_execute, handle_parse, Portal, PrepareStatement, @@ -835,6 +837,22 @@ impl SessionImpl { self.id } + pub fn inline_now_proc_time(&self) -> InlineNowProcTime { + // use the maximum committed_epoch over all tables as the now value, or current timestamp when there is no table + InlineNowProcTime::new( + self.env() + .hummock_snapshot_manager() + .acquire() + .version() + .state_table_info + .info() + .values() + .map(|info| Epoch(info.committed_epoch)) + .max() + .unwrap_or_else(Epoch::now), + ) + } + pub fn running_sql(&self) -> Option> { self.exec_context .lock() From 4e897e9d0c9da772dc89928960afe23ec2971da1 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 16:13:33 +0800 Subject: [PATCH 16/27] address comment --- src/frontend/src/scheduler/snapshot.rs | 21 +++++++------------ src/frontend/src/session.rs | 6 ++---- .../hummock_sdk/src/frontend_version.rs | 12 +++-------- src/storage/hummock_sdk/src/version.rs | 11 +++++++++- 4 files changed, 22 insertions(+), 28 deletions(-) diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 732b33c137f45..e728b74b30b65 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -26,7 +26,6 @@ use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::hummock::{HummockVersionDeltas, StateTableInfoDelta}; use tokio::sync::watch; -use crate::expr::InlineNowProcTime; use crate::meta_client::FrontendMetaClient; use crate::scheduler::SchedulerError; @@ -76,17 +75,6 @@ impl QuerySnapshot { }) } - pub fn inline_now_proc_time(&self) -> Result { - let epoch = match &self.snapshot { - ReadSnapshot::FrontendPinned { snapshot, .. } => { - snapshot.batch_query_epoch(&self.scan_tables)? - } - ReadSnapshot::Other(epoch) => *epoch, - ReadSnapshot::ReadUncommitted => Epoch::now(), - }; - Ok(InlineNowProcTime::new(epoch)) - } - /// Returns true if this snapshot is a barrier read. pub fn support_barrier_read(&self) -> bool { matches!(&self.snapshot, ReadSnapshot::ReadUncommitted) @@ -132,7 +120,13 @@ impl PinnedSnapshot { } }) })? - .unwrap_or_else(Epoch::now); + .unwrap_or_else(|| { + self.value + .state_table_info + .max_table_committed_epoch() + .map(Epoch) + .unwrap_or_else(Epoch::now) + }); Ok(epoch) } @@ -145,7 +139,6 @@ impl PinnedSnapshot { fn invalid_snapshot() -> FrontendHummockVersion { FrontendHummockVersion { id: INVALID_VERSION_ID, - max_committed_epoch: 0, state_table_info: HummockVersionStateTableInfo::from_protobuf(&HashMap::new()), table_change_log: Default::default(), } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 7f168b70e592a..552fb08eb60f0 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -847,10 +847,8 @@ impl SessionImpl { .acquire() .version() .state_table_info - .info() - .values() - .map(|info| Epoch(info.committed_epoch)) - .max() + .max_table_committed_epoch() + .map(Epoch) .unwrap_or_else(Epoch::now), ) } diff --git a/src/storage/hummock_sdk/src/frontend_version.rs b/src/storage/hummock_sdk/src/frontend_version.rs index 2086832d07714..254e151f7ec06 100644 --- a/src/storage/hummock_sdk/src/frontend_version.rs +++ b/src/storage/hummock_sdk/src/frontend_version.rs @@ -15,6 +15,7 @@ use std::collections::{HashMap, HashSet}; use risingwave_common::catalog::TableId; +use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_pb::hummock::hummock_version_delta::PbChangeLogDelta; use risingwave_pb::hummock::{ PbEpochNewChangeLog, PbHummockVersion, PbHummockVersionDelta, PbTableChangeLog, @@ -28,7 +29,6 @@ use crate::{HummockVersionId, INVALID_VERSION_ID}; #[derive(Clone, Debug)] pub struct FrontendHummockVersion { pub id: HummockVersionId, - pub max_committed_epoch: u64, pub state_table_info: HummockVersionStateTableInfo, pub table_change_log: HashMap>, } @@ -37,7 +37,6 @@ impl FrontendHummockVersion { pub fn from_version(version: &HummockVersion) -> Self { Self { id: version.id, - max_committed_epoch: version.max_committed_epoch, state_table_info: version.state_table_info.clone(), table_change_log: version .table_change_log @@ -66,7 +65,7 @@ impl FrontendHummockVersion { PbHummockVersion { id: self.id.0, levels: Default::default(), - max_committed_epoch: self.max_committed_epoch, + max_committed_epoch: INVALID_EPOCH, table_watermarks: Default::default(), table_change_logs: self .table_change_log @@ -95,7 +94,6 @@ impl FrontendHummockVersion { pub fn from_protobuf(value: PbHummockVersion) -> Self { Self { id: HummockVersionId(value.id), - max_committed_epoch: value.max_committed_epoch, state_table_info: HummockVersionStateTableInfo::from_protobuf(&value.state_table_info), table_change_log: value .table_change_logs @@ -125,7 +123,6 @@ impl FrontendHummockVersion { assert_eq!(self.id, delta.prev_id); } self.id = delta.id; - self.max_committed_epoch = delta.max_committed_epoch; let (changed_table_info, _) = self .state_table_info .apply_delta(&delta.state_table_info_delta, &delta.removed_table_id); @@ -142,7 +139,6 @@ impl FrontendHummockVersion { pub struct FrontendHummockVersionDelta { pub prev_id: HummockVersionId, pub id: HummockVersionId, - pub max_committed_epoch: u64, pub removed_table_id: HashSet, pub state_table_info_delta: HashMap, pub change_log_delta: HashMap>, @@ -153,7 +149,6 @@ impl FrontendHummockVersionDelta { Self { prev_id: delta.prev_id, id: delta.id, - max_committed_epoch: delta.max_committed_epoch, removed_table_id: delta.removed_table_ids.clone(), state_table_info_delta: delta.state_table_info_delta.clone(), change_log_delta: delta @@ -183,7 +178,7 @@ impl FrontendHummockVersionDelta { id: self.id.to_u64(), prev_id: self.prev_id.to_u64(), group_deltas: Default::default(), - max_committed_epoch: self.max_committed_epoch, + max_committed_epoch: INVALID_EPOCH, trivial_move: false, new_table_watermarks: Default::default(), removed_table_ids: self @@ -220,7 +215,6 @@ impl FrontendHummockVersionDelta { Self { prev_id: HummockVersionId::new(delta.prev_id), id: HummockVersionId::new(delta.id), - max_committed_epoch: delta.max_committed_epoch, removed_table_id: delta .removed_table_ids .iter() diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 0a10f7b46e402..7be1cbdb834c4 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -36,7 +36,9 @@ use crate::compaction_group::StaticCompactionGroupId; use crate::level::LevelsCommon; use crate::sstable_info::SstableInfo; use crate::table_watermark::TableWatermarks; -use crate::{CompactionGroupId, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID}; +use crate::{ + CompactionGroupId, HummockEpoch, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID, +}; #[derive(Debug, Clone, PartialEq)] pub struct HummockVersionStateTableInfo { @@ -204,6 +206,13 @@ impl HummockVersionStateTableInfo { pub fn compaction_group_member_tables(&self) -> &HashMap> { &self.compaction_group_member_tables } + + pub fn max_table_committed_epoch(&self) -> Option { + self.state_table_info + .values() + .map(|info| info.committed_epoch) + .max() + } } #[derive(Debug, Clone, PartialEq)] From 4723dbe6aa334fc1aadfb7c6064806cb6c10bae3 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 16:53:47 +0800 Subject: [PATCH 17/27] impl ScanTableVisitor --- src/frontend/src/handler/declare_cursor.rs | 18 +----- src/frontend/src/handler/query.rs | 34 +++++------ src/frontend/src/optimizer/mod.rs | 2 +- .../src/optimizer/plan_visitor/mod.rs | 2 + .../plan_visitor/scan_table_visitor.rs | 57 +++++++++++++++++++ src/frontend/src/session/cursor_manager.rs | 5 +- 6 files changed, 79 insertions(+), 39 deletions(-) create mode 100644 src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index 3ea8fb60b3ef0..f4ae44f8c2b3f 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -157,7 +157,7 @@ pub async fn create_chunk_stream_for_cursor( plan_fragmenter, query_mode, schema, - dependent_relations, + scan_tables, .. } = plan_fragmenter_result; @@ -170,22 +170,10 @@ pub async fn create_chunk_stream_for_cursor( match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => CursorDataChunkStream::LocalDataChunk(Some( - local_execute( - session.clone(), - query, - can_timeout_cancel, - dependent_relations, - ) - .await?, + local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, )), QueryMode::Distributed => CursorDataChunkStream::DistributedDataChunk(Some( - distribute_execute( - session.clone(), - query, - can_timeout_cancel, - dependent_relations, - ) - .await?, + distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, )), }, schema.fields.clone(), diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 510924d59d6bb..4ff0964d9145b 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -39,7 +39,7 @@ use crate::handler::HandlerArgs; use crate::optimizer::plan_node::Explain; use crate::optimizer::{ ExecutionModeDecider, OptimizerContext, OptimizerContextRef, RelationCollectorVisitor, - SysTableVisitor, + ScanTableVisitor, SysTableVisitor, }; use crate::planner::Planner; use crate::scheduler::plan_fragmenter::Query; @@ -206,6 +206,7 @@ pub struct BatchQueryPlanResult { // subset of the final one. i.e. the final one may contain more implicit dependencies on // indices. pub(crate) dependent_relations: HashSet, + pub(crate) scan_tables: HashSet, } fn gen_batch_query_plan( @@ -230,6 +231,8 @@ fn gen_batch_query_plan( let dependent_relations = RelationCollectorVisitor::collect_with(dependent_relations, batch_plan.clone()); + let scan_tables = ScanTableVisitor::collect(batch_plan.clone()); + let must_local = must_run_in_local_mode(batch_plan.clone()); let query_mode = match (must_dist, must_local) { @@ -259,7 +262,8 @@ fn gen_batch_query_plan( query_mode, schema, stmt_type, - dependent_relations: dependent_relations.into_iter().collect(), + dependent_relations, + scan_tables, }) } @@ -311,7 +315,7 @@ pub struct BatchPlanFragmenterResult { pub(crate) query_mode: QueryMode, pub(crate) schema: Schema, pub(crate) stmt_type: StatementType, - pub(crate) dependent_relations: HashSet, + pub(crate) scan_tables: HashSet, } pub fn gen_batch_plan_fragmenter( @@ -323,7 +327,8 @@ pub fn gen_batch_plan_fragmenter( query_mode, schema, stmt_type, - dependent_relations, + scan_tables, + .. } = plan_result; tracing::trace!( @@ -347,7 +352,7 @@ pub fn gen_batch_plan_fragmenter( query_mode, schema, stmt_type, - dependent_relations, + scan_tables, }) } @@ -361,8 +366,7 @@ pub async fn create_stream( query_mode, schema, stmt_type, - dependent_relations, - .. + scan_tables, } = plan_fragmenter_result; let mut can_timeout_cancel = true; @@ -393,13 +397,7 @@ pub async fn create_stream( let row_stream = match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new( - local_execute( - session.clone(), - query, - can_timeout_cancel, - dependent_relations, - ) - .await?, + local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, column_types, formats, session.clone(), @@ -407,13 +405,7 @@ pub async fn create_stream( // Local mode do not support cancel tasks. QueryMode::Distributed => { PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new( - distribute_execute( - session.clone(), - query, - can_timeout_cancel, - dependent_relations, - ) - .await?, + distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, column_types, formats, session.clone(), diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index de5c3deaf0d6b..1e1e7cb51a653 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -29,7 +29,7 @@ pub use plan_rewriter::PlanRewriter; mod plan_visitor; pub use plan_visitor::{ - ExecutionModeDecider, PlanVisitor, RelationCollectorVisitor, SysTableVisitor, + ExecutionModeDecider, PlanVisitor, RelationCollectorVisitor, ScanTableVisitor, SysTableVisitor, }; use risingwave_sqlparser::ast::OnConflict; diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index 63a0484cfdfd5..4bd8c4cd57e43 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -40,7 +40,9 @@ pub use cardinality_visitor::*; mod jsonb_stream_key_checker; pub use jsonb_stream_key_checker::*; mod distributed_dml_visitor; +mod scan_table_visitor; pub use distributed_dml_visitor::*; +pub use scan_table_visitor::*; use crate::for_all_plan_nodes; use crate::optimizer::plan_node::*; diff --git a/src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs b/src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs new file mode 100644 index 0000000000000..0cb385108cfbd --- /dev/null +++ b/src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs @@ -0,0 +1,57 @@ +// 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::HashSet; + +use risingwave_common::catalog::TableId; + +use super::{DefaultBehavior, DefaultValue}; +use crate::optimizer::plan_node::{BatchLogSeqScan, BatchLookupJoin}; +use crate::optimizer::plan_visitor::PlanVisitor; +use crate::PlanRef; + +#[derive(Debug, Clone, Default)] +pub struct ScanTableVisitor { + tables: HashSet, +} + +impl ScanTableVisitor { + pub fn collect(plan: PlanRef) -> HashSet { + let mut visitor = Self::default(); + visitor.visit(plan); + visitor.tables + } +} + +impl PlanVisitor for ScanTableVisitor { + type Result = (); + + type DefaultBehavior = impl DefaultBehavior; + + fn default_behavior() -> Self::DefaultBehavior { + DefaultValue + } + + fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) { + self.tables.insert(plan.core().table_desc.table_id); + } + + fn visit_batch_log_seq_scan(&mut self, plan: &BatchLogSeqScan) -> Self::Result { + self.tables.insert(plan.core().table_desc.table_id); + } + + fn visit_batch_lookup_join(&mut self, plan: &BatchLookupJoin) -> Self::Result { + self.tables.insert(plan.right_table_desc().table_id); + } +} diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index 9f14aae5a1931..3d5dbf4ee9a4e 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -14,7 +14,7 @@ use core::mem; use core::time::Duration; -use std::collections::{HashMap, VecDeque}; +use std::collections::{HashMap, HashSet, VecDeque}; use std::rc::Rc; use std::sync::Arc; use std::time::Instant; @@ -650,7 +650,8 @@ impl SubscriptionCursor { query_mode, schema, stmt_type: StatementType::SELECT, - dependent_relations: table_catalog.dependent_relations.iter().cloned().collect(), + dependent_relations: HashSet::from_iter([table_catalog.id]), + scan_tables: HashSet::from_iter([table_catalog.id]), }) } From 0d0cc2590862bd3a66d5ca70b5c4856f07dc919a Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 18:07:26 +0800 Subject: [PATCH 18/27] remove QuerySnapshot --- src/frontend/src/handler/create_table.rs | 1 + src/frontend/src/handler/declare_cursor.rs | 18 +++++- src/frontend/src/handler/query.rs | 58 ++++++++++++------- .../src/optimizer/logical_optimization.rs | 2 +- src/frontend/src/optimizer/mod.rs | 3 +- .../src/optimizer/plan_visitor/mod.rs | 4 +- ...sitor.rs => read_storage_table_visitor.rs} | 6 +- .../src/scheduler/distributed/query.rs | 24 ++++---- .../scheduler/distributed/query_manager.rs | 6 +- src/frontend/src/scheduler/local.rs | 21 +++---- src/frontend/src/scheduler/snapshot.rs | 47 +++++++++------ src/frontend/src/session.rs | 16 ----- src/frontend/src/session/cursor_manager.rs | 4 +- src/frontend/src/session/transaction.rs | 12 ++-- 14 files changed, 117 insertions(+), 105 deletions(-) rename src/frontend/src/optimizer/plan_visitor/{scan_table_visitor.rs => read_storage_table_visitor.rs} (93%) diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index d91785bd76dbd..e7b2b44226657 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -330,6 +330,7 @@ pub fn bind_sql_column_constraints( // here. Actually we do not require purity of the expression here since we're only to get a // snapshot value. let rewritten_expr_impl = session + .pinned_snapshot() .inline_now_proc_time() .rewrite_expr(expr_impl.clone()); diff --git a/src/frontend/src/handler/declare_cursor.rs b/src/frontend/src/handler/declare_cursor.rs index f4ae44f8c2b3f..8c521be2adacc 100644 --- a/src/frontend/src/handler/declare_cursor.rs +++ b/src/frontend/src/handler/declare_cursor.rs @@ -157,7 +157,7 @@ pub async fn create_chunk_stream_for_cursor( plan_fragmenter, query_mode, schema, - scan_tables, + read_storage_tables, .. } = plan_fragmenter_result; @@ -170,10 +170,22 @@ pub async fn create_chunk_stream_for_cursor( match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => CursorDataChunkStream::LocalDataChunk(Some( - local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + local_execute( + session.clone(), + query, + can_timeout_cancel, + &read_storage_tables, + ) + .await?, )), QueryMode::Distributed => CursorDataChunkStream::DistributedDataChunk(Some( - distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + distribute_execute( + session.clone(), + query, + can_timeout_cancel, + read_storage_tables, + ) + .await?, )), }, schema.fields.clone(), diff --git a/src/frontend/src/handler/query.rs b/src/frontend/src/handler/query.rs index 4ff0964d9145b..a8201d3c40cc6 100644 --- a/src/frontend/src/handler/query.rs +++ b/src/frontend/src/handler/query.rs @@ -38,8 +38,8 @@ use crate::handler::util::{to_pg_field, DataChunkToRowSetAdapter}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::Explain; use crate::optimizer::{ - ExecutionModeDecider, OptimizerContext, OptimizerContextRef, RelationCollectorVisitor, - ScanTableVisitor, SysTableVisitor, + ExecutionModeDecider, OptimizerContext, OptimizerContextRef, ReadStorageTableVisitor, + RelationCollectorVisitor, SysTableVisitor, }; use crate::planner::Planner; use crate::scheduler::plan_fragmenter::Query; @@ -205,8 +205,8 @@ pub struct BatchQueryPlanResult { // Note that these relations are only resolved in the binding phase, and it may only be a // subset of the final one. i.e. the final one may contain more implicit dependencies on // indices. - pub(crate) dependent_relations: HashSet, - pub(crate) scan_tables: HashSet, + pub(crate) dependent_relations: Vec, + pub(crate) read_storage_tables: HashSet, } fn gen_batch_query_plan( @@ -231,7 +231,7 @@ fn gen_batch_query_plan( let dependent_relations = RelationCollectorVisitor::collect_with(dependent_relations, batch_plan.clone()); - let scan_tables = ScanTableVisitor::collect(batch_plan.clone()); + let read_storage_tables = ReadStorageTableVisitor::collect(batch_plan.clone()); let must_local = must_run_in_local_mode(batch_plan.clone()); @@ -262,8 +262,8 @@ fn gen_batch_query_plan( query_mode, schema, stmt_type, - dependent_relations, - scan_tables, + dependent_relations: dependent_relations.into_iter().collect_vec(), + read_storage_tables, }) } @@ -315,7 +315,7 @@ pub struct BatchPlanFragmenterResult { pub(crate) query_mode: QueryMode, pub(crate) schema: Schema, pub(crate) stmt_type: StatementType, - pub(crate) scan_tables: HashSet, + pub(crate) read_storage_tables: HashSet, } pub fn gen_batch_plan_fragmenter( @@ -327,7 +327,7 @@ pub fn gen_batch_plan_fragmenter( query_mode, schema, stmt_type, - scan_tables, + read_storage_tables, .. } = plan_result; @@ -352,7 +352,7 @@ pub fn gen_batch_plan_fragmenter( query_mode, schema, stmt_type, - scan_tables, + read_storage_tables, }) } @@ -366,7 +366,7 @@ pub async fn create_stream( query_mode, schema, stmt_type, - scan_tables, + read_storage_tables, } = plan_fragmenter_result; let mut can_timeout_cancel = true; @@ -397,7 +397,13 @@ pub async fn create_stream( let row_stream = match query_mode { QueryMode::Auto => unreachable!(), QueryMode::Local => PgResponseStream::LocalQuery(DataChunkToRowSetAdapter::new( - local_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + local_execute( + session.clone(), + query, + can_timeout_cancel, + &read_storage_tables, + ) + .await?, column_types, formats, session.clone(), @@ -405,7 +411,13 @@ pub async fn create_stream( // Local mode do not support cancel tasks. QueryMode::Distributed => { PgResponseStream::DistributedQuery(DataChunkToRowSetAdapter::new( - distribute_execute(session.clone(), query, can_timeout_cancel, scan_tables).await?, + distribute_execute( + session.clone(), + query, + can_timeout_cancel, + read_storage_tables, + ) + .await?, column_types, formats, session.clone(), @@ -485,7 +497,7 @@ pub async fn distribute_execute( session: Arc, query: Query, can_timeout_cancel: bool, - scan_tables: HashSet, + read_storage_tables: HashSet, ) -> Result { let timeout = if cfg!(madsim) { None @@ -499,7 +511,7 @@ pub async fn distribute_execute( let query_manager = session.env().query_manager().clone(); query_manager - .schedule(execution_context, query, scan_tables) + .schedule(execution_context, query, read_storage_tables) .await .map_err(|err| err.into()) } @@ -508,7 +520,7 @@ pub async fn local_execute( session: Arc, query: Query, can_timeout_cancel: bool, - dependent_relations: HashSet, + read_storage_tables: &HashSet, ) -> Result { let timeout = if cfg!(madsim) { None @@ -519,12 +531,18 @@ pub async fn local_execute( }; let front_env = session.env(); - // TODO: if there's no table scan, we don't need to acquire snapshot. - let snapshot = session.pinned_snapshot(dependent_relations); + let snapshot = session.pinned_snapshot(); // TODO: Passing sql here - let execution = - LocalQueryExecution::new(query, front_env.clone(), "", snapshot, session, timeout)?; + let execution = LocalQueryExecution::new( + query, + front_env.clone(), + "", + snapshot.support_barrier_read(), + snapshot.batch_query_epoch(read_storage_tables)?, + session, + timeout, + ); Ok(execution.stream_rows()) } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index cdc05610910fe..93da63cf70a42 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -540,7 +540,7 @@ impl LogicalOptimizer { return plan; } - let mut v = ctx.session_ctx().inline_now_proc_time(); + let mut v = ctx.session_ctx().pinned_snapshot().inline_now_proc_time(); let plan = plan.rewrite_exprs_recursive(&mut v); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 1e1e7cb51a653..4e53d54e0a41e 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -29,7 +29,8 @@ pub use plan_rewriter::PlanRewriter; mod plan_visitor; pub use plan_visitor::{ - ExecutionModeDecider, PlanVisitor, RelationCollectorVisitor, ScanTableVisitor, SysTableVisitor, + ExecutionModeDecider, PlanVisitor, ReadStorageTableVisitor, RelationCollectorVisitor, + SysTableVisitor, }; use risingwave_sqlparser::ast::OnConflict; diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index 4bd8c4cd57e43..632d0bcf8bd9d 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -40,9 +40,9 @@ pub use cardinality_visitor::*; mod jsonb_stream_key_checker; pub use jsonb_stream_key_checker::*; mod distributed_dml_visitor; -mod scan_table_visitor; +mod read_storage_table_visitor; pub use distributed_dml_visitor::*; -pub use scan_table_visitor::*; +pub use read_storage_table_visitor::*; use crate::for_all_plan_nodes; use crate::optimizer::plan_node::*; diff --git a/src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs b/src/frontend/src/optimizer/plan_visitor/read_storage_table_visitor.rs similarity index 93% rename from src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs rename to src/frontend/src/optimizer/plan_visitor/read_storage_table_visitor.rs index 0cb385108cfbd..8119589305208 100644 --- a/src/frontend/src/optimizer/plan_visitor/scan_table_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/read_storage_table_visitor.rs @@ -22,11 +22,11 @@ use crate::optimizer::plan_visitor::PlanVisitor; use crate::PlanRef; #[derive(Debug, Clone, Default)] -pub struct ScanTableVisitor { +pub struct ReadStorageTableVisitor { tables: HashSet, } -impl ScanTableVisitor { +impl ReadStorageTableVisitor { pub fn collect(plan: PlanRef) -> HashSet { let mut visitor = Self::default(); visitor.visit(plan); @@ -34,7 +34,7 @@ impl ScanTableVisitor { } } -impl PlanVisitor for ScanTableVisitor { +impl PlanVisitor for ReadStorageTableVisitor { type Result = (); type DefaultBehavior = impl DefaultBehavior; diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index b0cee2beb5070..a285039aed0fb 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -40,7 +40,7 @@ use crate::scheduler::distributed::stage::StageEvent::ScheduledRoot; use crate::scheduler::distributed::StageEvent::Scheduled; use crate::scheduler::distributed::StageExecution; use crate::scheduler::plan_fragmenter::{Query, StageId, ROOT_TASK_ID, ROOT_TASK_OUTPUT_ID}; -use crate::scheduler::{ExecutionContextRef, QuerySnapshot, SchedulerError, SchedulerResult}; +use crate::scheduler::{ExecutionContextRef, SchedulerError, SchedulerResult}; /// Message sent to a `QueryRunner` to control its execution. #[derive(Debug)] @@ -124,7 +124,7 @@ impl QueryExecution { self: Arc, context: ExecutionContextRef, worker_node_manager: WorkerNodeSelector, - pinned_snapshot: QuerySnapshot, + batch_query_epoch: BatchQueryEpoch, compute_client_pool: ComputeClientPoolRef, catalog_reader: CatalogReader, query_execution_info: QueryExecutionInfoRef, @@ -137,7 +137,7 @@ impl QueryExecution { // reference of `pinned_snapshot`. Its ownership will be moved into `QueryRunner` so that it // can control when to release the snapshot. let stage_executions = self.gen_stage_executions( - pinned_snapshot.batch_query_epoch()?, + batch_query_epoch, context.clone(), worker_node_manager, compute_client_pool.clone(), @@ -182,13 +182,13 @@ impl QueryExecution { let span = tracing::info_span!( "distributed_execute", query_id = self.query.query_id.id, - epoch = ?pinned_snapshot.batch_query_epoch(), + epoch = ?batch_query_epoch, ); tracing::trace!("Starting query: {:?}", self.query.query_id); // Not trace the error here, it will be processed in scheduler. - tokio::spawn(async move { runner.run(pinned_snapshot).instrument(span).await }); + tokio::spawn(async move { runner.run().instrument(span).await }); let root_stage = root_stage_receiver .await @@ -296,7 +296,7 @@ impl Debug for QueryRunner { } impl QueryRunner { - async fn run(mut self, pinned_snapshot: QuerySnapshot) { + async fn run(mut self) { self.query_metrics.running_query_num.inc(); // Start leaf stages. let leaf_stages = self.query.leaf_stages(); @@ -310,8 +310,6 @@ impl QueryRunner { } let mut stages_with_table_scan = self.query.stages_with_table_scan(); let has_lookup_join_stage = self.query.has_lookup_join_stage(); - // To convince the compiler that `pinned_snapshot` will only be dropped once. - let mut pinned_snapshot_to_drop = Some(pinned_snapshot); let mut finished_stage_cnt = 0usize; while let Some(msg_inner) = self.msg_receiver.recv().await { @@ -331,7 +329,6 @@ impl QueryRunner { // thus they all successfully pinned a HummockVersion. // So we can now unpin their epoch. tracing::trace!("Query {:?} has scheduled all of its stages that have table scan (iterator creation).", self.query.query_id); - pinned_snapshot_to_drop.take(); } // For root stage, we execute in frontend local. We will pass the root fragment @@ -495,7 +492,7 @@ pub(crate) mod tests { use crate::scheduler::distributed::QueryExecution; use crate::scheduler::plan_fragmenter::{BatchPlanFragmenter, Query}; use crate::scheduler::{ - DistributedQueryMetrics, ExecutionContext, QueryExecutionInfo, QuerySnapshot, ReadSnapshot, + DistributedQueryMetrics, ExecutionContext, QueryExecutionInfo, ReadSnapshot, }; use crate::session::SessionImpl; use crate::utils::Condition; @@ -519,10 +516,9 @@ pub(crate) mod tests { .start( ExecutionContext::new(SessionImpl::mock().into(), None).into(), worker_node_selector, - QuerySnapshot::new( - ReadSnapshot::ReadUncommitted, - HashSet::from_iter([0.into()]) - ), + ReadSnapshot::ReadUncommitted + .batch_query_epoch(&HashSet::from_iter([0.into()])) + .unwrap(), compute_client_pool, catalog_reader, query_execution_info, diff --git a/src/frontend/src/scheduler/distributed/query_manager.rs b/src/frontend/src/scheduler/distributed/query_manager.rs index 6464759ebbd18..a448f0e5b5b9b 100644 --- a/src/frontend/src/scheduler/distributed/query_manager.rs +++ b/src/frontend/src/scheduler/distributed/query_manager.rs @@ -191,7 +191,7 @@ impl QueryManager { &self, context: ExecutionContextRef, query: Query, - scan_tables: HashSet, + read_storage_tables: HashSet, ) -> SchedulerResult { if let Some(query_limit) = self.disrtibuted_query_limit && self.query_metrics.running_query_num.get() as u64 == query_limit @@ -214,7 +214,7 @@ impl QueryManager { .add_query(query_id.clone(), query_execution.clone()); // TODO: if there's no table scan, we don't need to acquire snapshot. - let pinned_snapshot = context.session().pinned_snapshot(scan_tables); + let pinned_snapshot = context.session().pinned_snapshot(); let worker_node_manager_reader = WorkerNodeSelector::new( self.worker_node_manager.clone(), @@ -225,7 +225,7 @@ impl QueryManager { .start( context.clone(), worker_node_manager_reader, - pinned_snapshot, + pinned_snapshot.batch_query_epoch(&read_storage_tables)?, self.compute_client_pool.clone(), self.catalog_reader.clone(), self.query_execution_info.clone(), diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index b423dcce03f2f..b11d462b151c7 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -52,7 +52,7 @@ use crate::error::RwError; use crate::optimizer::plan_node::PlanNodeType; use crate::scheduler::plan_fragmenter::{ExecutionPlanNode, Query, StageId}; use crate::scheduler::task_context::FrontendBatchTaskContext; -use crate::scheduler::{QuerySnapshot, SchedulerError, SchedulerResult}; +use crate::scheduler::{SchedulerError, SchedulerResult}; use crate::session::{FrontendEnv, SessionImpl}; // TODO(error-handling): use a concrete error type. @@ -61,8 +61,6 @@ pub struct LocalQueryExecution { sql: String, query: Query, front_env: FrontendEnv, - // The snapshot will be released when LocalQueryExecution is dropped. - _snapshot: QuerySnapshot, batch_query_epoch: BatchQueryEpoch, session: Arc, worker_node_manager: WorkerNodeSelector, @@ -74,27 +72,24 @@ impl LocalQueryExecution { query: Query, front_env: FrontendEnv, sql: S, - snapshot: QuerySnapshot, + support_barrier_read: bool, + batch_query_epoch: BatchQueryEpoch, session: Arc, timeout: Option, - ) -> SchedulerResult { + ) -> Self { let sql = sql.into(); - let worker_node_manager = WorkerNodeSelector::new( - front_env.worker_node_manager_ref(), - snapshot.support_barrier_read(), - ); - let batch_query_epoch = snapshot.batch_query_epoch()?; + let worker_node_manager = + WorkerNodeSelector::new(front_env.worker_node_manager_ref(), support_barrier_read); - Ok(Self { + Self { sql, query, front_env, - _snapshot: snapshot, batch_query_epoch, session, worker_node_manager, timeout, - }) + } } fn shutdown_rx(&self) -> ShutdownToken { diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index e728b74b30b65..0ce10dd07e5ed 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -26,6 +26,7 @@ use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::hummock::{HummockVersionDeltas, StateTableInfoDelta}; use tokio::sync::watch; +use crate::expr::InlineNowProcTime; use crate::meta_client::FrontendMetaClient; use crate::scheduler::SchedulerError; @@ -45,25 +46,16 @@ pub enum ReadSnapshot { Other(Epoch), } -pub struct QuerySnapshot { - snapshot: ReadSnapshot, - scan_tables: HashSet, -} - -impl QuerySnapshot { - pub fn new(snapshot: ReadSnapshot, scan_tables: HashSet) -> Self { - Self { - snapshot, - scan_tables, - } - } - +impl ReadSnapshot { /// Get the [`BatchQueryEpoch`] for this snapshot. - pub fn batch_query_epoch(&self) -> Result { - Ok(match &self.snapshot { + pub fn batch_query_epoch( + &self, + read_storage_tables: &HashSet, + ) -> Result { + Ok(match self { ReadSnapshot::FrontendPinned { snapshot } => BatchQueryEpoch { epoch: Some(batch_query_epoch::Epoch::Committed( - snapshot.batch_query_epoch(&self.scan_tables)?.0, + snapshot.batch_query_epoch(read_storage_tables)?.0, )), }, ReadSnapshot::ReadUncommitted => BatchQueryEpoch { @@ -75,9 +67,23 @@ impl QuerySnapshot { }) } + pub fn inline_now_proc_time(&self) -> InlineNowProcTime { + let epoch = match self { + ReadSnapshot::FrontendPinned { snapshot } => snapshot + .value + .state_table_info + .max_table_committed_epoch() + .map(Epoch) + .unwrap_or_else(Epoch::now), + ReadSnapshot::ReadUncommitted => Epoch::now(), + ReadSnapshot::Other(epoch) => *epoch, + }; + InlineNowProcTime::new(epoch) + } + /// Returns true if this snapshot is a barrier read. pub fn support_barrier_read(&self) -> bool { - matches!(&self.snapshot, ReadSnapshot::ReadUncommitted) + matches!(self, ReadSnapshot::ReadUncommitted) } } @@ -97,9 +103,12 @@ impl std::fmt::Debug for PinnedSnapshot { pub type PinnedSnapshotRef = Arc; impl PinnedSnapshot { - fn batch_query_epoch(&self, scan_tables: &HashSet) -> Result { + fn batch_query_epoch( + &self, + read_storage_tables: &HashSet, + ) -> Result { // use the min committed epoch of tables involved in the scan - let epoch = scan_tables + let epoch = read_storage_tables .iter() .map(|table_id| { self.value diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 552fb08eb60f0..bb4e6928d3649 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -60,7 +60,6 @@ use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::types::DataType; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::cluster_limit::ActorCountPerParallelism; -use risingwave_common::util::epoch::Epoch; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_common::util::{cluster_limit, resource_util}; @@ -97,7 +96,6 @@ use crate::catalog::{ check_schema_writable, CatalogError, DatabaseId, OwnedByUserCatalog, SchemaId, TableId, }; use crate::error::{ErrorCode, Result, RwError}; -use crate::expr::InlineNowProcTime; use crate::handler::describe::infer_describe; use crate::handler::extended_handle::{ handle_bind, handle_execute, handle_parse, Portal, PrepareStatement, @@ -839,20 +837,6 @@ impl SessionImpl { self.id } - pub fn inline_now_proc_time(&self) -> InlineNowProcTime { - // use the maximum committed_epoch over all tables as the now value, or current timestamp when there is no table - InlineNowProcTime::new( - self.env() - .hummock_snapshot_manager() - .acquire() - .version() - .state_table_info - .max_table_committed_epoch() - .map(Epoch) - .unwrap_or_else(Epoch::now), - ) - } - pub fn running_sql(&self) -> Option> { self.exec_context .lock() diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index 3d5dbf4ee9a4e..aa798a236b5c8 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -650,8 +650,8 @@ impl SubscriptionCursor { query_mode, schema, stmt_type: StatementType::SELECT, - dependent_relations: HashSet::from_iter([table_catalog.id]), - scan_tables: HashSet::from_iter([table_catalog.id]), + dependent_relations: table_catalog.dependent_relations.clone(), + read_storage_tables: HashSet::from_iter([table_catalog.id]), }) } diff --git a/src/frontend/src/session/transaction.rs b/src/frontend/src/session/transaction.rs index 05fde4f91fd6f..085a66c01af10 100644 --- a/src/frontend/src/session/transaction.rs +++ b/src/frontend/src/session/transaction.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::{Arc, Weak}; @@ -22,9 +21,8 @@ use risingwave_hummock_sdk::EpochWithGap; use super::SessionImpl; use crate::catalog::catalog_service::CatalogWriter; -use crate::catalog::TableId; use crate::error::{ErrorCode, Result}; -use crate::scheduler::{QuerySnapshot, ReadSnapshot}; +use crate::scheduler::ReadSnapshot; use crate::user::user_service::UserInfoWriter; /// Globally unique transaction id in this frontend instance. @@ -215,9 +213,8 @@ impl SessionImpl { /// Acquires and pins a snapshot for the current transaction. /// /// If a snapshot is already acquired, returns it directly. - pub fn pinned_snapshot(&self, scan_tables: HashSet) -> QuerySnapshot { - let snapshot = self - .txn_ctx() + pub fn pinned_snapshot(&self) -> ReadSnapshot { + self.txn_ctx() .snapshot .get_or_insert_with(|| { // query_epoch must be pure epoch @@ -240,8 +237,7 @@ impl SessionImpl { } } }) - .clone(); - QuerySnapshot::new(snapshot, scan_tables) + .clone() } } From fee2a0f821f5aaefc0f0cab0c062920680091f3d Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 23 Sep 2024 18:52:13 +0800 Subject: [PATCH 19/27] use epoch::now --- src/frontend/src/scheduler/snapshot.rs | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 0ce10dd07e5ed..49667b2aa143b 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -129,13 +129,7 @@ impl PinnedSnapshot { } }) })? - .unwrap_or_else(|| { - self.value - .state_table_info - .max_table_committed_epoch() - .map(Epoch) - .unwrap_or_else(Epoch::now) - }); + .unwrap_or_else(Epoch::now); Ok(epoch) } From 44a3b4a34cc999d8d88355981edb39812f9564cc Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 25 Sep 2024 14:47:32 +0800 Subject: [PATCH 20/27] separate wait epoch of batch and streaming --- proto/common.proto | 7 ++- src/batch/src/executor/log_row_seq_scan.rs | 22 ++++++-- src/batch/src/executor/mod.rs | 4 +- src/batch/src/task/task_manager.rs | 4 +- src/compute/tests/cdc_tests.rs | 4 +- src/compute/tests/integration_tests.rs | 10 ++-- .../optimizer/plan_node/batch_log_seq_scan.rs | 12 +++-- .../optimizer/plan_node/generic/log_scan.rs | 4 ++ src/frontend/src/scheduler/snapshot.rs | 7 ++- src/frontend/src/session/cursor_manager.rs | 16 ++++++ src/storage/hummock_sdk/src/lib.rs | 22 +++++--- src/storage/hummock_trace/src/opts.rs | 9 +++- .../src/hummock/store/hummock_storage.rs | 54 ++++++++++++++++--- .../hummock/store/local_hummock_storage.rs | 41 +++++++++++--- src/storage/src/hummock/utils.rs | 46 +++++----------- .../src/table/batch_table/storage_table.rs | 12 +++-- .../executor/backfill/snapshot_backfill.rs | 13 +++-- 17 files changed, 201 insertions(+), 86 deletions(-) diff --git a/proto/common.proto b/proto/common.proto index 05d938cc26523..1030d07d7c343 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -105,9 +105,14 @@ message WorkerSlotMapping { repeated uint64 data = 2; } +message BatchQueryCommittedEpoch { + uint64 epoch = 1; + uint64 hummock_version_id = 2; +} + message BatchQueryEpoch { oneof epoch { - uint64 committed = 1; + BatchQueryCommittedEpoch committed = 1; uint64 current = 2; uint64 backup = 3; uint64 time_travel = 4; diff --git a/src/batch/src/executor/log_row_seq_scan.rs b/src/batch/src/executor/log_row_seq_scan.rs index a36440f1e010d..b71772e18768e 100644 --- a/src/batch/src/executor/log_row_seq_scan.rs +++ b/src/batch/src/executor/log_row_seq_scan.rs @@ -25,6 +25,7 @@ use risingwave_common::catalog::{ColumnId, Field, Schema}; use risingwave_common::hash::VirtualNode; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ScalarImpl; +use risingwave_hummock_sdk::{HummockReadEpoch, HummockVersionId}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::plan_common::StorageTableDesc; @@ -50,6 +51,7 @@ pub struct LogRowSeqScanExecutor { table: StorageTable, old_epoch: u64, new_epoch: u64, + version_id: HummockVersionId, } impl LogRowSeqScanExecutor { @@ -57,6 +59,7 @@ impl LogRowSeqScanExecutor { table: StorageTable, old_epoch: u64, new_epoch: u64, + version_id: HummockVersionId, chunk_size: usize, identity: String, metrics: Option, @@ -74,6 +77,7 @@ impl LogRowSeqScanExecutor { table, old_epoch, new_epoch, + version_id, } } } @@ -128,12 +132,18 @@ impl BoxedExecutorBuilder for LogStoreRowSeqScanExecutorBuilder { unreachable!("invalid new epoch: {:?}", log_store_seq_scan_node.new_epoch) }; + assert_eq!(old_epoch.hummock_version_id, new_epoch.hummock_version_id); + let version_id = old_epoch.hummock_version_id; + let old_epoch = old_epoch.epoch; + let new_epoch = new_epoch.epoch; + dispatch_state_store!(source.context().state_store(), state_store, { let table = StorageTable::new_partial(state_store, column_ids, vnodes, table_desc); Ok(Box::new(LogRowSeqScanExecutor::new( table, - *old_epoch, - *new_epoch, + old_epoch, + new_epoch, + HummockVersionId::new(version_id), chunk_size as usize, source.plan_node().get_identity().clone(), metrics, @@ -164,6 +174,7 @@ impl LogRowSeqScanExecutor { table, old_epoch, new_epoch, + version_id, schema, .. } = *self; @@ -180,6 +191,7 @@ impl LogRowSeqScanExecutor { table.clone(), old_epoch, new_epoch, + version_id, chunk_size, histogram, Arc::new(schema.clone()), @@ -196,13 +208,17 @@ impl LogRowSeqScanExecutor { table: Arc>, old_epoch: u64, new_epoch: u64, + version_id: HummockVersionId, chunk_size: usize, histogram: Option>, schema: Arc, ) { // Range Scan. let iter = table - .batch_iter_log_with_pk_bounds(old_epoch, new_epoch) + .batch_iter_log_with_pk_bounds( + old_epoch, + HummockReadEpoch::BatchQueryCommitted(new_epoch, version_id), + ) .await? .flat_map(|r| { futures::stream::iter(std::iter::from_coroutine( diff --git a/src/batch/src/executor/mod.rs b/src/batch/src/executor/mod.rs index 07be18ca72988..53dabccaf260f 100644 --- a/src/batch/src/executor/mod.rs +++ b/src/batch/src/executor/mod.rs @@ -260,7 +260,7 @@ impl<'a, C: BatchTaskContext> ExecutorBuilder<'a, C> { #[cfg(test)] mod tests { - use risingwave_hummock_sdk::to_committed_batch_query_epoch; + use risingwave_hummock_sdk::test_batch_query_epoch; use risingwave_pb::batch_plan::PlanNode; use crate::executor::ExecutorBuilder; @@ -278,7 +278,7 @@ mod tests { &plan_node, task_id, ComputeNodeContext::for_test(), - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), ShutdownToken::empty(), ); let child_plan = &PlanNode::default(); diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index f25ce88379905..4db15df2dbe85 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -137,12 +137,12 @@ impl BatchManager { tid: &PbTaskId, plan: PlanFragment, ) -> Result<()> { - use risingwave_hummock_sdk::to_committed_batch_query_epoch; + use risingwave_hummock_sdk::test_batch_query_epoch; self.fire_task( tid, plan, - to_committed_batch_query_epoch(0), + test_batch_query_epoch(), ComputeNodeContext::for_test(), StateReporter::new_with_test(), TracingContext::none(), diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 8fea0f48fa82d..fa92e4eefc3a9 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -39,7 +39,7 @@ use risingwave_connector::source::cdc::external::{ }; use risingwave_connector::source::cdc::DebeziumCdcSplit; use risingwave_connector::source::SplitImpl; -use risingwave_hummock_sdk::to_committed_batch_query_epoch; +use risingwave_hummock_sdk::test_batch_query_epoch; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_stream::common::table::state_table::StateTable; @@ -384,7 +384,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { table.clone(), vec![ScanRange::full()], true, - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), 1024, "RowSeqExecutor2".to_string(), None, diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 13a76c6989b48..f36cd6cf8164f 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -41,7 +41,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_connector::source::reader::desc::test_utils::create_source_desc_builder; use risingwave_dml::dml_manager::DmlManager; -use risingwave_hummock_sdk::to_committed_batch_query_epoch; +use risingwave_hummock_sdk::test_batch_query_epoch; use risingwave_pb::catalog::StreamSourceInfo; use risingwave_pb::plan_common::PbRowFormatType; use risingwave_storage::memory::MemoryStateStore; @@ -263,7 +263,7 @@ async fn test_table_materialize() -> StreamResult<()> { table.clone(), vec![ScanRange::full()], true, - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), 1024, "RowSeqExecutor2".to_string(), None, @@ -334,7 +334,7 @@ async fn test_table_materialize() -> StreamResult<()> { table.clone(), vec![ScanRange::full()], true, - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), 1024, "RowSeqScanExecutor2".to_string(), None, @@ -414,7 +414,7 @@ async fn test_table_materialize() -> StreamResult<()> { table, vec![ScanRange::full()], true, - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), 1024, "RowSeqScanExecutor2".to_string(), None, @@ -490,7 +490,7 @@ async fn test_row_seq_scan() -> StreamResult<()> { table, vec![ScanRange::full()], true, - to_committed_batch_query_epoch(u64::MAX), + test_batch_query_epoch(), 1, "RowSeqScanExecutor2".to_string(), None, diff --git a/src/frontend/src/optimizer/plan_node/batch_log_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_log_seq_scan.rs index 93132ce65e51c..de8cadda4502d 100644 --- a/src/frontend/src/optimizer/plan_node/batch_log_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_log_seq_scan.rs @@ -15,7 +15,7 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::LogRowSeqScanNode; -use risingwave_pb::common::BatchQueryEpoch; +use risingwave_pb::common::{BatchQueryCommittedEpoch, BatchQueryEpoch}; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; @@ -112,12 +112,18 @@ impl TryToBatchPb for BatchLogSeqScan { vnode_bitmap: None, old_epoch: Some(BatchQueryEpoch { epoch: Some(risingwave_pb::common::batch_query_epoch::Epoch::Committed( - self.core.old_epoch, + BatchQueryCommittedEpoch { + epoch: self.core.old_epoch, + hummock_version_id: 0, + }, )), }), new_epoch: Some(BatchQueryEpoch { epoch: Some(risingwave_pb::common::batch_query_epoch::Epoch::Committed( - self.core.new_epoch, + BatchQueryCommittedEpoch { + epoch: self.core.new_epoch, + hummock_version_id: 0, + }, )), }), })) diff --git a/src/frontend/src/optimizer/plan_node/generic/log_scan.rs b/src/frontend/src/optimizer/plan_node/generic/log_scan.rs index 9a1e53aeab758..793cbaea4f205 100644 --- a/src/frontend/src/optimizer/plan_node/generic/log_scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/log_scan.rs @@ -20,6 +20,7 @@ use pretty_xmlish::Pretty; use risingwave_common::catalog::{Field, Schema, TableDesc}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::ColumnOrder; +use risingwave_hummock_sdk::HummockVersionId; use crate::catalog::ColumnId; use crate::optimizer::optimizer_context::OptimizerContextRef; @@ -44,6 +45,7 @@ pub struct LogScan { pub old_epoch: u64, pub new_epoch: u64, + pub version_id: HummockVersionId, } impl LogScan { @@ -101,6 +103,7 @@ impl LogScan { ctx: OptimizerContextRef, old_epoch: u64, new_epoch: u64, + version_id: HummockVersionId, ) -> Self { Self { table_name, @@ -110,6 +113,7 @@ impl LogScan { ctx, old_epoch, new_epoch, + version_id, } } diff --git a/src/frontend/src/scheduler/snapshot.rs b/src/frontend/src/scheduler/snapshot.rs index 49667b2aa143b..bb1d98aa5f8f7 100644 --- a/src/frontend/src/scheduler/snapshot.rs +++ b/src/frontend/src/scheduler/snapshot.rs @@ -22,7 +22,7 @@ use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_hummock_sdk::{ FrontendHummockVersion, FrontendHummockVersionDelta, HummockVersionId, INVALID_VERSION_ID, }; -use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; +use risingwave_pb::common::{batch_query_epoch, BatchQueryCommittedEpoch, BatchQueryEpoch}; use risingwave_pb::hummock::{HummockVersionDeltas, StateTableInfoDelta}; use tokio::sync::watch; @@ -55,7 +55,10 @@ impl ReadSnapshot { Ok(match self { ReadSnapshot::FrontendPinned { snapshot } => BatchQueryEpoch { epoch: Some(batch_query_epoch::Epoch::Committed( - snapshot.batch_query_epoch(read_storage_tables)?.0, + BatchQueryCommittedEpoch { + epoch: snapshot.batch_query_epoch(read_storage_tables)?.0, + hummock_version_id: snapshot.value.id.to_u64(), + }, )), }, ReadSnapshot::ReadUncommitted => BatchQueryEpoch { diff --git a/src/frontend/src/session/cursor_manager.rs b/src/frontend/src/session/cursor_manager.rs index aa798a236b5c8..5270aced1f13f 100644 --- a/src/frontend/src/session/cursor_manager.rs +++ b/src/frontend/src/session/cursor_manager.rs @@ -30,6 +30,7 @@ use risingwave_common::catalog::Field; use risingwave_common::error::BoxedError; use risingwave_common::session_config::QueryMode; use risingwave_common::types::DataType; +use risingwave_hummock_sdk::HummockVersionId; use risingwave_sqlparser::ast::{Ident, ObjectName, Statement}; use super::SessionImpl; @@ -523,6 +524,18 @@ impl SubscriptionCursor { let init_query_timer = Instant::now(); let (chunk_stream, fields) = if let Some(rw_timestamp) = rw_timestamp { let context = OptimizerContext::from_handler_args(handle_args); + let version_id = { + let version = session.env.hummock_snapshot_manager.acquire(); + let version = version.version(); + if !version + .state_table_info + .info() + .contains_key(dependent_table_id) + { + return Err(anyhow!("table id {dependent_table_id} has been dropped").into()); + } + version.id + }; let plan_fragmenter_result = gen_batch_plan_fragmenter( &session, Self::create_batch_plan_for_cursor( @@ -531,6 +544,7 @@ impl SubscriptionCursor { context.into(), rw_timestamp, rw_timestamp, + version_id, )?, )?; create_chunk_stream_for_cursor(session, plan_fragmenter_result).await? @@ -606,6 +620,7 @@ impl SubscriptionCursor { context: OptimizerContextRef, old_epoch: u64, new_epoch: u64, + version_id: HummockVersionId, ) -> Result { let out_col_idx = table_catalog .columns @@ -621,6 +636,7 @@ impl SubscriptionCursor { context, old_epoch, new_epoch, + version_id, ); let batch_log_seq_scan = BatchLogSeqScan::new(core); let schema = batch_log_seq_scan diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index 9eed880f8a5be..f1fee55fb1fca 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -208,8 +208,10 @@ impl PartialEq for LocalSstableInfo { /// Package read epoch of hummock, it be used for `wait_epoch` #[derive(Debug, Clone, Copy)] pub enum HummockReadEpoch { - /// We need to wait the `max_committed_epoch` + /// We need to wait the `committed_epoch` of the read table Committed(HummockEpoch), + /// We need to wait the `committed_epoch` of the read table and also the hummock version to the version id + BatchQueryCommitted(HummockEpoch, HummockVersionId), /// We don't need to wait epoch, we usually do stream reading with it. NoWait(HummockEpoch), /// We don't need to wait epoch. @@ -220,7 +222,10 @@ pub enum HummockReadEpoch { impl From for HummockReadEpoch { fn from(e: BatchQueryEpoch) -> Self { match e.epoch.unwrap() { - batch_query_epoch::Epoch::Committed(epoch) => HummockReadEpoch::Committed(epoch), + batch_query_epoch::Epoch::Committed(epoch) => HummockReadEpoch::BatchQueryCommitted( + epoch.epoch, + HummockVersionId::new(epoch.hummock_version_id), + ), batch_query_epoch::Epoch::Current(epoch) => { if epoch != HummockEpoch::MAX { warn!( @@ -236,19 +241,20 @@ impl From for HummockReadEpoch { } } -pub fn to_committed_batch_query_epoch(epoch: u64) -> BatchQueryEpoch { +pub fn test_batch_query_epoch() -> BatchQueryEpoch { BatchQueryEpoch { - epoch: Some(batch_query_epoch::Epoch::Committed(epoch)), + epoch: Some(batch_query_epoch::Epoch::Current(u64::MAX)), } } impl HummockReadEpoch { pub fn get_epoch(&self) -> HummockEpoch { *match self { - HummockReadEpoch::Committed(epoch) => epoch, - HummockReadEpoch::NoWait(epoch) => epoch, - HummockReadEpoch::Backup(epoch) => epoch, - HummockReadEpoch::TimeTravel(epoch) => epoch, + HummockReadEpoch::Committed(epoch) + | HummockReadEpoch::BatchQueryCommitted(epoch, _) + | HummockReadEpoch::NoWait(epoch) + | HummockReadEpoch::Backup(epoch) + | HummockReadEpoch::TimeTravel(epoch) => epoch, } } } diff --git a/src/storage/hummock_trace/src/opts.rs b/src/storage/hummock_trace/src/opts.rs index e337152289bcc..480f0315eb396 100644 --- a/src/storage/hummock_trace/src/opts.rs +++ b/src/storage/hummock_trace/src/opts.rs @@ -18,7 +18,7 @@ use risingwave_common::bitmap::Bitmap; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::EpochPair; -use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_hummock_sdk::{HummockReadEpoch, HummockVersionId}; use risingwave_pb::common::PbBuffer; use crate::TracedBytes; @@ -197,6 +197,7 @@ pub type TracedHummockEpoch = u64; #[derive(Debug, Clone, PartialEq, Eq, Decode, Encode)] pub enum TracedHummockReadEpoch { Committed(TracedHummockEpoch), + BatchQueryReadCommitted(TracedHummockEpoch, u64), NoWait(TracedHummockEpoch), Backup(TracedHummockEpoch), TimeTravel(TracedHummockEpoch), @@ -206,6 +207,9 @@ impl From for TracedHummockReadEpoch { fn from(value: HummockReadEpoch) -> Self { match value { HummockReadEpoch::Committed(epoch) => Self::Committed(epoch), + HummockReadEpoch::BatchQueryCommitted(epoch, version_id) => { + Self::BatchQueryReadCommitted(epoch, version_id.to_u64()) + } HummockReadEpoch::NoWait(epoch) => Self::NoWait(epoch), HummockReadEpoch::Backup(epoch) => Self::Backup(epoch), HummockReadEpoch::TimeTravel(epoch) => Self::TimeTravel(epoch), @@ -217,6 +221,9 @@ impl From for HummockReadEpoch { fn from(value: TracedHummockReadEpoch) -> Self { match value { TracedHummockReadEpoch::Committed(epoch) => Self::Committed(epoch), + TracedHummockReadEpoch::BatchQueryReadCommitted(epoch, version_id) => { + Self::BatchQueryCommitted(epoch, HummockVersionId::new(version_id)) + } TracedHummockReadEpoch::NoWait(epoch) => Self::NoWait(epoch), TracedHummockReadEpoch::Backup(epoch) => Self::Backup(epoch), TracedHummockReadEpoch::TimeTravel(epoch) => Self::TimeTravel(epoch), diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index f665bf3edc15f..434a4ba989d52 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -53,7 +53,7 @@ use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, use crate::hummock::local_version::recent_versions::RecentVersions; use crate::hummock::observer_manager::HummockObserverNode; use crate::hummock::time_travel_version_cache::SimpleTimeTravelVersionCache; -use crate::hummock::utils::wait_for_epoch; +use crate::hummock::utils::wait_for_update; use crate::hummock::write_limiter::{WriteLimiter, WriteLimiterRef}; use crate::hummock::{ HummockEpoch, HummockError, HummockResult, HummockStorageIterator, HummockStorageRevIterator, @@ -633,14 +633,54 @@ impl StateStore for HummockStorage { wait_epoch: HummockReadEpoch, options: TryWaitEpochOptions, ) -> StorageResult<()> { - let wait_epoch = match wait_epoch { - HummockReadEpoch::Committed(epoch) => { - assert!(!is_max_epoch(epoch), "epoch should not be MAX EPOCH"); - epoch - } + let (wait_epoch, wait_version) = match wait_epoch { + HummockReadEpoch::Committed(epoch) => (epoch, None), + HummockReadEpoch::BatchQueryCommitted(epoch, version_id) => (epoch, Some(version_id)), _ => return Ok(()), }; - wait_for_epoch(&self.version_update_notifier_tx, wait_epoch, options).await + assert!(!is_max_epoch(wait_epoch), "epoch should not be MAX EPOCH"); + let is_valid_version_id = |id| match wait_version { + None => true, + Some(version_id) => id >= version_id, + }; + // fast path by checking recent_versions + { + let recent_versions = self.recent_versions.load(); + let latest_version = recent_versions.latest_version().version(); + if is_valid_version_id(latest_version.id) + && let Some(committed_epoch) = + latest_version.table_committed_epoch(options.table_id) + && committed_epoch >= wait_epoch + { + return Ok(()); + } + } + wait_for_update( + &self.version_update_notifier_tx, + |version| { + if !is_valid_version_id(version.version().id) { + return Ok(false); + } + let committed_epoch = version + .version() + .table_committed_epoch(options.table_id) + .ok_or_else(|| { + HummockError::wait_epoch(format!( + "table id {} has been dropped", + options.table_id + )) + })?; + Ok(committed_epoch >= wait_epoch) + }, + || { + format!( + "try_wait_epoch: epoch: {}, version_id: {:?}", + wait_epoch, wait_version + ) + }, + ) + .await?; + Ok(()) } fn sync(&self, epoch: u64, table_ids: HashSet) -> impl SyncFuture { diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index ae3815ca551a5..95cee5899cdbf 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -45,7 +45,7 @@ use crate::hummock::shared_buffer::shared_buffer_batch::{ use crate::hummock::store::version::{read_filter_for_version, HummockVersionReader}; use crate::hummock::utils::{ do_delete_sanity_check, do_insert_sanity_check, do_update_sanity_check, sanity_check_enabled, - wait_for_epoch, + wait_for_update, }; use crate::hummock::write_limiter::WriteLimiterRef; use crate::hummock::{ @@ -136,14 +136,43 @@ impl LocalHummockStorage { } async fn wait_for_epoch(&self, wait_epoch: u64) -> StorageResult<()> { - wait_for_epoch( + let mut prev_committed_epoch = None; + let prev_committed_epoch = &mut prev_committed_epoch; + wait_for_update( &self.version_update_notifier_tx, - wait_epoch, - TryWaitEpochOptions { - table_id: self.table_id, + |version| { + let committed_epoch = version.version().table_committed_epoch(self.table_id); + let ret = if let Some(committed_epoch) = committed_epoch { + if committed_epoch >= wait_epoch { + Ok(true) + } else { + Ok(false) + } + } else if prev_committed_epoch.is_none() { + warn!( + table_id = self.table_id.table_id, + version_id = version.id().to_u64(), + "table id not exist yet, wait for registering" + ); + Ok(false) + } else { + Err(HummockError::wait_epoch(format!( + "table {} has been dropped", + self.table_id + ))) + }; + *prev_committed_epoch = committed_epoch; + ret + }, + || { + format!( + "wait_for_epoch: epoch: {}, table_id: {}", + wait_epoch, self.table_id + ) }, ) - .await + .await?; + Ok(()) } pub async fn iter_flushed( diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 235edc884ae5b..d9a006cdafede 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -33,13 +33,13 @@ use risingwave_hummock_sdk::key::{ use risingwave_hummock_sdk::sstable_info::SstableInfo; use tokio::sync::oneshot::{channel, Receiver, Sender}; -use super::{HummockError, SstableStoreRef}; +use super::{HummockError, HummockResult, SstableStoreRef}; use crate::error::StorageResult; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::CachePolicy; use crate::mem_table::{KeyOp, MemTableError}; use crate::monitor::MemoryCollector; -use crate::store::{OpConsistencyLevel, ReadOptions, StateStoreRead, TryWaitEpochOptions}; +use crate::store::{OpConsistencyLevel, ReadOptions, StateStoreRead}; pub fn range_overlap( search_key_range: &R, @@ -575,25 +575,15 @@ pub(crate) fn filter_with_delete_range<'a>( }) } -pub(crate) async fn wait_for_epoch( +pub(crate) async fn wait_for_update( notifier: &tokio::sync::watch::Sender, - wait_epoch: u64, - options: TryWaitEpochOptions, -) -> StorageResult<()> { + mut inspect_fn: impl FnMut(&PinnedVersion) -> HummockResult, + mut periodic_debug_info: impl FnMut() -> String, +) -> HummockResult<()> { let mut receiver = notifier.subscribe(); - let mut committed_epoch = { - // avoid unnecessary check in the loop if the value does not change - let committed_epoch = receiver - .borrow_and_update() - .version() - .table_committed_epoch(options.table_id); - if let Some(committed_epoch) = committed_epoch - && committed_epoch >= wait_epoch - { - return Ok(()); - } - committed_epoch - }; + if inspect_fn(&receiver.borrow_and_update())? { + return Ok(()); + } let start_time = Instant::now(); loop { match tokio::time::timeout(Duration::from_secs(30), receiver.changed()).await { @@ -608,29 +598,19 @@ pub(crate) async fn wait_for_epoch( // CN with the same distribution as the upstream MV. // See #3845 for more details. tracing::warn!( - epoch = wait_epoch, - ?committed_epoch, - table_id = options.table_id.table_id, + info = periodic_debug_info(), elapsed = ?start_time.elapsed(), - "wait_epoch timeout when waiting for version update", + "timeout when waiting for version update", ); continue; } Ok(Err(_)) => { - return Err(HummockError::wait_epoch("tx dropped").into()); + return Err(HummockError::wait_epoch("tx dropped")); } Ok(Ok(_)) => { - // TODO: should handle the corner case of drop table - let new_committed_epoch = receiver - .borrow() - .version() - .table_committed_epoch(options.table_id); - if let Some(committed_epoch) = new_committed_epoch - && committed_epoch >= wait_epoch - { + if inspect_fn(&receiver.borrow())? { return Ok(()); } - committed_epoch = new_committed_epoch; } } } diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index d368542db4a06..f7a8ac745cca6 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -760,7 +760,7 @@ impl StorageTableInner { pub async fn batch_iter_log_with_pk_bounds( &self, start_epoch: u64, - end_epoch: u64, + end_epoch: HummockReadEpoch, ) -> StorageResult> + Send + 'static> { let pk_prefix = OwnedRow::default(); let start_key = self.serialize_pk_bound(&pk_prefix, Unbounded, true); @@ -987,18 +987,22 @@ impl StorageTableInnerIterLogInner { table_key_range: TableKeyRange, read_options: ReadLogOptions, start_epoch: u64, - end_epoch: u64, + end_epoch: HummockReadEpoch, ) -> StorageResult { store .try_wait_epoch( - HummockReadEpoch::Committed(end_epoch), + end_epoch, TryWaitEpochOptions { table_id: read_options.table_id, }, ) .await?; let iter = store - .iter_log((start_epoch, end_epoch), table_key_range, read_options) + .iter_log( + (start_epoch, end_epoch.get_epoch()), + table_key_range, + read_options, + ) .await?; let iter = Self { iter, diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index a3cba25a0572a..89801a3cf4133 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -218,13 +218,12 @@ impl SnapshotBackfillExecutor { // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. - let stream = - upstream_buffer - .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( - barrier_epoch.prev, - barrier_epoch.prev, - )) - .await?; + let stream = upstream_buffer + .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( + barrier_epoch.prev, + HummockReadEpoch::Committed(barrier_epoch.prev), + )) + .await?; let data_types = self.upstream_table.schema().data_types(); let builder = create_builder(None, self.chunk_size, data_types); let stream = read_change_log(stream, builder); From ab3e10da8b00b5d98fc0f85fb00972dd2b54434a Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 25 Sep 2024 16:52:35 +0800 Subject: [PATCH 21/27] refine --- .../src/hummock/store/hummock_storage.rs | 98 ++++++++++--------- .../hummock/store/local_hummock_storage.rs | 42 +------- src/storage/src/hummock/utils.rs | 49 ++++++++++ 3 files changed, 105 insertions(+), 84 deletions(-) diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 434a4ba989d52..da5870376d3b5 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -53,7 +53,7 @@ use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, use crate::hummock::local_version::recent_versions::RecentVersions; use crate::hummock::observer_manager::HummockObserverNode; use crate::hummock::time_travel_version_cache::SimpleTimeTravelVersionCache; -use crate::hummock::utils::wait_for_update; +use crate::hummock::utils::{wait_for_epoch, wait_for_update}; use crate::hummock::write_limiter::{WriteLimiter, WriteLimiterRef}; use crate::hummock::{ HummockEpoch, HummockError, HummockResult, HummockStorageIterator, HummockStorageRevIterator, @@ -633,53 +633,61 @@ impl StateStore for HummockStorage { wait_epoch: HummockReadEpoch, options: TryWaitEpochOptions, ) -> StorageResult<()> { - let (wait_epoch, wait_version) = match wait_epoch { - HummockReadEpoch::Committed(epoch) => (epoch, None), - HummockReadEpoch::BatchQueryCommitted(epoch, version_id) => (epoch, Some(version_id)), - _ => return Ok(()), - }; - assert!(!is_max_epoch(wait_epoch), "epoch should not be MAX EPOCH"); - let is_valid_version_id = |id| match wait_version { - None => true, - Some(version_id) => id >= version_id, - }; - // fast path by checking recent_versions - { - let recent_versions = self.recent_versions.load(); - let latest_version = recent_versions.latest_version().version(); - if is_valid_version_id(latest_version.id) - && let Some(committed_epoch) = - latest_version.table_committed_epoch(options.table_id) - && committed_epoch >= wait_epoch - { - return Ok(()); + match wait_epoch { + HummockReadEpoch::Committed(wait_epoch) => { + assert!(!is_max_epoch(wait_epoch), "epoch should not be MAX EPOCH"); + wait_for_epoch( + &self.version_update_notifier_tx, + wait_epoch, + options.table_id, + ) + .await?; } - } - wait_for_update( - &self.version_update_notifier_tx, - |version| { - if !is_valid_version_id(version.version().id) { - return Ok(false); + HummockReadEpoch::BatchQueryCommitted(wait_epoch, wait_version_id) => { + assert!(!is_max_epoch(wait_epoch), "epoch should not be MAX EPOCH"); + // fast path by checking recent_versions + { + let recent_versions = self.recent_versions.load(); + let latest_version = recent_versions.latest_version().version(); + if latest_version.id >= wait_version_id + && let Some(committed_epoch) = + latest_version.table_committed_epoch(options.table_id) + && committed_epoch >= wait_epoch + { + return Ok(()); + } } - let committed_epoch = version - .version() - .table_committed_epoch(options.table_id) - .ok_or_else(|| { - HummockError::wait_epoch(format!( - "table id {} has been dropped", - options.table_id - )) - })?; - Ok(committed_epoch >= wait_epoch) - }, - || { - format!( - "try_wait_epoch: epoch: {}, version_id: {:?}", - wait_epoch, wait_version + wait_for_update( + &self.version_update_notifier_tx, + |version| { + if wait_version_id < version.id() { + return Ok(false); + } + let committed_epoch = version + .version() + .table_committed_epoch(options.table_id) + .ok_or_else(|| { + // In batch query, since we have ensured that the current version must be after the + // `wait_version_id`, when seeing that the table_id not exist in the latest version, + // the table must have been dropped. + HummockError::wait_epoch(format!( + "table id {} has been dropped", + options.table_id + )) + })?; + Ok(committed_epoch >= wait_epoch) + }, + || { + format!( + "try_wait_epoch: epoch: {}, version_id: {:?}", + wait_epoch, wait_version_id + ) + }, ) - }, - ) - .await?; + .await?; + } + _ => {} + }; Ok(()) } diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 95cee5899cdbf..d0082f21b31f9 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -45,7 +45,7 @@ use crate::hummock::shared_buffer::shared_buffer_batch::{ use crate::hummock::store::version::{read_filter_for_version, HummockVersionReader}; use crate::hummock::utils::{ do_delete_sanity_check, do_insert_sanity_check, do_update_sanity_check, sanity_check_enabled, - wait_for_update, + wait_for_epoch, }; use crate::hummock::write_limiter::WriteLimiterRef; use crate::hummock::{ @@ -135,44 +135,8 @@ impl LocalHummockStorage { .await } - async fn wait_for_epoch(&self, wait_epoch: u64) -> StorageResult<()> { - let mut prev_committed_epoch = None; - let prev_committed_epoch = &mut prev_committed_epoch; - wait_for_update( - &self.version_update_notifier_tx, - |version| { - let committed_epoch = version.version().table_committed_epoch(self.table_id); - let ret = if let Some(committed_epoch) = committed_epoch { - if committed_epoch >= wait_epoch { - Ok(true) - } else { - Ok(false) - } - } else if prev_committed_epoch.is_none() { - warn!( - table_id = self.table_id.table_id, - version_id = version.id().to_u64(), - "table id not exist yet, wait for registering" - ); - Ok(false) - } else { - Err(HummockError::wait_epoch(format!( - "table {} has been dropped", - self.table_id - ))) - }; - *prev_committed_epoch = committed_epoch; - ret - }, - || { - format!( - "wait_for_epoch: epoch: {}, table_id: {}", - wait_epoch, self.table_id - ) - }, - ) - .await?; - Ok(()) + pub async fn wait_for_epoch(&self, wait_epoch: u64) -> StorageResult<()> { + wait_for_epoch(&self.version_update_notifier_tx, wait_epoch, self.table_id).await } pub async fn iter_flushed( diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index d9a006cdafede..983d4c8b2a1dc 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -32,6 +32,7 @@ use risingwave_hummock_sdk::key::{ }; use risingwave_hummock_sdk::sstable_info::SstableInfo; use tokio::sync::oneshot::{channel, Receiver, Sender}; +use tracing::warn; use super::{HummockError, HummockResult, SstableStoreRef}; use crate::error::StorageResult; @@ -575,6 +576,54 @@ pub(crate) fn filter_with_delete_range<'a>( }) } +/// Wait for the `commited_epoch` of `table_id` to reach `wait_epoch`. +/// +/// When the `table_id` does not exist in the latest version, we assume that +/// the table is not created yet, and will wait until the table is created. +pub(crate) async fn wait_for_epoch( + notifier: &tokio::sync::watch::Sender, + wait_epoch: u64, + table_id: TableId, +) -> StorageResult<()> { + let mut prev_committed_epoch = None; + let prev_committed_epoch = &mut prev_committed_epoch; + wait_for_update( + notifier, + |version| { + let committed_epoch = version.version().table_committed_epoch(table_id); + let ret = if let Some(committed_epoch) = committed_epoch { + if committed_epoch >= wait_epoch { + Ok(true) + } else { + Ok(false) + } + } else if prev_committed_epoch.is_none() { + warn!( + table_id = table_id.table_id, + version_id = version.id().to_u64(), + "table id not exist yet, wait for registering" + ); + Ok(false) + } else { + Err(HummockError::wait_epoch(format!( + "table {} has been dropped", + table_id + ))) + }; + *prev_committed_epoch = committed_epoch; + ret + }, + || { + format!( + "wait_for_epoch: epoch: {}, table_id: {}", + wait_epoch, table_id + ) + }, + ) + .await?; + Ok(()) +} + pub(crate) async fn wait_for_update( notifier: &tokio::sync::watch::Sender, mut inspect_fn: impl FnMut(&PinnedVersion) -> HummockResult, From 3b10660edca5621dd43cb496b8d5f4f7710257f0 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 26 Sep 2024 13:47:35 +0800 Subject: [PATCH 22/27] fix --- src/storage/hummock_sdk/src/lib.rs | 9 +++++++++ src/storage/src/table/batch_table/storage_table.rs | 10 ++-------- 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index f1fee55fb1fca..6058d3a16d5e8 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -257,6 +257,15 @@ impl HummockReadEpoch { | HummockReadEpoch::TimeTravel(epoch) => epoch, } } + + pub fn is_read_committed(&self) -> bool { + match self { + HummockReadEpoch::Committed(_) + | HummockReadEpoch::TimeTravel(_) + | HummockReadEpoch::BatchQueryCommitted(_, _) => true, + HummockReadEpoch::NoWait(_) | HummockReadEpoch::Backup(_) => false, + } + } } pub struct SstObjectIdRange { // inclusive diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index f7a8ac745cca6..b5049c7ba1cda 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -362,10 +362,7 @@ impl StorageTableInner { ) -> StorageResult> { let epoch = wait_epoch.get_epoch(); let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); - let read_committed = matches!( - wait_epoch, - HummockReadEpoch::TimeTravel(_) | HummockReadEpoch::Committed(_) - ); + let read_committed = wait_epoch.is_read_committed(); self.store .try_wait_epoch( wait_epoch, @@ -498,10 +495,7 @@ impl StorageTableInner { let iterators: Vec<_> = try_join_all(table_key_ranges.map(|table_key_range| { let prefix_hint = prefix_hint.clone(); let read_backup = matches!(wait_epoch, HummockReadEpoch::Backup(_)); - let read_committed = matches!( - wait_epoch, - HummockReadEpoch::TimeTravel(_) | HummockReadEpoch::Committed(_) - ); + let read_committed = wait_epoch.is_read_committed(); async move { let read_options = ReadOptions { prefix_hint, From 49c7cdeb51f5d2fc31a24f50288c3b70d6cc9672 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 26 Sep 2024 13:49:39 +0800 Subject: [PATCH 23/27] fix typo --- src/storage/src/hummock/utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 983d4c8b2a1dc..f8d67683f49c9 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -576,7 +576,7 @@ pub(crate) fn filter_with_delete_range<'a>( }) } -/// Wait for the `commited_epoch` of `table_id` to reach `wait_epoch`. +/// Wait for the `committed_epoch` of `table_id` to reach `wait_epoch`. /// /// When the `table_id` does not exist in the latest version, we assume that /// the table is not created yet, and will wait until the table is created. From 02b068257543eeb02713ad5a90b210eeda7f747b Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 26 Sep 2024 14:41:06 +0800 Subject: [PATCH 24/27] fix --- src/storage/src/hummock/store/hummock_storage.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index da5870376d3b5..aa328f883e1c1 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -660,7 +660,7 @@ impl StateStore for HummockStorage { wait_for_update( &self.version_update_notifier_tx, |version| { - if wait_version_id < version.id() { + if wait_version_id > version.id() { return Ok(false); } let committed_epoch = version From c74a62ac80d5e3838081d07e06fd4d9b83cce0e2 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 26 Sep 2024 18:42:11 +0800 Subject: [PATCH 25/27] fix borrow and update --- src/storage/src/hummock/utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index f8d67683f49c9..a1283adde7624 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -657,7 +657,7 @@ pub(crate) async fn wait_for_update( return Err(HummockError::wait_epoch("tx dropped")); } Ok(Ok(_)) => { - if inspect_fn(&receiver.borrow())? { + if inspect_fn(&receiver.borrow_and_update())? { return Ok(()); } } From eba2456af670ec72018b892ba33ec221fdb55649 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 26 Sep 2024 19:16:26 +0800 Subject: [PATCH 26/27] add log --- src/storage/src/hummock/utils.rs | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index a1283adde7624..6cdd7d73cec95 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -586,10 +586,19 @@ pub(crate) async fn wait_for_epoch( table_id: TableId, ) -> StorageResult<()> { let mut prev_committed_epoch = None; + let mut call_count = 0; + let mut prev_call_time: Option = None; let prev_committed_epoch = &mut prev_committed_epoch; + let call_count = &mut call_count; + let prev_call_time = &mut prev_call_time; wait_for_update( notifier, |version| { + *call_count += 1; + if *call_count > 1 { + warn!(call_count, prev_elapsed = ?prev_call_time.unwrap().elapsed(), version_id = version.id().to_u64(),"call inspect again"); + } + *prev_call_time = Some(Instant::now()); let committed_epoch = version.version().table_committed_epoch(table_id); let ret = if let Some(committed_epoch) = committed_epoch { if committed_epoch >= wait_epoch { From cb0ccd2bf4d484297448cc098551482c4d1dfafd Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 27 Sep 2024 12:23:23 +0800 Subject: [PATCH 27/27] remove log --- src/storage/src/hummock/utils.rs | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 6cdd7d73cec95..87c40992ea513 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -32,7 +32,6 @@ use risingwave_hummock_sdk::key::{ }; use risingwave_hummock_sdk::sstable_info::SstableInfo; use tokio::sync::oneshot::{channel, Receiver, Sender}; -use tracing::warn; use super::{HummockError, HummockResult, SstableStoreRef}; use crate::error::StorageResult; @@ -586,19 +585,10 @@ pub(crate) async fn wait_for_epoch( table_id: TableId, ) -> StorageResult<()> { let mut prev_committed_epoch = None; - let mut call_count = 0; - let mut prev_call_time: Option = None; let prev_committed_epoch = &mut prev_committed_epoch; - let call_count = &mut call_count; - let prev_call_time = &mut prev_call_time; wait_for_update( notifier, |version| { - *call_count += 1; - if *call_count > 1 { - warn!(call_count, prev_elapsed = ?prev_call_time.unwrap().elapsed(), version_id = version.id().to_u64(),"call inspect again"); - } - *prev_call_time = Some(Instant::now()); let committed_epoch = version.version().table_committed_epoch(table_id); let ret = if let Some(committed_epoch) = committed_epoch { if committed_epoch >= wait_epoch { @@ -607,11 +597,6 @@ pub(crate) async fn wait_for_epoch( Ok(false) } } else if prev_committed_epoch.is_none() { - warn!( - table_id = table_id.table_id, - version_id = version.id().to_u64(), - "table id not exist yet, wait for registering" - ); Ok(false) } else { Err(HummockError::wait_epoch(format!(