From 5468379f1223d09b8de88664c0d5ad559165d796 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 10 Jul 2024 10:52:47 +0800 Subject: [PATCH 01/13] store table snapshot in txn manager --- Cargo.lock | 1 + src/query/storages/common/txn/Cargo.toml | 1 + src/query/storages/common/txn/src/manager.rs | 22 +++++++++++++++++++ .../processors/multi_table_insert_commit.rs | 18 +++++++++++---- .../common/processors/sink_commit.rs | 13 ++++++++--- 5 files changed, 48 insertions(+), 7 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 8aa2b226b2fb2..a115e745cf19c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5433,6 +5433,7 @@ version = "0.1.0" dependencies = [ "databend-common-meta-app", "databend-common-meta-types", + "databend-storages-common-table-meta", "parking_lot 0.12.1", "serde", "uuid", diff --git a/src/query/storages/common/txn/Cargo.toml b/src/query/storages/common/txn/Cargo.toml index 85177ff5bdadc..c712f9bfbfb7e 100644 --- a/src/query/storages/common/txn/Cargo.toml +++ b/src/query/storages/common/txn/Cargo.toml @@ -9,6 +9,7 @@ edition = { workspace = true } [dependencies] databend-common-meta-app = { workspace = true } databend-common-meta-types = { workspace = true } +databend-storages-common-table-meta = { workspace = true } parking_lot = { workspace = true } serde = { version = "1.0.194", features = ["derive"] } uuid = { workspace = true } diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 962ebfcc37d33..60d0ebb5536c9 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -26,6 +26,7 @@ use databend_common_meta_app::schema::UpdateTableMetaReq; use databend_common_meta_app::schema::UpsertTableCopiedFileReq; use databend_common_meta_app::tenant::Tenant; use databend_common_meta_types::MatchSeq; +use databend_storages_common_table_meta::meta::TableSnapshot; use parking_lot::Mutex; use serde::Deserialize; use serde::Serialize; @@ -58,6 +59,9 @@ pub struct TxnBuffer { stream_tables: HashMap, need_purge_files: Vec<(StageInfo, Vec)>, + + // table_id -> latest snapshot + snapshots: HashMap, } #[derive(Debug, Clone)] @@ -74,6 +78,7 @@ impl TxnBuffer { self.update_stream_meta.clear(); self.deduplicated_labels.clear(); self.stream_tables.clear(); + self.snapshots.clear(); } fn update_multi_table_meta(&mut self, mut req: UpdateMultiTableMetaReq) { @@ -104,6 +109,19 @@ impl TxnBuffer { .or_insert(stream_meta.clone()); } } + + fn upsert_table_snapshot(&mut self, table_id: u64, mut snapshot: TableSnapshot) { + match self.snapshots.get_mut(&table_id) { + Some(previous) => { + snapshot.prev_snapshot_id = previous.prev_snapshot_id; + assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + *previous = snapshot; + } + None => { + self.snapshots.insert(table_id, snapshot); + } + } + } } impl TxnManager { @@ -281,4 +299,8 @@ impl TxnManager { pub fn need_purge_files(&mut self) -> Vec<(StageInfo, Vec)> { std::mem::take(&mut self.txn_buffer.need_purge_files) } + + pub fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: TableSnapshot) { + self.txn_buffer.upsert_table_snapshot(table_id, snapshot); + } } diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index 74c7cda756446..08078e6a04c8c 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -90,7 +90,8 @@ impl AsyncSink for CommitMultiTableInsert { snapshot_generator.set_conflict_resolve_context(commit_meta.conflict_resolve_context); let table = self.tables.get(&table_id).unwrap(); update_table_metas.push(( - build_update_table_meta_req(table.as_ref(), &snapshot_generator).await?, + build_update_table_meta_req(table.as_ref(), &snapshot_generator, self.ctx.as_ref()) + .await?, table.get_table_info().clone(), )); snapshot_generators.insert(table_id, snapshot_generator); @@ -173,6 +174,7 @@ impl AsyncSink for CommitMultiTableInsert { *req = build_update_table_meta_req( table.as_ref(), snapshot_generators.get(&tid).unwrap(), + self.ctx.as_ref(), ) .await?; break; @@ -227,22 +229,30 @@ impl AsyncSink for CommitMultiTableInsert { async fn build_update_table_meta_req( table: &dyn Table, snapshot_generator: &AppendGenerator, + ctx: &dyn TableContext, ) -> Result { let fuse_table = FuseTable::try_from_table(table)?; let previous = fuse_table.read_table_snapshot().await?; let snapshot = snapshot_generator.generate_new_snapshot( table.schema().as_ref().clone(), fuse_table.cluster_key_meta.clone(), - previous, + previous.clone(), Some(fuse_table.table_info.ident.seq), )?; // write snapshot - let dal = fuse_table.get_operator(); let location_generator = &fuse_table.meta_location_generator; let location = location_generator .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; - dal.write(&location, snapshot.to_bytes()?).await?; + let is_active = ctx.txn_mgr().lock().is_active(); + if is_active { + ctx.txn_mgr() + .lock() + .upsert_table_snapshot(fuse_table.get_id(), snapshot.clone()); + } else { + let dal = fuse_table.get_operator(); + dal.write(&location, snapshot.to_bytes()?).await?; + } // build new table meta let new_table_meta = diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 4def9bbf6ff38..1ed290b521f97 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -284,7 +284,7 @@ where F: SnapshotGenerator + Send + 'static match self.snapshot_gen.generate_new_snapshot( schema, cluster_key_meta, - previous, + previous.clone(), Some(table_info.ident.seq), ) { Ok(snapshot) => { @@ -358,8 +358,15 @@ where F: SnapshotGenerator + Send + 'static let location = self .location_gen .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; - - self.dal.write(&location, data).await?; + let is_active = self.ctx.txn_mgr().lock().is_active(); + if is_active { + self.ctx + .txn_mgr() + .lock() + .upsert_table_snapshot(table_info.ident.table_id, snapshot.clone()); + } else { + self.dal.write(&location, data).await?; + } let catalog = self.ctx.get_catalog(table_info.catalog()).await?; match FuseTable::update_table_meta( From 060fe76aab0741b3a042c6f1d828d400765b28ca Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 10 Jul 2024 16:32:24 +0800 Subject: [PATCH 02/13] write snapshot in commit statement --- .../interpreters/interpreter_txn_commit.rs | 27 +++++++++++++++++++ src/query/storages/common/txn/src/manager.rs | 8 ++++++ 2 files changed, 35 insertions(+) diff --git a/src/query/service/src/interpreters/interpreter_txn_commit.rs b/src/query/service/src/interpreters/interpreter_txn_commit.rs index d943f3906d63c..2f78974548799 100644 --- a/src/query/service/src/interpreters/interpreter_txn_commit.rs +++ b/src/query/service/src/interpreters/interpreter_txn_commit.rs @@ -16,6 +16,7 @@ use std::sync::Arc; use databend_common_exception::ErrorCode; use databend_common_exception::Result; +use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::TableContext; use databend_storages_common_txn::TxnManagerRef; use log::error; @@ -58,6 +59,32 @@ impl Interpreter for CommitInterpreter { if is_active { let catalog = self.ctx.get_default_catalog()?; + { + let snapshots = self.ctx.txn_mgr().lock().snapshots(); + for (table_id, snapshot) in snapshots { + let table_info = self + .ctx + .txn_mgr() + .lock() + .get_table_from_buffer_by_id(table_id) + .ok_or_else(|| { + ErrorCode::UnknownTable(format!( + "Unknown table id in txn manager: {}", + table_id + )) + })?; + let table = catalog.get_table_by_info(&table_info)?; + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let dal = fuse_table.get_operator(); + let location = fuse_table.snapshot_loc().await?.ok_or_else(|| { + ErrorCode::Internal(format!( + "Table {} has no snapshot location", + table_info.name + )) + })?; + dal.write(&location, snapshot.to_bytes()?).await?; + } + } let req = self.ctx.txn_mgr().lock().req(); let update_summary = { diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 60d0ebb5536c9..2055b736a1cbd 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -122,6 +122,10 @@ impl TxnBuffer { } } } + + fn snapshots(&mut self) -> HashMap { + std::mem::take(&mut self.snapshots) + } } impl TxnManager { @@ -303,4 +307,8 @@ impl TxnManager { pub fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: TableSnapshot) { self.txn_buffer.upsert_table_snapshot(table_id, snapshot); } + + pub fn snapshots(&mut self) -> HashMap { + self.txn_buffer.snapshots() + } } From a0538cb38ae9d5574bf1352b8df41ab687478a84 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Wed, 10 Jul 2024 21:33:32 +0800 Subject: [PATCH 03/13] add test and fix bug --- src/query/storages/common/txn/src/manager.rs | 29 +++++---- .../storages/fuse/src/operations/commit.rs | 6 +- .../processors/multi_table_insert_commit.rs | 12 +++- .../common/processors/sink_commit.rs | 22 +++++-- .../06_0002_stream_txn_consume.test | 60 +++++++++++++++++++ 5 files changed, 104 insertions(+), 25 deletions(-) diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 2055b736a1cbd..f23ec6d078c96 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -61,7 +61,7 @@ pub struct TxnBuffer { need_purge_files: Vec<(StageInfo, Vec)>, // table_id -> latest snapshot - snapshots: HashMap, + snapshots: HashMap>, } #[derive(Debug, Clone)] @@ -110,20 +110,15 @@ impl TxnBuffer { } } - fn upsert_table_snapshot(&mut self, table_id: u64, mut snapshot: TableSnapshot) { - match self.snapshots.get_mut(&table_id) { - Some(previous) => { - snapshot.prev_snapshot_id = previous.prev_snapshot_id; - assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); - *previous = snapshot; - } - None => { - self.snapshots.insert(table_id, snapshot); - } - } + fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: Arc) { + self.snapshots.insert(table_id, snapshot); } - fn snapshots(&mut self) -> HashMap { + fn get_table_snapshot_by_id(&self, table_id: u64) -> Option<&TableSnapshot> { + self.snapshots.get(&table_id).map(|s| s.as_ref()) + } + + fn snapshots(&mut self) -> HashMap> { std::mem::take(&mut self.snapshots) } } @@ -304,11 +299,15 @@ impl TxnManager { std::mem::take(&mut self.txn_buffer.need_purge_files) } - pub fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: TableSnapshot) { + pub fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: Arc) { self.txn_buffer.upsert_table_snapshot(table_id, snapshot); } - pub fn snapshots(&mut self) -> HashMap { + pub fn snapshots(&mut self) -> HashMap> { self.txn_buffer.snapshots() } + + pub fn get_table_snapshot_by_id(&self, table_id: u64) -> Option<&TableSnapshot> { + self.txn_buffer.get_table_snapshot_by_id(table_id) + } } diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index 2baf524b52b6a..983da342183da 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -144,7 +144,7 @@ impl FuseTable { catalog, table_info, location_generator, - snapshot, + Arc::new(snapshot), snapshot_location, copied_files, &[], @@ -202,7 +202,7 @@ impl FuseTable { catalog: Arc, table_info: &TableInfo, location_generator: &TableMetaLocationGenerator, - snapshot: TableSnapshot, + snapshot: Arc, snapshot_location: String, copied_files: &Option, update_stream_meta: &[UpdateStreamMetaReq], @@ -233,7 +233,7 @@ impl FuseTable { .await?; // update_table_meta succeed, populate the snapshot cache item and try keeping a hit file of last snapshot - TableSnapshot::cache().put(snapshot_location.clone(), Arc::new(snapshot)); + TableSnapshot::cache().put(snapshot_location.clone(), snapshot); Self::write_last_snapshot_hint(operator, location_generator, snapshot_location).await; Ok(()) diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index 08078e6a04c8c..2d7f7790be23e 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -233,7 +233,7 @@ async fn build_update_table_meta_req( ) -> Result { let fuse_table = FuseTable::try_from_table(table)?; let previous = fuse_table.read_table_snapshot().await?; - let snapshot = snapshot_generator.generate_new_snapshot( + let mut snapshot = snapshot_generator.generate_new_snapshot( table.schema().as_ref().clone(), fuse_table.cluster_key_meta.clone(), previous.clone(), @@ -246,9 +246,17 @@ async fn build_update_table_meta_req( .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; let is_active = ctx.txn_mgr().lock().is_active(); if is_active { + if let Some(previous) = ctx + .txn_mgr() + .lock() + .get_table_snapshot_by_id(fuse_table.get_id()) + { + snapshot.prev_snapshot_id = previous.prev_snapshot_id; + assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + } ctx.txn_mgr() .lock() - .upsert_table_snapshot(fuse_table.get_id(), snapshot.clone()); + .upsert_table_snapshot(fuse_table.get_id(), Arc::new(snapshot.clone())); } else { let dal = fuse_table.get_operator(); dal.write(&location, snapshot.to_bytes()?).await?; diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 1ed290b521f97..3f0412610450c 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -352,28 +352,40 @@ where F: SnapshotGenerator + Send + 'static } State::TryCommit { data, - snapshot, + mut snapshot, table_info, } => { let location = self .location_gen .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; let is_active = self.ctx.txn_mgr().lock().is_active(); - if is_active { + let snapshot_ref = if is_active { + if let Some(previous) = self + .ctx + .txn_mgr() + .lock() + .get_table_snapshot_by_id(table_info.ident.table_id) + { + snapshot.prev_snapshot_id = previous.prev_snapshot_id; + assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + } + let snapshot_ref = Arc::new(snapshot); self.ctx .txn_mgr() .lock() - .upsert_table_snapshot(table_info.ident.table_id, snapshot.clone()); + .upsert_table_snapshot(table_info.ident.table_id, snapshot_ref.clone()); + snapshot_ref } else { self.dal.write(&location, data).await?; - } + Arc::new(snapshot) + }; let catalog = self.ctx.get_catalog(table_info.catalog()).await?; match FuseTable::update_table_meta( catalog.clone(), &table_info, &self.location_gen, - snapshot, + snapshot_ref, location, &self.copied_files, &self.update_stream_meta, diff --git a/tests/sqllogictests/suites/ee/06_ee_stream/06_0002_stream_txn_consume.test b/tests/sqllogictests/suites/ee/06_ee_stream/06_0002_stream_txn_consume.test index 0fcec811afee2..f0c66e3cef106 100644 --- a/tests/sqllogictests/suites/ee/06_ee_stream/06_0002_stream_txn_consume.test +++ b/tests/sqllogictests/suites/ee/06_ee_stream/06_0002_stream_txn_consume.test @@ -54,30 +54,90 @@ select a from s_append_only; 2 3 +query I +select count(*) from fuse_snapshot('test_txn_stream','t_append_only'); +---- +1 + statement ok BEGIN; statement ok INSERT INTO t_append_only VALUES(4), (5); +query I +select count(*) from fuse_snapshot('test_txn_stream','t_append_only'); +---- +2 + statement ok INSERT INTO t_consume_append_only_1 SELECT a FROM s_append_only; +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_1'); +---- +1 + statement ok INSERT INTO t_consume_append_only_2 SELECT a FROM s_append_only; +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_2'); +---- +1 + statement ok INSERT INTO t_append_only VALUES(6), (7); +query I +select count(*) from fuse_snapshot('test_txn_stream','t_append_only'); +---- +2 + statement ok INSERT INTO t_consume_append_only_3 SELECT a FROM s_append_only; +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_3'); +---- +1 + statement ok INSERT INTO t_consume_append_only_4 SELECT a FROM s_append_only_1; +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_4'); +---- +1 + statement ok COMMIT; +query I +select count(*) from fuse_snapshot('test_txn_stream','t_append_only'); +---- +2 + +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_1'); +---- +1 + +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_2'); +---- +1 + +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_3'); +---- +1 + +query I +select count(*) from fuse_snapshot('test_txn_stream','t_consume_append_only_4'); +---- +1 + statement ok INSERT INTO t_consume_append_only_5 SELECT a FROM s_append_only; From 9beb1616e1ca3c16e035d1341b2d48b3eb236e0a Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 11:26:44 +0800 Subject: [PATCH 04/13] read snapshot from txn manager --- Cargo.lock | 2 + src/query/ee/Cargo.toml | 1 + src/query/ee/src/fail_safe/handler.rs | 4 +- .../fuse/operations/virtual_columns.rs | 2 +- .../tests/it/inverted_index/index_refresh.rs | 5 +- .../ee/tests/it/inverted_index/pruning.rs | 3 +- .../fuse/operations/virtual_columns.rs | 4 +- .../src/interpreters/interpreter_delete.rs | 2 +- .../interpreters/interpreter_merge_into.rs | 15 +++--- .../src/interpreters/interpreter_replace.rs | 15 +++--- .../interpreter_table_add_column.rs | 5 +- .../interpreters/interpreter_table_analyze.rs | 4 +- .../interpreter_table_drop_column.rs | 2 +- .../interpreter_table_modify_column.rs | 2 +- src/query/service/src/test_kits/check.rs | 3 +- src/query/service/src/test_kits/fuse.rs | 6 ++- src/query/service/tests/it/sql/exec/mod.rs | 7 +-- .../it/storages/fuse/operations/analyze.rs | 2 +- .../tests/it/storages/fuse/operations/gc.rs | 10 +++- .../mutation/segments_compact_mutator.rs | 2 +- .../storages/fuse/operations/table_analyze.rs | 15 ++++-- src/query/storages/common/txn/src/manager.rs | 6 +-- src/query/storages/fuse/Cargo.toml | 1 + src/query/storages/fuse/src/fuse_table.rs | 46 +++++++++++++------ .../storages/fuse/src/operations/analyze.rs | 2 +- .../storages/fuse/src/operations/changes.rs | 9 ++-- .../storages/fuse/src/operations/commit.rs | 2 +- .../processors/multi_table_insert_commit.rs | 2 +- .../common/processors/sink_commit.rs | 4 +- .../storages/fuse/src/operations/compact.rs | 10 ++-- .../fuse/src/operations/inverted_index.rs | 2 +- .../storages/fuse/src/operations/navigate.rs | 2 +- .../src/operations/read/fuse_rows_fetcher.rs | 4 ++ .../operations/read/native_rows_fetcher.rs | 6 ++- .../operations/read/parquet_rows_fetcher.rs | 6 ++- .../fuse/src/operations/read_partitions.rs | 2 +- .../storages/fuse/src/operations/recluster.rs | 2 +- .../storages/fuse/src/operations/truncate.rs | 2 +- .../storages/fuse/src/operations/update.rs | 2 +- .../clustering_information.rs | 2 +- .../table_functions/fuse_blocks/fuse_block.rs | 2 +- .../fuse_columns/fuse_column.rs | 2 +- .../fuse_encodings/fuse_encoding.rs | 2 +- .../fuse_segments/fuse_segment.rs | 2 +- .../fuse_snapshots/fuse_snapshot.rs | 2 +- .../fuse_statistics/fuse_statistic.rs | 2 +- src/query/storages/stream/src/stream_table.rs | 3 +- 47 files changed, 155 insertions(+), 83 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index a115e745cf19c..7e5faecca78f1 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4293,6 +4293,7 @@ dependencies = [ "databend-storages-common-io", "databend-storages-common-pruner", "databend-storages-common-table-meta", + "databend-storages-common-txn", "enum-as-inner 0.5.1", "futures", "futures-util", @@ -4905,6 +4906,7 @@ dependencies = [ "databend-storages-common-cache", "databend-storages-common-pruner", "databend-storages-common-table-meta", + "databend-storages-common-txn", "derive-visitor", "futures", "futures-util", diff --git a/src/query/ee/Cargo.toml b/src/query/ee/Cargo.toml index 503dd52d2db53..68e20ad0ca322 100644 --- a/src/query/ee/Cargo.toml +++ b/src/query/ee/Cargo.toml @@ -52,6 +52,7 @@ databend-query = { workspace = true } databend-storages-common-cache = { workspace = true } databend-storages-common-pruner = { workspace = true } databend-storages-common-table-meta = { workspace = true } +databend-storages-common-txn = { workspace = true } derive-visitor = { workspace = true } futures = { workspace = true } futures-util = { workspace = true } diff --git a/src/query/ee/src/fail_safe/handler.rs b/src/query/ee/src/fail_safe/handler.rs index a87ca5dd2f6f0..e62a4733b887f 100644 --- a/src/query/ee/src/fail_safe/handler.rs +++ b/src/query/ee/src/fail_safe/handler.rs @@ -38,11 +38,11 @@ use databend_enterprise_fail_safe::FailSafeHandlerWrapper; use databend_storages_common_cache::LoadParams; use databend_storages_common_table_meta::meta::CompactSegmentInfo; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_txn::TxnManager; use log::info; use log::warn; use opendal::ErrorKind; use opendal::Operator; - pub struct RealFailSafeHandler {} impl RealFailSafeHandler {} @@ -154,7 +154,7 @@ impl Amender { if e.code() == ErrorCode::STORAGE_NOT_FOUND { let snapshot_location = table.snapshot_loc().await?.unwrap(); self.recover_object(&snapshot_location).await?; - let snapshot = table.read_table_snapshot().await?; + let snapshot = table.read_table_snapshot(TxnManager::init()).await?; let schema = table.schema(); let operator = table.get_operator(); if let Some(snapshot) = snapshot { diff --git a/src/query/ee/src/storages/fuse/operations/virtual_columns.rs b/src/query/ee/src/storages/fuse/operations/virtual_columns.rs index a182e1759430c..94b24c09e0914 100644 --- a/src/query/ee/src/storages/fuse/operations/virtual_columns.rs +++ b/src/query/ee/src/storages/fuse/operations/virtual_columns.rs @@ -53,7 +53,7 @@ pub async fn do_refresh_virtual_column( return Ok(()); } - let snapshot_opt = fuse_table.read_table_snapshot().await?; + let snapshot_opt = fuse_table.read_table_snapshot(ctx.txn_mgr()).await?; let snapshot = if let Some(val) = snapshot_opt { val } else { diff --git a/src/query/ee/tests/it/inverted_index/index_refresh.rs b/src/query/ee/tests/it/inverted_index/index_refresh.rs index 630806df74f58..821b2ec4f178f 100644 --- a/src/query/ee/tests/it/inverted_index/index_refresh.rs +++ b/src/query/ee/tests/it/inverted_index/index_refresh.rs @@ -33,6 +33,7 @@ use databend_query::interpreters::RefreshTableIndexInterpreter; use databend_query::test_kits::append_string_sample_data; use databend_query::test_kits::*; use databend_storages_common_cache::LoadParams; +use databend_storages_common_txn::TxnManager; use tantivy::schema::Field; use tantivy::tokenizer::LowerCaser; use tantivy::tokenizer::SimpleTokenizer; @@ -95,7 +96,9 @@ async fn test_fuse_do_refresh_inverted_index() -> Result<()> { let table_schema = new_fuse_table.schema(); // get index location from new table snapshot - let new_snapshot = new_fuse_table.read_table_snapshot().await?; + let new_snapshot = new_fuse_table + .read_table_snapshot(TxnManager::init()) + .await?; assert!(new_snapshot.is_some()); let new_snapshot = new_snapshot.unwrap(); diff --git a/src/query/ee/tests/it/inverted_index/pruning.rs b/src/query/ee/tests/it/inverted_index/pruning.rs index 03d7045d33bb1..d4760668d9665 100644 --- a/src/query/ee/tests/it/inverted_index/pruning.rs +++ b/src/query/ee/tests/it/inverted_index/pruning.rs @@ -55,6 +55,7 @@ use databend_storages_common_pruner::BlockMetaIndex; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::OPT_KEY_DATABASE_ID; +use databend_storages_common_txn::TxnManager; use opendal::Operator; async fn apply_block_pruning( @@ -531,7 +532,7 @@ async fn test_block_pruner() -> Result<()> { let new_table = table.refresh(ctx.as_ref()).await?; let fuse_table = FuseTable::do_create(new_table.get_table_info().clone())?; - let snapshot = fuse_table.read_table_snapshot().await?; + let snapshot = fuse_table.read_table_snapshot(TxnManager::init()).await?; assert!(snapshot.is_some()); let snapshot = snapshot.unwrap(); diff --git a/src/query/ee/tests/it/storages/fuse/operations/virtual_columns.rs b/src/query/ee/tests/it/storages/fuse/operations/virtual_columns.rs index fe997b45f41cd..708a0acfb3f74 100644 --- a/src/query/ee/tests/it/storages/fuse/operations/virtual_columns.rs +++ b/src/query/ee/tests/it/storages/fuse/operations/virtual_columns.rs @@ -19,10 +19,10 @@ use databend_common_storages_fuse::io::MetaReaders; use databend_common_storages_fuse::io::TableMetaLocationGenerator; use databend_common_storages_fuse::FuseStorageFormat; use databend_common_storages_fuse::FuseTable; +use databend_common_storages_fuse::TableContext; use databend_enterprise_query::storages::fuse::operations::virtual_columns::do_refresh_virtual_column; use databend_query::test_kits::*; use databend_storages_common_cache::LoadParams; - #[tokio::test(flavor = "multi_thread")] async fn test_fuse_do_refresh_virtual_column() -> Result<()> { let fixture = TestFixture::setup().await?; @@ -45,7 +45,7 @@ async fn test_fuse_do_refresh_virtual_column() -> Result<()> { let virtual_columns = vec!["v['a']".to_string(), "v[0]".to_string()]; let table_ctx = fixture.new_query_ctx().await?; - let snapshot_opt = fuse_table.read_table_snapshot().await?; + let snapshot_opt = fuse_table.read_table_snapshot(table_ctx.txn_mgr()).await?; let snapshot = snapshot_opt.unwrap(); let projection = Projection::Columns(vec![]); diff --git a/src/query/service/src/interpreters/interpreter_delete.rs b/src/query/service/src/interpreters/interpreter_delete.rs index 56df3f5fa0264..07b4a64892094 100644 --- a/src/query/service/src/interpreters/interpreter_delete.rs +++ b/src/query/service/src/interpreters/interpreter_delete.rs @@ -186,7 +186,7 @@ impl Interpreter for DeleteInterpreter { let mut build_res = PipelineBuildResult::create(); // check if table is empty - let Some(snapshot) = fuse_table.read_table_snapshot().await? else { + let Some(snapshot) = fuse_table.read_table_snapshot(self.ctx.txn_mgr()).await? else { // no snapshot, no deletion return Ok(build_res); }; diff --git a/src/query/service/src/interpreters/interpreter_merge_into.rs b/src/query/service/src/interpreters/interpreter_merge_into.rs index 882029b421db0..f2bb9f35a484c 100644 --- a/src/query/service/src/interpreters/interpreter_merge_into.rs +++ b/src/query/service/src/interpreters/interpreter_merge_into.rs @@ -137,12 +137,15 @@ impl MergeIntoInterpreter { // Prepare MergeIntoBuildInfo for PhysicalPlanBuilder to build MergeInto physical plan. let table_info = fuse_table.get_table_info(); - let table_snapshot = fuse_table.read_table_snapshot().await?.unwrap_or_else(|| { - Arc::new(TableSnapshot::new_empty_snapshot( - fuse_table.schema().as_ref().clone(), - Some(table_info.ident.seq), - )) - }); + let table_snapshot = fuse_table + .read_table_snapshot(self.ctx.txn_mgr()) + .await? + .unwrap_or_else(|| { + Arc::new(TableSnapshot::new_empty_snapshot( + fuse_table.schema().as_ref().clone(), + Some(table_info.ident.seq), + )) + }); let update_stream_meta = dml_build_update_stream_req(self.ctx.clone(), &merge_into.meta_data).await?; let merge_into_build_info = MergeIntoBuildInfo { diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 93d442fb1eb18..99ba601b89ba5 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -164,12 +164,15 @@ impl ReplaceInterpreter { })?; let table_info = fuse_table.get_table_info(); - let base_snapshot = fuse_table.read_table_snapshot().await?.unwrap_or_else(|| { - Arc::new(TableSnapshot::new_empty_snapshot( - schema.as_ref().clone(), - Some(table_info.ident.seq), - )) - }); + let base_snapshot = fuse_table + .read_table_snapshot(self.ctx.txn_mgr()) + .await? + .unwrap_or_else(|| { + Arc::new(TableSnapshot::new_empty_snapshot( + schema.as_ref().clone(), + Some(table_info.ident.seq), + )) + }); let is_multi_node = !self.ctx.get_cluster().is_empty(); let is_value_source = matches!(self.plan.source, InsertInputSource::Values(_)); diff --git a/src/query/service/src/interpreters/interpreter_table_add_column.rs b/src/query/service/src/interpreters/interpreter_table_add_column.rs index ba9aed2f38857..c675b45ed7e4a 100644 --- a/src/query/service/src/interpreters/interpreter_table_add_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_add_column.rs @@ -120,7 +120,7 @@ impl Interpreter for AddTableColumnInterpreter { let table_id = table_info.ident.table_id; let table_version = table_info.ident.seq; - generate_new_snapshot(table.as_ref(), &mut new_table_meta).await?; + generate_new_snapshot(table.as_ref(), &mut new_table_meta, self.ctx.as_ref()).await?; let req = UpdateTableMetaReq { table_id, @@ -140,9 +140,10 @@ impl Interpreter for AddTableColumnInterpreter { pub(crate) async fn generate_new_snapshot( table: &dyn Table, new_table_meta: &mut TableMeta, + ctx: &dyn TableContext, ) -> Result<()> { if let Ok(fuse_table) = FuseTable::try_from_table(table) { - if let Some(snapshot) = fuse_table.read_table_snapshot().await? { + if let Some(snapshot) = fuse_table.read_table_snapshot(ctx.txn_mgr()).await? { let mut new_snapshot = TableSnapshot::from_previous( snapshot.as_ref(), Some(fuse_table.get_table_info().ident.seq), diff --git a/src/query/service/src/interpreters/interpreter_table_analyze.rs b/src/query/service/src/interpreters/interpreter_table_analyze.rs index 5cac4bb7614eb..7499afae961e1 100644 --- a/src/query/service/src/interpreters/interpreter_table_analyze.rs +++ b/src/query/service/src/interpreters/interpreter_table_analyze.rs @@ -72,7 +72,7 @@ impl Interpreter for AnalyzeTableInterpreter { Err(_) => return Ok(PipelineBuildResult::create()), }; - let r = table.read_table_snapshot().await; + let r = table.read_table_snapshot(self.ctx.txn_mgr()).await; let snapshot_opt = match r { Err(e) => return Err(e), Ok(v) => v, @@ -98,7 +98,7 @@ impl Interpreter for AnalyzeTableInterpreter { .await { Ok(t) => !t - .read_table_snapshot() + .read_table_snapshot(self.ctx.txn_mgr()) .await .is_ok_and(|s| s.is_some_and(|s| s.prev_table_seq.is_some())), Err(_) => true, diff --git a/src/query/service/src/interpreters/interpreter_table_drop_column.rs b/src/query/service/src/interpreters/interpreter_table_drop_column.rs index e07c24cc19acd..b4587e230e1d7 100644 --- a/src/query/service/src/interpreters/interpreter_table_drop_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_drop_column.rs @@ -130,7 +130,7 @@ impl Interpreter for DropTableColumnInterpreter { let table_id = table_info.ident.table_id; let table_version = table_info.ident.seq; - generate_new_snapshot(table.as_ref(), &mut new_table_meta).await?; + generate_new_snapshot(table.as_ref(), &mut new_table_meta, self.ctx.as_ref()).await?; let req = UpdateTableMetaReq { table_id, diff --git a/src/query/service/src/interpreters/interpreter_table_modify_column.rs b/src/query/service/src/interpreters/interpreter_table_modify_column.rs index 65172d934e943..563daf0708d5b 100644 --- a/src/query/service/src/interpreters/interpreter_table_modify_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_modify_column.rs @@ -164,7 +164,7 @@ impl ModifyTableColumnInterpreter { let fuse_table = FuseTable::try_from_table(table.as_ref())?; let prev_snapshot_id = fuse_table - .read_table_snapshot() + .read_table_snapshot(self.ctx.txn_mgr()) .await .map_or(None, |v| v.map(|snapshot| snapshot.snapshot_id)); diff --git a/src/query/service/src/test_kits/check.rs b/src/query/service/src/test_kits/check.rs index ce9c6b934f717..eab99e3ab0a8a 100644 --- a/src/query/service/src/test_kits/check.rs +++ b/src/query/service/src/test_kits/check.rs @@ -27,6 +27,7 @@ use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_PREFIX; use databend_common_storages_fuse::FUSE_TBL_SNAPSHOT_STATISTICS_PREFIX; use databend_common_storages_fuse::FUSE_TBL_XOR_BLOOM_INDEX_PREFIX; +use databend_storages_common_txn::TxnManager; use futures::TryStreamExt; use walkdir::WalkDir; @@ -174,7 +175,7 @@ pub async fn check_data_dir( if check_table_statistic_file.is_some() { let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let snapshot_opt = fuse_table.read_table_snapshot().await?; + let snapshot_opt = fuse_table.read_table_snapshot(TxnManager::init()).await?; assert!(snapshot_opt.is_some()); let snapshot = snapshot_opt.unwrap(); let ts_location_opt = snapshot.table_statistics_location.clone(); diff --git a/src/query/service/src/test_kits/fuse.rs b/src/query/service/src/test_kits/fuse.rs index 4bbc4279e6ebe..68700b51274c1 100644 --- a/src/query/service/src/test_kits/fuse.rs +++ b/src/query/service/src/test_kits/fuse.rs @@ -44,6 +44,7 @@ use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; +use databend_storages_common_txn::TxnManager; use futures_util::TryStreamExt; use opendal::Operator; use serde::Serialize; @@ -63,7 +64,10 @@ pub async fn generate_snapshot_with_segments( segment_locations: Vec, time_stamp: Option>, ) -> Result { - let current_snapshot = fuse_table.read_table_snapshot().await?.unwrap(); + let current_snapshot = fuse_table + .read_table_snapshot(TxnManager::init()) + .await? + .unwrap(); let operator = fuse_table.get_operator(); let location_gen = fuse_table.meta_location_generator(); let mut new_snapshot = TableSnapshot::from_previous( diff --git a/src/query/service/tests/it/sql/exec/mod.rs b/src/query/service/tests/it/sql/exec/mod.rs index 3c1081f4606d0..22a5691558e58 100644 --- a/src/query/service/tests/it/sql/exec/mod.rs +++ b/src/query/service/tests/it/sql/exec/mod.rs @@ -11,7 +11,6 @@ // 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 databend_common_base::base::tokio; use databend_common_base::runtime::Runtime; use databend_common_base::runtime::TrySpawn; @@ -20,6 +19,7 @@ use databend_common_exception::Result; use databend_common_sql::plans::Plan; use databend_common_sql::Planner; use databend_common_storages_fuse::FuseTable; +use databend_common_storages_fuse::TableContext; use databend_query::interpreters::Interpreter; use databend_query::interpreters::OptimizeTableInterpreter; use databend_query::test_kits::*; @@ -68,6 +68,7 @@ pub async fn test_snapshot_consistency() -> Result<()> { .await?; } + let ctx_clone = ctx.clone(); let query_task = async move { // 2. test compact and select concurrency let query = format!( @@ -112,7 +113,7 @@ pub async fn test_snapshot_consistency() -> Result<()> { )) }) .unwrap(); - let snapshot0 = fuse_table0.read_table_snapshot().await?; + let snapshot0 = fuse_table0.read_table_snapshot(ctx_clone.txn_mgr()).await?; let fuse_table1 = table1 .as_any() @@ -125,7 +126,7 @@ pub async fn test_snapshot_consistency() -> Result<()> { )) }) .unwrap(); - let snapshot1 = fuse_table1.read_table_snapshot().await?; + let snapshot1 = fuse_table1.read_table_snapshot(ctx_clone.txn_mgr()).await?; let res = match (snapshot0, snapshot1) { (None, None) => true, diff --git a/src/query/service/tests/it/storages/fuse/operations/analyze.rs b/src/query/service/tests/it/storages/fuse/operations/analyze.rs index 2a23d26433b5b..37b85eb88b253 100644 --- a/src/query/service/tests/it/storages/fuse/operations/analyze.rs +++ b/src/query/service/tests/it/storages/fuse/operations/analyze.rs @@ -83,7 +83,7 @@ async fn test_fuse_snapshot_analyze_and_truncate() -> Result<()> { .await?; let table = catalog.get_table(&ctx.get_tenant(), &db, &tbl).await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let snapshot_opt = fuse_table.read_table_snapshot().await?; + let snapshot_opt = fuse_table.read_table_snapshot(ctx.txn_mgr()).await?; assert!(snapshot_opt.is_some()); assert!(snapshot_opt.unwrap().table_statistics_location.is_none()); } diff --git a/src/query/service/tests/it/storages/fuse/operations/gc.rs b/src/query/service/tests/it/storages/fuse/operations/gc.rs index 48fa9944d6a46..617ff6ef2d89e 100644 --- a/src/query/service/tests/it/storages/fuse/operations/gc.rs +++ b/src/query/service/tests/it/storages/fuse/operations/gc.rs @@ -83,7 +83,10 @@ async fn test_fuse_purge_normal_orphan_snapshot() -> Result<()> { // create orphan snapshot, its timestamp is larger than the current one { - let current_snapshot = fuse_table.read_table_snapshot().await?.unwrap(); + let current_snapshot = fuse_table + .read_table_snapshot(ctx.txn_mgr()) + .await? + .unwrap(); let operator = fuse_table.get_operator(); let location_gen = fuse_table.meta_location_generator(); let orphan_snapshot_id = Uuid::new_v4(); @@ -190,7 +193,10 @@ async fn test_fuse_purge_orphan_retention() -> Result<()> { // no we have 2 snapshot, 2 segment, 2 blocks let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let base_snapshot = fuse_table.read_table_snapshot().await?.unwrap(); + let base_snapshot = fuse_table + .read_table_snapshot(ctx.txn_mgr()) + .await? + .unwrap(); let base_timestamp = base_snapshot.timestamp.unwrap(); // 2. prepare `seg_2` diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs index d910915cc9cea..41c4f081a2ea0 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/segments_compact_mutator.rs @@ -240,7 +240,7 @@ async fn build_mutator( ctx: Arc, limit: Option, ) -> Result> { - let snapshot_opt = tbl.read_table_snapshot().await?; + let snapshot_opt = tbl.read_table_snapshot(ctx.txn_mgr()).await?; let base_snapshot = if let Some(val) = snapshot_opt { val } else { diff --git a/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs b/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs index 5c8758feb7843..f950b5e20076f 100644 --- a/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs +++ b/src/query/service/tests/it/storages/fuse/operations/table_analyze.rs @@ -125,7 +125,10 @@ async fn test_table_update_analyze_statistics() -> Result<()> { // check summary after update let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let after_update = fuse_table.read_table_snapshot().await?.unwrap(); + let after_update = fuse_table + .read_table_snapshot(ctx.txn_mgr()) + .await? + .unwrap(); let base_summary = after_update.summary.clone(); let id_stats = base_summary.col_stats.get(&0).unwrap(); assert_eq!(id_stats.max(), &Scalar::Number(NumberScalar::Int32(3))); @@ -159,7 +162,10 @@ async fn test_table_update_analyze_statistics() -> Result<()> { // check summary after analyze let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let after_analyze = fuse_table.read_table_snapshot().await?.unwrap(); + let after_analyze = fuse_table + .read_table_snapshot(ctx.txn_mgr()) + .await? + .unwrap(); let last_summary = after_analyze.summary.clone(); let id_stats = last_summary.col_stats.get(&0).unwrap(); assert_eq!(id_stats.max(), &Scalar::Number(NumberScalar::Int32(3))); @@ -204,7 +210,10 @@ async fn test_table_analyze_without_prev_table_seq() -> Result<()> { let operator = fuse_table.get_operator(); // genenrate snapshot without prev_table_seq - let snapshot_0 = fuse_table.read_table_snapshot().await?.unwrap(); + let snapshot_0 = fuse_table + .read_table_snapshot(ctx.txn_mgr()) + .await? + .unwrap(); let snapshot_1 = TableSnapshot::from_previous(&snapshot_0, None); let snapshot_loc_1 = location_gen .snapshot_location_from_uuid(&snapshot_1.snapshot_id, TableSnapshot::VERSION)?; diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index f23ec6d078c96..363389e804606 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -114,8 +114,8 @@ impl TxnBuffer { self.snapshots.insert(table_id, snapshot); } - fn get_table_snapshot_by_id(&self, table_id: u64) -> Option<&TableSnapshot> { - self.snapshots.get(&table_id).map(|s| s.as_ref()) + fn get_table_snapshot_by_id(&self, table_id: u64) -> Option> { + self.snapshots.get(&table_id).cloned() } fn snapshots(&mut self) -> HashMap> { @@ -307,7 +307,7 @@ impl TxnManager { self.txn_buffer.snapshots() } - pub fn get_table_snapshot_by_id(&self, table_id: u64) -> Option<&TableSnapshot> { + pub fn get_table_snapshot_by_id(&self, table_id: u64) -> Option> { self.txn_buffer.get_table_snapshot_by_id(table_id) } } diff --git a/src/query/storages/fuse/Cargo.toml b/src/query/storages/fuse/Cargo.toml index b6d0dfd8d42db..861f57567c14b 100644 --- a/src/query/storages/fuse/Cargo.toml +++ b/src/query/storages/fuse/Cargo.toml @@ -47,6 +47,7 @@ databend-storages-common-index = { workspace = true } databend-storages-common-io = { workspace = true } databend-storages-common-pruner = { workspace = true } databend-storages-common-table-meta = { workspace = true } +databend-storages-common-txn = { workspace = true } enum-as-inner = "0.5" futures = { workspace = true } futures-util = { workspace = true } diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index a53a1cfa33b39..728a8c98d6cfb 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -76,6 +76,8 @@ use databend_storages_common_table_meta::table::OPT_KEY_STORAGE_FORMAT; use databend_storages_common_table_meta::table::OPT_KEY_STORAGE_PREFIX; use databend_storages_common_table_meta::table::OPT_KEY_TABLE_ATTACHED_DATA_URI; use databend_storages_common_table_meta::table::OPT_KEY_TABLE_COMPRESSION; +use databend_storages_common_txn::TxnManager; +use databend_storages_common_txn::TxnManagerRef; use log::error; use log::warn; use opendal::Operator; @@ -101,7 +103,6 @@ use crate::FUSE_OPT_KEY_BLOCK_PER_SEGMENT; use crate::FUSE_OPT_KEY_ROW_PER_BLOCK; use crate::FUSE_OPT_KEY_ROW_PER_PAGE; use crate::FUSE_TBL_LAST_SNAPSHOT_HINT; - #[derive(Clone)] pub struct FuseTable { pub(crate) table_info: TableInfo, @@ -138,7 +139,7 @@ impl FuseTable { if need_refresh_schema { let table = Self::do_create(table_info.as_ref().clone())?; - let snapshot = table.read_table_snapshot().await?; + let snapshot = table.read_table_snapshot(TxnManager::init()).await?; let schema = snapshot .ok_or_else(|| { ErrorCode::ShareStorageError( @@ -323,7 +324,20 @@ impl FuseTable { #[minitrace::trace] #[async_backtrace::framed] - pub async fn read_table_snapshot(&self) -> Result>> { + pub async fn read_table_snapshot( + &self, + txn_mgr: TxnManagerRef, + ) -> Result>> { + { + let guard = txn_mgr.lock(); + if guard.is_active() { + if let Some(snapshot) = + guard.get_table_snapshot_by_id(self.table_info.ident.table_id) + { + return Ok(Some(snapshot.clone())); + } + } + } let reader = MetaReaders::table_snapshot_reader(self.get_operator()); self.read_table_snapshot_with_reader(reader).await } @@ -556,7 +570,7 @@ impl Table for FuseTable { let cluster_key_meta = new_table_meta.cluster_key(); let schema = self.schema().as_ref().clone(); - let prev = self.read_table_snapshot().await?; + let prev = self.read_table_snapshot(ctx.txn_mgr()).await?; let prev_version = self.snapshot_format_version(None).await?; let prev_timestamp = prev.as_ref().and_then(|v| v.timestamp); let prev_snapshot_id = prev.as_ref().map(|v| (v.snapshot_id, prev_version)); @@ -610,7 +624,7 @@ impl Table for FuseTable { let schema = self.schema().as_ref().clone(); - let prev = self.read_table_snapshot().await?; + let prev = self.read_table_snapshot(ctx.txn_mgr()).await?; let prev_version = self.snapshot_format_version(None).await?; let prev_timestamp = prev.as_ref().and_then(|v| v.timestamp); let prev_statistics_location = prev @@ -748,12 +762,15 @@ impl Table for FuseTable { let stats = match self.table_type { FuseTableType::Attached => { - let snapshot = self.read_table_snapshot().await?.ok_or_else(|| { - // For table created with "ATTACH TABLE ... READ_ONLY"statement, this should be unreachable: - // IO or Deserialization related error should have already been thrown, thus - // `Internal` error is used. - ErrorCode::Internal("Failed to load snapshot of read_only attach table") - })?; + let snapshot = self + .read_table_snapshot(ctx.txn_mgr()) + .await? + .ok_or_else(|| { + // For table created with "ATTACH TABLE ... READ_ONLY"statement, this should be unreachable: + // IO or Deserialization related error should have already been thrown, thus + // `Internal` error is used. + ErrorCode::Internal("Failed to load snapshot of read_only attach table") + })?; let summary = &snapshot.summary; TableStatistics { num_rows: Some(summary.row_count), @@ -782,9 +799,9 @@ impl Table for FuseTable { #[async_backtrace::framed] async fn column_statistics_provider( &self, - _ctx: Arc, + ctx: Arc, ) -> Result> { - let provider = if let Some(snapshot) = self.read_table_snapshot().await? { + let provider = if let Some(snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? { let stats = &snapshot.summary.col_stats; let table_statistics = self.read_table_snapshot_statistics(Some(&snapshot)).await?; if let Some(table_statistics) = table_statistics { @@ -843,7 +860,7 @@ impl Table for FuseTable { #[async_backtrace::framed] async fn generage_changes_query( &self, - _ctx: Arc, + ctx: Arc, database_name: &str, table_name: &str, _consume: bool, @@ -867,6 +884,7 @@ impl Table for FuseTable { location, format!("{}.{} {}", database_name, table_name, desc), *seq, + ctx.as_ref(), ) .await } diff --git a/src/query/storages/fuse/src/operations/analyze.rs b/src/query/storages/fuse/src/operations/analyze.rs index 66f100bf39942..022bf04a20e8d 100644 --- a/src/query/storages/fuse/src/operations/analyze.rs +++ b/src/query/storages/fuse/src/operations/analyze.rs @@ -119,7 +119,7 @@ impl SinkAnalyzeState { .await?; let table = FuseTable::try_from_table(table.as_ref())?; - let snapshot = table.read_table_snapshot().await?; + let snapshot = table.read_table_snapshot(self.ctx.txn_mgr()).await?; if snapshot.is_none() { return Ok(true); } diff --git a/src/query/storages/fuse/src/operations/changes.rs b/src/query/storages/fuse/src/operations/changes.rs index 708b009819afd..a8e1d68fdfaa7 100644 --- a/src/query/storages/fuse/src/operations/changes.rs +++ b/src/query/storages/fuse/src/operations/changes.rs @@ -123,6 +123,7 @@ impl FuseTable { base_location: &Option, table_desc: String, seq: u64, + ctx: &dyn TableContext, ) -> Result { let cols = self .schema() @@ -133,7 +134,7 @@ impl FuseTable { let suffix = format!("{:08x}", Utc::now().timestamp()); - let optimized_mode = self.optimize_stream_mode(mode, base_location).await?; + let optimized_mode = self.optimize_stream_mode(mode, base_location, ctx).await?; let query = match optimized_mode { StreamMode::AppendOnly => { let append_alias = format!("_change_append${}", suffix); @@ -213,12 +214,13 @@ impl FuseTable { &self, mode: &StreamMode, base_location: &Option, + ctx: &dyn TableContext, ) -> Result { match mode { StreamMode::AppendOnly => Ok(StreamMode::AppendOnly), StreamMode::Standard => { if let Some(base_location) = base_location { - if let Some(latest_snapshot) = self.read_table_snapshot().await? { + if let Some(latest_snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? { let latest_segments: HashSet<&Location> = HashSet::from_iter(&latest_snapshot.segments); @@ -437,7 +439,8 @@ impl FuseTable { let (base_snapshot, _) = SnapshotsIO::read_snapshot(base_location.clone(), self.get_operator()).await?; let base_summary = base_snapshot.summary.clone(); - let latest_summary = if let Some(snapshot) = self.read_table_snapshot().await? { + let latest_summary = if let Some(snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? + { snapshot.summary.clone() } else { return Ok(None); diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index 983da342183da..45bf5469b8734 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -344,7 +344,7 @@ impl FuseTable { FuseTable::try_from_table(latest_table_ref.as_ref())?; latest_snapshot = latest_fuse_table - .read_table_snapshot() + .read_table_snapshot(ctx.txn_mgr()) .await? .ok_or_else(|| { ErrorCode::Internal( diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index 2d7f7790be23e..5b0b7841b376e 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -232,7 +232,7 @@ async fn build_update_table_meta_req( ctx: &dyn TableContext, ) -> Result { let fuse_table = FuseTable::try_from_table(table)?; - let previous = fuse_table.read_table_snapshot().await?; + let previous = fuse_table.read_table_snapshot(ctx.txn_mgr()).await?; let mut snapshot = snapshot_generator.generate_new_snapshot( table.schema().as_ref().clone(), fuse_table.cluster_key_meta.clone(), diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 3f0412610450c..ec04953581757 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -323,7 +323,7 @@ where F: SnapshotGenerator + Send + 'static let schema = self.table.schema().as_ref().clone(); let fuse_table = FuseTable::try_from_table(self.table.as_ref())?.to_owned(); - let previous = fuse_table.read_table_snapshot().await?; + let previous = fuse_table.read_table_snapshot(self.ctx.txn_mgr()).await?; // save current table info when commit to meta server // if table_id not match, update table meta will fail let table_info = fuse_table.table_info.clone(); @@ -512,7 +512,7 @@ where F: SnapshotGenerator + Send + 'static State::RefreshTable => { self.table = self.table.refresh(self.ctx.as_ref()).await?; let fuse_table = FuseTable::try_from_table(self.table.as_ref())?.to_owned(); - let previous = fuse_table.read_table_snapshot().await?; + let previous = fuse_table.read_table_snapshot(self.ctx.txn_mgr()).await?; let cluster_key_meta = fuse_table.cluster_key_meta.clone(); self.state = State::GenerateSnapshot { previous, diff --git a/src/query/storages/fuse/src/operations/compact.rs b/src/query/storages/fuse/src/operations/compact.rs index c97a688c9996a..d9d5832832600 100644 --- a/src/query/storages/fuse/src/operations/compact.rs +++ b/src/query/storages/fuse/src/operations/compact.rs @@ -59,7 +59,7 @@ impl FuseTable { num_segment_limit: Option, ) -> Result<()> { let compact_options = if let Some(v) = self - .compact_options_with_segment_limit(num_segment_limit) + .compact_options_with_segment_limit(num_segment_limit, ctx.as_ref()) .await? { v @@ -89,7 +89,7 @@ impl FuseTable { limits: CompactionLimits, ) -> Result)>> { let compact_options = if let Some(v) = self - .compact_options(limits.segment_limit, limits.block_limit) + .compact_options(limits.segment_limit, limits.block_limit, ctx.as_ref()) .await? { v @@ -230,8 +230,9 @@ impl FuseTable { async fn compact_options_with_segment_limit( &self, num_segment_limit: Option, + ctx: &dyn TableContext, ) -> Result> { - self.compact_options(num_segment_limit, None).await + self.compact_options(num_segment_limit, None, ctx).await } #[async_backtrace::framed] @@ -239,8 +240,9 @@ impl FuseTable { &self, num_segment_limit: Option, num_block_limit: Option, + ctx: &dyn TableContext, ) -> Result> { - let snapshot_opt = self.read_table_snapshot().await?; + let snapshot_opt = self.read_table_snapshot(ctx.txn_mgr()).await?; let base_snapshot = if let Some(val) = snapshot_opt { val } else { diff --git a/src/query/storages/fuse/src/operations/inverted_index.rs b/src/query/storages/fuse/src/operations/inverted_index.rs index b6c82f256bbbb..9a8104b68f367 100644 --- a/src/query/storages/fuse/src/operations/inverted_index.rs +++ b/src/query/storages/fuse/src/operations/inverted_index.rs @@ -83,7 +83,7 @@ impl FuseTable { segment_locs: Option>, pipeline: &mut Pipeline, ) -> Result<()> { - let Some(snapshot) = self.read_table_snapshot().await? else { + let Some(snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? else { return Ok(()); }; diff --git a/src/query/storages/fuse/src/operations/navigate.rs b/src/query/storages/fuse/src/operations/navigate.rs index 70379ee220890..804aaa58e3794 100644 --- a/src/query/storages/fuse/src/operations/navigate.rs +++ b/src/query/storages/fuse/src/operations/navigate.rs @@ -222,7 +222,7 @@ impl FuseTable { ) -> Result<(Arc, Vec)> { let retention = Duration::days(ctx.get_settings().get_data_retention_time_in_days()? as i64); - let root_snapshot = if let Some(snapshot) = self.read_table_snapshot().await? { + let root_snapshot = if let Some(snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? { snapshot } else { return Err(ErrorCode::TableHistoricalDataNotFound( diff --git a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs index e2240a318b2a7..8c7b672d474b8 100644 --- a/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/fuse_rows_fetcher.rs @@ -86,6 +86,7 @@ pub fn row_fetch_processor( block_reader.clone(), column_leaves.clone(), max_threads, + ctx.clone(), ), need_wrap_nullable, ) @@ -100,6 +101,7 @@ pub fn row_fetch_processor( block_reader.clone(), column_leaves.clone(), max_threads, + ctx.clone(), ), need_wrap_nullable, ) @@ -120,6 +122,7 @@ pub fn row_fetch_processor( block_reader.clone(), read_settings, max_threads, + ctx.clone(), ), need_wrap_nullable, ) @@ -134,6 +137,7 @@ pub fn row_fetch_processor( block_reader.clone(), read_settings, max_threads, + ctx.clone(), ), need_wrap_nullable, ) diff --git a/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs index a2eebac1ef03e..3a70d18547d43 100644 --- a/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/native_rows_fetcher.rs @@ -25,6 +25,7 @@ use databend_common_catalog::plan::split_row_id; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Projection; use databend_common_catalog::table::Table; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_expression::DataSchema; @@ -57,13 +58,14 @@ pub(super) struct NativeRowsFetcher { // To control the parallelism of fetching blocks. max_threads: usize, + ctx: Arc, } #[async_trait::async_trait] impl RowsFetcher for NativeRowsFetcher { #[async_backtrace::framed] async fn on_start(&mut self) -> Result<()> { - self.snapshot = self.table.read_table_snapshot().await?; + self.snapshot = self.table.read_table_snapshot(self.ctx.txn_mgr()).await?; Ok(()) } @@ -167,6 +169,7 @@ impl NativeRowsFetcher { reader: Arc, column_leaves: Arc>>, max_threads: usize, + ctx: Arc, ) -> Self { let schema = table.schema(); let segment_reader = @@ -183,6 +186,7 @@ impl NativeRowsFetcher { part_map: HashMap::new(), segment_blocks_cache: HashMap::new(), max_threads, + ctx, } } diff --git a/src/query/storages/fuse/src/operations/read/parquet_rows_fetcher.rs b/src/query/storages/fuse/src/operations/read/parquet_rows_fetcher.rs index 7a8313c417b9b..f0a265b16b426 100644 --- a/src/query/storages/fuse/src/operations/read/parquet_rows_fetcher.rs +++ b/src/query/storages/fuse/src/operations/read/parquet_rows_fetcher.rs @@ -23,6 +23,7 @@ use databend_common_catalog::plan::split_row_id; use databend_common_catalog::plan::PartInfoPtr; use databend_common_catalog::plan::Projection; use databend_common_catalog::table::Table; +use databend_common_catalog::table_context::TableContext; use databend_common_exception::Result; use databend_common_expression::DataBlock; use databend_common_expression::DataSchema; @@ -56,13 +57,14 @@ pub(super) struct ParquetRowsFetcher { // To control the parallelism of fetching blocks. max_threads: usize, + ctx: Arc, } #[async_trait::async_trait] impl RowsFetcher for ParquetRowsFetcher { #[async_backtrace::framed] async fn on_start(&mut self) -> Result<()> { - self.snapshot = self.table.read_table_snapshot().await?; + self.snapshot = self.table.read_table_snapshot(self.ctx.txn_mgr()).await?; Ok(()) } @@ -157,6 +159,7 @@ impl ParquetRowsFetcher { reader: Arc, settings: ReadSettings, max_threads: usize, + ctx: Arc, ) -> Self { let schema = table.schema(); let segment_reader = @@ -172,6 +175,7 @@ impl ParquetRowsFetcher { part_map: HashMap::new(), segment_blocks_cache: HashMap::new(), max_threads, + ctx, } } diff --git a/src/query/storages/fuse/src/operations/read_partitions.rs b/src/query/storages/fuse/src/operations/read_partitions.rs index 40a97bc02aa79..88feda9d8f8f5 100644 --- a/src/query/storages/fuse/src/operations/read_partitions.rs +++ b/src/query/storages/fuse/src/operations/read_partitions.rs @@ -71,7 +71,7 @@ impl FuseTable { .await; } - let snapshot = self.read_table_snapshot().await?; + let snapshot = self.read_table_snapshot(ctx.txn_mgr()).await?; match snapshot { Some(snapshot) => { let snapshot_loc = self diff --git a/src/query/storages/fuse/src/operations/recluster.rs b/src/query/storages/fuse/src/operations/recluster.rs index a2238fd8ead13..93d464c7dff95 100644 --- a/src/query/storages/fuse/src/operations/recluster.rs +++ b/src/query/storages/fuse/src/operations/recluster.rs @@ -74,7 +74,7 @@ impl FuseTable { return Ok(None); } - let snapshot_opt = self.read_table_snapshot().await?; + let snapshot_opt = self.read_table_snapshot(ctx.txn_mgr()).await?; let snapshot = if let Some(val) = snapshot_opt { val } else { diff --git a/src/query/storages/fuse/src/operations/truncate.rs b/src/query/storages/fuse/src/operations/truncate.rs index f3397cc6e2571..120565a7c0ec7 100644 --- a/src/query/storages/fuse/src/operations/truncate.rs +++ b/src/query/storages/fuse/src/operations/truncate.rs @@ -37,7 +37,7 @@ impl FuseTable { pipeline: &mut Pipeline, mode: TruncateMode, ) -> Result<()> { - if let Some(prev_snapshot) = self.read_table_snapshot().await? { + if let Some(prev_snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? { // Delete operation commit can retry multi-times if table version mismatched. let prev_snapshot_id = if !matches!(mode, TruncateMode::Delete) { Some(prev_snapshot.snapshot_id) diff --git a/src/query/storages/fuse/src/operations/update.rs b/src/query/storages/fuse/src/operations/update.rs index fb64613f91b70..77706259b77b2 100644 --- a/src/query/storages/fuse/src/operations/update.rs +++ b/src/query/storages/fuse/src/operations/update.rs @@ -46,7 +46,7 @@ impl FuseTable { col_indices: Vec, query_row_id_col: bool, ) -> Result>> { - let snapshot_opt = self.read_table_snapshot().await?; + let snapshot_opt = self.read_table_snapshot(ctx.txn_mgr()).await?; // check if table is empty let snapshot = if let Some(val) = snapshot_opt { diff --git a/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information.rs b/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information.rs index 051a99ad29ba7..ac98de9e65823 100644 --- a/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information.rs +++ b/src/query/storages/fuse/src/table_functions/clustering_information/clustering_information.rs @@ -126,7 +126,7 @@ impl<'a> ClusteringInformation<'a> { } }; - let snapshot = self.table.read_table_snapshot().await?; + let snapshot = self.table.read_table_snapshot(self.ctx.txn_mgr()).await?; let now = Utc::now(); let timestamp = snapshot .as_ref() diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs index c0658d350e354..9e5b68b4d3652 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs @@ -67,7 +67,7 @@ impl<'a> FuseBlock<'a> { pub async fn get_blocks(&self) -> Result { let tbl = self.table; let snapshot_id = self.snapshot_id.clone(); - let maybe_snapshot = tbl.read_table_snapshot().await?; + let maybe_snapshot = tbl.read_table_snapshot(self.ctx.txn_mgr()).await?; if let Some(snapshot) = maybe_snapshot { if let Some(snapshot_id) = snapshot_id { // prepare the stream of snapshot diff --git a/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs b/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs index b4082995abbd8..0aa6626d229b3 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs @@ -67,7 +67,7 @@ impl<'a> FuseColumn<'a> { pub async fn get_blocks(&self) -> Result { let tbl = self.table; let snapshot_id = self.snapshot_id.clone(); - let maybe_snapshot = tbl.read_table_snapshot().await?; + let maybe_snapshot = tbl.read_table_snapshot(self.ctx.txn_mgr()).await?; if let Some(snapshot) = maybe_snapshot { if let Some(snapshot_id) = snapshot_id { // prepare the stream of snapshot diff --git a/src/query/storages/fuse/src/table_functions/fuse_encodings/fuse_encoding.rs b/src/query/storages/fuse/src/table_functions/fuse_encodings/fuse_encoding.rs index fd88c487e9bd4..e6ea24befb8c3 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_encodings/fuse_encoding.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_encodings/fuse_encoding.rs @@ -83,7 +83,7 @@ impl<'a> FuseEncoding<'a> { continue; } let mut columns_info = vec![]; - let snapshot = table.read_table_snapshot().await?; + let snapshot = table.read_table_snapshot(self.ctx.txn_mgr()).await?; if snapshot.is_none() { continue; } diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs index 8c109b65251d7..87750473ce677 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs @@ -61,7 +61,7 @@ impl<'a> FuseSegment<'a> { pub async fn get_segments(&self) -> Result { let tbl = self.table; let snapshot_id = self.snapshot_id.clone(); - let maybe_snapshot = tbl.read_table_snapshot().await?; + let maybe_snapshot = tbl.read_table_snapshot(self.ctx.txn_mgr()).await?; if let Some(snapshot) = maybe_snapshot { // find the element by snapshot_id in stream if let Some(snapshot_id) = snapshot_id { diff --git a/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot.rs b/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot.rs index c21e733bb4a2c..d08050c77020b 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_snapshots/fuse_snapshot.rs @@ -48,7 +48,7 @@ impl<'a> FuseSnapshot<'a> { pub async fn get_snapshots(self, limit: Option) -> Result { let meta_location_generator = self.table.meta_location_generator.clone(); let snapshot_location = self.table.snapshot_loc().await?; - let snapshot = self.table.read_table_snapshot().await?; + let snapshot = self.table.read_table_snapshot(self.ctx.txn_mgr()).await?; if let Some(snapshot_location) = snapshot_location { let snapshot_version = TableMetaLocationGenerator::snapshot_version(snapshot_location.as_str()); diff --git a/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic.rs b/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic.rs index 16626b7ecd511..650fb82f531bb 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_statistics/fuse_statistic.rs @@ -42,7 +42,7 @@ impl<'a> FuseStatistic<'a> { #[async_backtrace::framed] pub async fn get_statistic(self) -> Result { - let snapshot_opt = self.table.read_table_snapshot().await?; + let snapshot_opt = self.table.read_table_snapshot(self.ctx.txn_mgr()).await?; if let Some(snapshot) = snapshot_opt { let table_statistics = self .table diff --git a/src/query/storages/stream/src/stream_table.rs b/src/query/storages/stream/src/stream_table.rs index 846c8b6da7147..22a4962fee0de 100644 --- a/src/query/storages/stream/src/stream_table.rs +++ b/src/query/storages/stream/src/stream_table.rs @@ -317,7 +317,7 @@ impl Table for StreamTable { table_name: &str, consume: bool, ) -> Result { - let table = self.source_table(ctx).await?; + let table = self.source_table(ctx.clone()).await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; let table_desc = if consume { format!("{}.{} with consume", database_name, table_name) @@ -330,6 +330,7 @@ impl Table for StreamTable { &self.snapshot_loc(), table_desc, self.offset()?, + ctx.as_ref(), ) .await } From 59f6c3ffe0237d5541f08d69525cdd0d7d2758dd Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 20:29:57 +0800 Subject: [PATCH 05/13] update --- src/query/storages/fuse/src/fuse_table.rs | 12 +---- .../fuse/src/io/read/meta/meta_readers.rs | 28 ++++++++--- src/query/storages/fuse/src/io/snapshots.rs | 50 +++++++++++-------- .../storages/fuse/src/operations/changes.rs | 28 ++++++++--- .../table_functions/fuse_blocks/fuse_block.rs | 3 +- .../fuse_segments/fuse_segment.rs | 3 +- .../storages/system/src/streams_table.rs | 1 + 7 files changed, 77 insertions(+), 48 deletions(-) diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index 728a8c98d6cfb..73e42c81fafd9 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -328,17 +328,7 @@ impl FuseTable { &self, txn_mgr: TxnManagerRef, ) -> Result>> { - { - let guard = txn_mgr.lock(); - if guard.is_active() { - if let Some(snapshot) = - guard.get_table_snapshot_by_id(self.table_info.ident.table_id) - { - return Ok(Some(snapshot.clone())); - } - } - } - let reader = MetaReaders::table_snapshot_reader(self.get_operator()); + let reader = MetaReaders::table_snapshot_reader(self.get_operator(),txn_mgr); self.read_table_snapshot_with_reader(reader).await } diff --git a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs index 2a5d30a780073..e24bf4165d19d 100644 --- a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs +++ b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs @@ -35,6 +35,7 @@ use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; use databend_storages_common_table_meta::meta::TableSnapshotStatisticsVersion; use databend_storages_common_table_meta::readers::VersionedReader; +use databend_storages_common_txn::TxnManagerRef; use futures::AsyncSeek; use futures_util::AsyncSeekExt; use opendal::Buffer; @@ -47,7 +48,8 @@ use self::thrift_file_meta_read::read_thrift_file_metadata; pub type TableSnapshotStatisticsReader = InMemoryItemCacheReader>; pub type BloomIndexMetaReader = InMemoryItemCacheReader>; -pub type TableSnapshotReader = InMemoryItemCacheReader>; +pub type TableSnapshotReader = + InMemoryItemCacheReader>; pub type CompactSegmentInfoReader = InMemoryItemCacheReader< CompactSegmentInfo, LoaderWrapper<(Operator, TableSchemaRef)>, @@ -74,15 +76,18 @@ impl MetaReaders { CompactSegmentInfoReader::new(None, LoaderWrapper((dal, schema))) } - pub fn table_snapshot_reader(dal: Operator) -> TableSnapshotReader { + pub fn table_snapshot_reader(dal: Operator, txn_mgr: TxnManagerRef) -> TableSnapshotReader { TableSnapshotReader::new( CacheManager::instance().get_table_snapshot_cache(), - LoaderWrapper(dal), + LoaderWrapper((dal, txn_mgr)), ) } - pub fn table_snapshot_reader_without_cache(dal: Operator) -> TableSnapshotReader { - TableSnapshotReader::new(None, LoaderWrapper(dal)) + pub fn table_snapshot_reader_without_cache( + dal: Operator, + txn_mgr: TxnManagerRef, + ) -> TableSnapshotReader { + TableSnapshotReader::new(None, LoaderWrapper((dal, txn_mgr))) } pub fn table_snapshot_statistics_reader(dal: Operator) -> TableSnapshotStatisticsReader { @@ -112,10 +117,19 @@ impl MetaReaders { pub struct LoaderWrapper(T); #[async_trait::async_trait] -impl Loader for LoaderWrapper { +impl Loader for LoaderWrapper<(Operator, TxnManagerRef)> { #[async_backtrace::framed] async fn load(&self, params: &LoadParams) -> Result { - let reader = bytes_reader(&self.0, params.location.as_str(), params.len_hint).await?; + let (dal, txn_mgr) = &self.0; + { + let guard = txn_mgr.lock(); + if guard.is_active() { + if let Some(snapshot) = guard.get_table_snapshot_by_id(params.table_id) { + return Ok(snapshot.as_ref().clone()); + } + } + } + let reader = bytes_reader(dal, params.location.as_str(), params.len_hint).await?; let version = SnapshotVersion::try_from(params.ver)?; version.read(reader.reader()) } diff --git a/src/query/storages/fuse/src/io/snapshots.rs b/src/query/storages/fuse/src/io/snapshots.rs index ee3bb05ed6c34..ff0075e1644a4 100644 --- a/src/query/storages/fuse/src/io/snapshots.rs +++ b/src/query/storages/fuse/src/io/snapshots.rs @@ -30,6 +30,7 @@ use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotLite; +use databend_storages_common_txn::TxnManagerRef; use futures::stream::StreamExt; use futures_util::TryStreamExt; use log::info; @@ -54,6 +55,7 @@ pub struct SnapshotLiteExtended { } // Read snapshot related operations. +#[derive(Clone)] pub struct SnapshotsIO { ctx: Arc, operator: Operator, @@ -76,8 +78,9 @@ impl SnapshotsIO { pub async fn read_snapshot( snapshot_location: String, data_accessor: Operator, + txn_mgr: TxnManagerRef, ) -> Result<(Arc, FormatVersion)> { - let reader = MetaReaders::table_snapshot_reader(data_accessor); + let reader = MetaReaders::table_snapshot_reader(data_accessor, txn_mgr); let ver: u64 = TableMetaLocationGenerator::snapshot_version(snapshot_location.as_str()); let load_params = LoadParams { location: snapshot_location, @@ -91,11 +94,12 @@ impl SnapshotsIO { #[async_backtrace::framed] async fn read_snapshot_lite( + self, snapshot_location: String, data_accessor: Operator, min_snapshot_timestamp: Option>, ) -> Result { - let reader = MetaReaders::table_snapshot_reader(data_accessor); + let reader = MetaReaders::table_snapshot_reader(data_accessor, self.ctx.txn_mgr()); let ver = TableMetaLocationGenerator::snapshot_version(snapshot_location.as_str()); let load_params = LoadParams { location: snapshot_location, @@ -130,14 +134,16 @@ impl SnapshotsIO { ) -> Result>> { // combine all the tasks. let mut iter = snapshot_files.iter(); - let tasks = std::iter::from_fn(move || { + let tasks = std::iter::from_fn(|| { iter.next().map(|location| { - Self::read_snapshot_lite( - location.clone(), - self.operator.clone(), - min_snapshot_timestamp, - ) - .in_span(Span::enter_with_local_parent(full_name!())) + let self_clone = self.clone(); + self_clone + .read_snapshot_lite( + location.clone(), + self.operator.clone(), + min_snapshot_timestamp, + ) + .in_span(Span::enter_with_local_parent(full_name!())) }) }); @@ -202,8 +208,9 @@ impl SnapshotsIO { } } - let (root_snapshot, format_version) = - Self::read_snapshot(root_snapshot_file.clone(), data_accessor.clone()).await?; + let (root_snapshot, format_version) = self + .read_snapshot(root_snapshot_file.clone(), data_accessor.clone()) + .await?; Ok(Self::chain_snapshots( snapshot_lites, @@ -221,7 +228,7 @@ impl SnapshotsIO { root_snapshot: String, limit: Option, ) -> Result> { - let table_snapshot_reader = MetaReaders::table_snapshot_reader(dal); + let table_snapshot_reader = MetaReaders::table_snapshot_reader(dal, self.ctx.txn_mgr()); let format_version = TableMetaLocationGenerator::snapshot_version(root_snapshot.as_str()); let lite_snapshot_stream = table_snapshot_reader .snapshot_history(root_snapshot, format_version, location_generator) @@ -237,12 +244,13 @@ impl SnapshotsIO { #[async_backtrace::framed] async fn read_snapshot_lite_extend( + self, snapshot_location: String, data_accessor: Operator, root_snapshot: Arc, ignore_timestamp: bool, ) -> Result { - let reader = MetaReaders::table_snapshot_reader(data_accessor); + let reader = MetaReaders::table_snapshot_reader(data_accessor, self.ctx.txn_mgr()); let ver = TableMetaLocationGenerator::snapshot_version(snapshot_location.as_str()); let load_params = LoadParams { location: snapshot_location, @@ -301,13 +309,15 @@ impl SnapshotsIO { let mut iter = snapshot_files.iter(); let tasks = std::iter::from_fn(move || { iter.next().map(|location| { - Self::read_snapshot_lite_extend( - location.clone(), - self.operator.clone(), - root_snapshot.clone(), - ignore_timestamp, - ) - .in_span(Span::enter_with_local_parent(full_name!())) + let self_clone = self.clone(); + self_clone + .read_snapshot_lite_extend( + location.clone(), + self.operator.clone(), + root_snapshot.clone(), + ignore_timestamp, + ) + .in_span(Span::enter_with_local_parent(full_name!())) }) }); diff --git a/src/query/storages/fuse/src/operations/changes.rs b/src/query/storages/fuse/src/operations/changes.rs index a8e1d68fdfaa7..ef9a9820d3e55 100644 --- a/src/query/storages/fuse/src/operations/changes.rs +++ b/src/query/storages/fuse/src/operations/changes.rs @@ -224,9 +224,12 @@ impl FuseTable { let latest_segments: HashSet<&Location> = HashSet::from_iter(&latest_snapshot.segments); - let (base_snapshot, _) = - SnapshotsIO::read_snapshot(base_location.clone(), self.get_operator()) - .await?; + let (base_snapshot, _) = SnapshotsIO::read_snapshot( + base_location.clone(), + self.get_operator(), + ctx.txn_mgr(), + ) + .await?; let base_segments = HashSet::from_iter(&base_snapshot.segments); // If the base segments are a subset of the latest segments, @@ -342,16 +345,24 @@ impl FuseTable { let latest = self.snapshot_loc().await?; let latest_segments = if let Some(snapshot) = latest { - let (sn, _) = - SnapshotsIO::read_snapshot(snapshot.to_string(), self.get_operator()).await?; + let (sn, _) = SnapshotsIO::read_snapshot( + snapshot.to_string(), + self.get_operator(), + ctx.txn_mgr(), + ) + .await?; HashSet::from_iter(sn.segments.clone()) } else { HashSet::new() }; let base_segments = if let Some(snapshot) = base { - let (sn, _) = - SnapshotsIO::read_snapshot(snapshot.to_string(), self.get_operator()).await?; + let (sn, _) = SnapshotsIO::read_snapshot( + snapshot.to_string(), + self.get_operator(), + ctx.txn_mgr(), + ) + .await?; HashSet::from_iter(sn.segments.clone()) } else { HashSet::new() @@ -437,7 +448,8 @@ impl FuseTable { }; let (base_snapshot, _) = - SnapshotsIO::read_snapshot(base_location.clone(), self.get_operator()).await?; + SnapshotsIO::read_snapshot(base_location.clone(), self.get_operator(), ctx.txn_mgr()) + .await?; let base_summary = base_snapshot.summary.clone(); let latest_summary = if let Some(snapshot) = self.read_table_snapshot(ctx.txn_mgr()).await? { diff --git a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs index 9e5b68b4d3652..fead256fa2370 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_blocks/fuse_block.rs @@ -75,7 +75,8 @@ impl<'a> FuseBlock<'a> { let snapshot_location = tbl .meta_location_generator .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; - let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); + let reader = + MetaReaders::table_snapshot_reader(tbl.get_operator(), self.ctx.txn_mgr()); let mut snapshot_stream = reader.snapshot_history( snapshot_location, snapshot_version, diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs index 87750473ce677..4e5a5c946da86 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs @@ -70,7 +70,8 @@ impl<'a> FuseSegment<'a> { let snapshot_location = tbl .meta_location_generator .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; - let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); + let reader = + MetaReaders::table_snapshot_reader((tbl.get_operator(), self.ctx.txn_mgr())); let mut snapshot_stream = reader.snapshot_history( snapshot_location, snapshot_version, diff --git a/src/query/storages/system/src/streams_table.rs b/src/query/storages/system/src/streams_table.rs index 6cb655a724460..5734f2a2d6624 100644 --- a/src/query/storages/system/src/streams_table.rs +++ b/src/query/storages/system/src/streams_table.rs @@ -252,6 +252,7 @@ impl AsyncSystemTable for StreamsTable { reason = SnapshotsIO::read_snapshot( location, fuse_table.get_operator(), + ctx.txn_mgr(), ) .await .err() From 8b8e398c7467789cccfe71b02269eb3ab8901664 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 20:43:16 +0800 Subject: [PATCH 06/13] update --- src/query/storages/common/txn/src/manager.rs | 26 ++++++++++++++----- .../processors/multi_table_insert_commit.rs | 8 +++--- .../common/processors/sink_commit.rs | 18 ++++++++++--- 3 files changed, 39 insertions(+), 13 deletions(-) diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 363389e804606..c836090ae2fd5 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -60,8 +60,8 @@ pub struct TxnBuffer { need_purge_files: Vec<(StageInfo, Vec)>, - // table_id -> latest snapshot - snapshots: HashMap>, + // location -> latest snapshot + snapshots: HashMap>, } #[derive(Debug, Clone)] @@ -110,8 +110,16 @@ impl TxnBuffer { } } - fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: Arc) { - self.snapshots.insert(table_id, snapshot); + fn upsert_table_snapshot( + &mut self, + old_location: Option<&str>, + new_location: &str, + snapshot: Arc, + ) { + if let Some(old_location) = old_location { + self.snapshots.remove(old_location); + } + self.snapshots.insert(new_location.to_string(), snapshot); } fn get_table_snapshot_by_id(&self, table_id: u64) -> Option> { @@ -299,8 +307,14 @@ impl TxnManager { std::mem::take(&mut self.txn_buffer.need_purge_files) } - pub fn upsert_table_snapshot(&mut self, table_id: u64, snapshot: Arc) { - self.txn_buffer.upsert_table_snapshot(table_id, snapshot); + pub fn upsert_table_snapshot( + &mut self, + old_location: Option<&str>, + new_location: &str, + snapshot: Arc, + ) { + self.txn_buffer + .upsert_table_snapshot(old_location, new_location, snapshot); } pub fn snapshots(&mut self) -> HashMap> { diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index 5b0b7841b376e..06fbbcfc8de30 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -254,9 +254,11 @@ async fn build_update_table_meta_req( snapshot.prev_snapshot_id = previous.prev_snapshot_id; assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); } - ctx.txn_mgr() - .lock() - .upsert_table_snapshot(fuse_table.get_id(), Arc::new(snapshot.clone())); + ctx.txn_mgr().lock().upsert_table_snapshot( + fuse_table.snapshot_loc(), + &location, + Arc::new(snapshot.clone()), + ); } else { let dal = fuse_table.get_operator(); dal.write(&location, snapshot.to_bytes()?).await?; diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index ec04953581757..29d2f8d924888 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -60,6 +60,7 @@ enum State { RefreshTable, GenerateSnapshot { previous: Option>, + previous_location: Option, cluster_key_meta: Option, table_info: TableInfo, }, @@ -67,6 +68,7 @@ enum State { data: Vec, snapshot: TableSnapshot, table_info: TableInfo, + previous_location: Option, }, Abort(ErrorCode), Finish, @@ -251,6 +253,7 @@ where F: SnapshotGenerator + Send + 'static previous, cluster_key_meta, table_info, + previous_location, } => { let change_tracking_enabled_during_commit = { let no_change_tracking_at_beginning = !self.change_tracking; @@ -292,6 +295,7 @@ where F: SnapshotGenerator + Send + 'static data: snapshot.to_bytes()?, snapshot, table_info, + previous_location, }; } Err(e) => { @@ -324,6 +328,7 @@ where F: SnapshotGenerator + Send + 'static let fuse_table = FuseTable::try_from_table(self.table.as_ref())?.to_owned(); let previous = fuse_table.read_table_snapshot(self.ctx.txn_mgr()).await?; + let previous_location = fuse_table.snapshot_loc().await?; // save current table info when commit to meta server // if table_id not match, update table meta will fail let table_info = fuse_table.table_info.clone(); @@ -347,6 +352,7 @@ where F: SnapshotGenerator + Send + 'static previous, cluster_key_meta: fuse_table.cluster_key_meta.clone(), table_info, + previous_location, }; } } @@ -354,6 +360,7 @@ where F: SnapshotGenerator + Send + 'static data, mut snapshot, table_info, + previous_location, } => { let location = self .location_gen @@ -370,10 +377,11 @@ where F: SnapshotGenerator + Send + 'static assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); } let snapshot_ref = Arc::new(snapshot); - self.ctx - .txn_mgr() - .lock() - .upsert_table_snapshot(table_info.ident.table_id, snapshot_ref.clone()); + self.ctx.txn_mgr().lock().upsert_table_snapshot( + previous_location.as_ref(), + &location, + snapshot_ref.clone(), + ); snapshot_ref } else { self.dal.write(&location, data).await?; @@ -513,11 +521,13 @@ where F: SnapshotGenerator + Send + 'static self.table = self.table.refresh(self.ctx.as_ref()).await?; let fuse_table = FuseTable::try_from_table(self.table.as_ref())?.to_owned(); let previous = fuse_table.read_table_snapshot(self.ctx.txn_mgr()).await?; + let previous_location = fuse_table.snapshot_loc().await?; let cluster_key_meta = fuse_table.cluster_key_meta.clone(); self.state = State::GenerateSnapshot { previous, cluster_key_meta, table_info: fuse_table.table_info.clone(), + previous_location, }; } _ => return Err(ErrorCode::Internal("It's a bug.")), From 2bd5603d180ffb79bc0c662ebeb433502321f980 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 22:07:31 +0800 Subject: [PATCH 07/13] update --- src/query/catalog/src/table.rs | 2 + src/query/ee/src/stream/handler.rs | 1 + .../interpreters/interpreter_table_analyze.rs | 1 + .../interpreters/interpreter_table_create.rs | 2 +- .../interpreters/interpreter_txn_commit.rs | 42 +++++------ .../tests/it/storages/fuse/operations/gc.rs | 2 +- .../it/storages/fuse/operations/navigate.rs | 8 +-- src/query/sql/src/planner/binder/table.rs | 4 +- src/query/storages/common/txn/src/manager.rs | 12 ++-- src/query/storages/fuse/src/fuse_table.rs | 26 ++++--- .../fuse/src/io/read/meta/meta_readers.rs | 2 +- src/query/storages/fuse/src/io/snapshots.rs | 9 ++- .../storages/fuse/src/operations/changes.rs | 13 ++-- .../processors/multi_table_insert_commit.rs | 16 ++--- .../common/processors/sink_commit.rs | 20 +++--- src/query/storages/fuse/src/operations/gc.rs | 2 +- .../storages/fuse/src/operations/navigate.rs | 70 ++++++++++++------- .../storages/fuse/src/operations/revert.rs | 1 + .../fuse_columns/fuse_column.rs | 3 +- .../fuse_segments/fuse_segment.rs | 2 +- .../storages/system/src/streams_table.rs | 2 +- 21 files changed, 144 insertions(+), 96 deletions(-) diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index cd1a45e4114cb..03bbca0b2e3cf 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -40,6 +40,7 @@ use databend_common_storage::StorageMetrics; use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::ChangeType; +use databend_storages_common_txn::TxnManagerRef; use crate::plan::DataSourceInfo; use crate::plan::DataSourcePlan; @@ -298,6 +299,7 @@ pub trait Table: Sync + Send { &self, navigation: &TimeNavigation, abort_checker: AbortChecker, + _txn_mgr:TxnManagerRef, ) -> Result> { let _ = navigation; let _ = abort_checker; diff --git a/src/query/ee/src/stream/handler.rs b/src/query/ee/src/stream/handler.rs index 1b9145d6d2341..9cf5f75a8e8ba 100644 --- a/src/query/ee/src/stream/handler.rs +++ b/src/query/ee/src/stream/handler.rs @@ -103,6 +103,7 @@ impl StreamHandler for RealStreamHandler { "".to_string(), plan.navigation.as_ref(), abort_checker, + ctx.txn_mgr(), ) .await?; table.check_changes_valid(&table.get_table_info().desc, change_desc.seq)?; diff --git a/src/query/service/src/interpreters/interpreter_table_analyze.rs b/src/query/service/src/interpreters/interpreter_table_analyze.rs index 7499afae961e1..5f1177d7080e2 100644 --- a/src/query/service/src/interpreters/interpreter_table_analyze.rs +++ b/src/query/service/src/interpreters/interpreter_table_analyze.rs @@ -94,6 +94,7 @@ impl Interpreter for AnalyzeTableInterpreter { table_statistics.snapshot_id.simple().to_string(), ), self.ctx.clone().get_abort_checker(), + self.ctx.txn_mgr(), ) .await { diff --git a/src/query/service/src/interpreters/interpreter_table_create.rs b/src/query/service/src/interpreters/interpreter_table_create.rs index 78bdd3478abef..61a6816cb7213 100644 --- a/src/query/service/src/interpreters/interpreter_table_create.rs +++ b/src/query/service/src/interpreters/interpreter_table_create.rs @@ -319,7 +319,7 @@ impl CreateTableInterpreter { // using application level data operator is a temp workaround // please see discussions https://github.com/datafuselabs/databend/pull/10424 let operator = self.ctx.get_application_level_data_operator()?.operator(); - let reader = MetaReaders::table_snapshot_reader(operator); + let reader = MetaReaders::table_snapshot_reader(operator, self.ctx.txn_mgr()); let params = LoadParams { location: snapshot_loc.clone(), diff --git a/src/query/service/src/interpreters/interpreter_txn_commit.rs b/src/query/service/src/interpreters/interpreter_txn_commit.rs index 2f78974548799..8e98dc4107545 100644 --- a/src/query/service/src/interpreters/interpreter_txn_commit.rs +++ b/src/query/service/src/interpreters/interpreter_txn_commit.rs @@ -62,27 +62,27 @@ impl Interpreter for CommitInterpreter { { let snapshots = self.ctx.txn_mgr().lock().snapshots(); for (table_id, snapshot) in snapshots { - let table_info = self - .ctx - .txn_mgr() - .lock() - .get_table_from_buffer_by_id(table_id) - .ok_or_else(|| { - ErrorCode::UnknownTable(format!( - "Unknown table id in txn manager: {}", - table_id - )) - })?; - let table = catalog.get_table_by_info(&table_info)?; - let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let dal = fuse_table.get_operator(); - let location = fuse_table.snapshot_loc().await?.ok_or_else(|| { - ErrorCode::Internal(format!( - "Table {} has no snapshot location", - table_info.name - )) - })?; - dal.write(&location, snapshot.to_bytes()?).await?; + // let table_info = self + // .ctx + // .txn_mgr() + // .lock() + // .get_table_from_buffer_by_id(table_id) + // .ok_or_else(|| { + // ErrorCode::UnknownTable(format!( + // "Unknown table id in txn manager: {}", + // table_id + // )) + // })?; + // let table = catalog.get_table_by_info(&table_info)?; + // let fuse_table = FuseTable::try_from_table(table.as_ref())?; + // let dal = fuse_table.get_operator(); + // let location = fuse_table.snapshot_loc().await?.ok_or_else(|| { + // ErrorCode::Internal(format!( + // "Table {} has no snapshot location", + // table_info.name + // )) + // })?; + // dal.write(&location, snapshot.to_bytes()?).await?; } } let req = self.ctx.txn_mgr().lock().req(); diff --git a/src/query/service/tests/it/storages/fuse/operations/gc.rs b/src/query/service/tests/it/storages/fuse/operations/gc.rs index 617ff6ef2d89e..244c42444fc2e 100644 --- a/src/query/service/tests/it/storages/fuse/operations/gc.rs +++ b/src/query/service/tests/it/storages/fuse/operations/gc.rs @@ -273,7 +273,7 @@ async fn test_fuse_purge_older_version() -> Result<()> { let time_point = now - Duration::hours(12); let snapshot_loc = fuse_table.snapshot_loc().await?.unwrap(); let table = fuse_table - .navigate_to_time_point(snapshot_loc, time_point, ctx.clone().get_abort_checker()) + .navigate_to_time_point(snapshot_loc, time_point, ctx.clone().get_abort_checker(),ctx.txn_mgr()) .await?; let keep_last_snapshot = true; table diff --git a/src/query/service/tests/it/storages/fuse/operations/navigate.rs b/src/query/service/tests/it/storages/fuse/operations/navigate.rs index e5bfac6690c5c..d2fa1690a0f94 100644 --- a/src/query/service/tests/it/storages/fuse/operations/navigate.rs +++ b/src/query/service/tests/it/storages/fuse/operations/navigate.rs @@ -107,7 +107,7 @@ async fn test_fuse_navigate() -> Result<()> { let ctx = fixture.new_query_ctx().await?; // navigate from the instant that is just one ms before the timestamp of the latest snapshot let tbl = fuse_table - .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker()) + .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker(),ctx.txn_mgr()) .await?; // check we got the snapshot of the first insertion @@ -121,7 +121,7 @@ async fn test_fuse_navigate() -> Result<()> { .sub(chrono::Duration::milliseconds(1)); // navigate from the instant that is just one ms before the timestamp of the last insertion let res = fuse_table - .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker()) + .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker(),ctx.txn_mgr()) .await; match res { Ok(_) => panic!("historical data should not exist"), @@ -134,7 +134,7 @@ async fn test_fuse_navigate() -> Result<()> { let checker = ctx.clone().get_abort_checker(); assert!(checker.is_aborting()); let res = fuse_table - .navigate_to_time_point(loc, instant, ctx.get_abort_checker()) + .navigate_to_time_point(loc, instant, ctx.get_abort_checker(),ctx.txn_mgr()) .await; assert!(res.is_err()); @@ -209,7 +209,7 @@ async fn test_navigate_for_purge() -> Result<()> { // 2. grab the history let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(),TxnManager::init()); let loc = fuse_table.snapshot_loc().await?.unwrap(); assert_eq!(third_snapshot, loc); let version = TableMetaLocationGenerator::snapshot_version(loc.as_str()); diff --git a/src/query/sql/src/planner/binder/table.rs b/src/query/sql/src/planner/binder/table.rs index 1e9265a006eee..2d6c92932a71d 100644 --- a/src/query/sql/src/planner/binder/table.rs +++ b/src/query/sql/src/planner/binder/table.rs @@ -497,7 +497,9 @@ impl Binder { .await?; if let Some(desc) = navigation { - table_meta = table_meta.navigate_to(desc, abort_checker).await?; + table_meta = table_meta + .navigate_to(desc, abort_checker, self.ctx.txn_mgr()) + .await?; } Ok(table_meta) }) diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index c836090ae2fd5..30d830d200466 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -122,11 +122,11 @@ impl TxnBuffer { self.snapshots.insert(new_location.to_string(), snapshot); } - fn get_table_snapshot_by_id(&self, table_id: u64) -> Option> { - self.snapshots.get(&table_id).cloned() + fn get_table_snapshot_by_location(&self, location: &str) -> Option> { + self.snapshots.get(location).cloned() } - fn snapshots(&mut self) -> HashMap> { + fn snapshots(&mut self) -> HashMap> { std::mem::take(&mut self.snapshots) } } @@ -317,11 +317,11 @@ impl TxnManager { .upsert_table_snapshot(old_location, new_location, snapshot); } - pub fn snapshots(&mut self) -> HashMap> { + pub fn snapshots(&mut self) -> HashMap> { self.txn_buffer.snapshots() } - pub fn get_table_snapshot_by_id(&self, table_id: u64) -> Option> { - self.txn_buffer.get_table_snapshot_by_id(table_id) + pub fn get_table_snapshot_by_location(&self, location: &str) -> Option> { + self.txn_buffer.get_table_snapshot_by_location(location) } } diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index 73e42c81fafd9..ab233129120c6 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -328,14 +328,17 @@ impl FuseTable { &self, txn_mgr: TxnManagerRef, ) -> Result>> { - let reader = MetaReaders::table_snapshot_reader(self.get_operator(),txn_mgr); + let reader = MetaReaders::table_snapshot_reader(self.get_operator(), txn_mgr); self.read_table_snapshot_with_reader(reader).await } #[minitrace::trace] #[async_backtrace::framed] - pub async fn read_table_snapshot_without_cache(&self) -> Result>> { - let reader = MetaReaders::table_snapshot_reader_without_cache(self.get_operator()); + pub async fn read_table_snapshot_without_cache( + &self, + txn_mgr: TxnManagerRef, + ) -> Result>> { + let reader = MetaReaders::table_snapshot_reader_without_cache(self.get_operator(), txn_mgr); self.read_table_snapshot_with_reader(reader).await } @@ -819,11 +822,12 @@ impl Table for FuseTable { &self, navigation: &TimeNavigation, abort_checker: AbortChecker, + txn_mgr: TxnManagerRef, ) -> Result> { match navigation { - TimeNavigation::TimeTravel(point) => { - Ok(self.navigate_to_point(point, abort_checker).await?) - } + TimeNavigation::TimeTravel(point) => Ok(self + .navigate_to_point(point, abort_checker, txn_mgr) + .await?), TimeNavigation::Changes { append_only, at, @@ -831,7 +835,7 @@ impl Table for FuseTable { desc, } => { let mut end_point = if let Some(end) = end { - self.navigate_to_point(end, abort_checker.clone()) + self.navigate_to_point(end, abort_checker.clone(), txn_mgr.clone()) .await? .as_ref() .clone() @@ -839,7 +843,13 @@ impl Table for FuseTable { self.clone() }; let changes_desc = end_point - .get_change_descriptor(*append_only, desc.clone(), Some(at), abort_checker) + .get_change_descriptor( + *append_only, + desc.clone(), + Some(at), + abort_checker, + txn_mgr, + ) .await?; end_point.changes_desc = Some(changes_desc); Ok(Arc::new(end_point)) diff --git a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs index e24bf4165d19d..6f7e346515332 100644 --- a/src/query/storages/fuse/src/io/read/meta/meta_readers.rs +++ b/src/query/storages/fuse/src/io/read/meta/meta_readers.rs @@ -124,7 +124,7 @@ impl Loader for LoaderWrapper<(Operator, TxnManagerRef)> { { let guard = txn_mgr.lock(); if guard.is_active() { - if let Some(snapshot) = guard.get_table_snapshot_by_id(params.table_id) { + if let Some(snapshot) = guard.get_table_snapshot_by_location(¶ms.location) { return Ok(snapshot.as_ref().clone()); } } diff --git a/src/query/storages/fuse/src/io/snapshots.rs b/src/query/storages/fuse/src/io/snapshots.rs index ff0075e1644a4..c2bb0a48bd453 100644 --- a/src/query/storages/fuse/src/io/snapshots.rs +++ b/src/query/storages/fuse/src/io/snapshots.rs @@ -208,9 +208,12 @@ impl SnapshotsIO { } } - let (root_snapshot, format_version) = self - .read_snapshot(root_snapshot_file.clone(), data_accessor.clone()) - .await?; + let (root_snapshot, format_version) = Self::read_snapshot( + root_snapshot_file.clone(), + data_accessor.clone(), + self.ctx.txn_mgr(), + ) + .await?; Ok(Self::chain_snapshots( snapshot_lites, diff --git a/src/query/storages/fuse/src/operations/changes.rs b/src/query/storages/fuse/src/operations/changes.rs index ef9a9820d3e55..67c10d10e2a17 100644 --- a/src/query/storages/fuse/src/operations/changes.rs +++ b/src/query/storages/fuse/src/operations/changes.rs @@ -43,6 +43,7 @@ use databend_storages_common_table_meta::table::ChangeType; use databend_storages_common_table_meta::table::StreamMode; use databend_storages_common_table_meta::table::OPT_KEY_CHANGE_TRACKING_BEGIN_VER; use databend_storages_common_table_meta::table::OPT_KEY_TABLE_VER; +use databend_storages_common_txn::TxnManagerRef; use log::info; use crate::io::SegmentsIO; @@ -65,10 +66,11 @@ impl FuseTable { desc: String, navigation: Option<&NavigationPoint>, abort_checker: AbortChecker, + txn_mgr: TxnManagerRef, ) -> Result { // To support analyze table, we move the change tracking check out of the function. let source = if let Some(point) = navigation { - self.navigate_to_point(point, abort_checker) + self.navigate_to_point(point, abort_checker, txn_mgr.clone()) .await? .as_ref() .clone() @@ -84,9 +86,12 @@ impl FuseTable { .parse::()?, Some(_) => { if let Some(snapshot_loc) = &location { - let (snapshot, _) = - SnapshotsIO::read_snapshot(snapshot_loc.clone(), self.get_operator()) - .await?; + let (snapshot, _) = SnapshotsIO::read_snapshot( + snapshot_loc.clone(), + self.get_operator(), + txn_mgr, + ) + .await?; let Some(prev_table_seq) = snapshot.prev_table_seq else { return Err(ErrorCode::IllegalStream( "The stream navigation at point has not table version".to_string(), diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index 06fbbcfc8de30..e768671c68769 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -246,16 +246,16 @@ async fn build_update_table_meta_req( .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; let is_active = ctx.txn_mgr().lock().is_active(); if is_active { - if let Some(previous) = ctx - .txn_mgr() - .lock() - .get_table_snapshot_by_id(fuse_table.get_id()) - { - snapshot.prev_snapshot_id = previous.prev_snapshot_id; - assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + let old_location = fuse_table.snapshot_loc().await?; + if let Some(l) = &old_location { + if let Some(previous) = ctx.txn_mgr().lock().get_table_snapshot_by_location(l) { + snapshot.prev_snapshot_id = previous.prev_snapshot_id; + assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + } } + ctx.txn_mgr().lock().upsert_table_snapshot( - fuse_table.snapshot_loc(), + old_location.as_ref().map(|l| l.as_str()), &location, Arc::new(snapshot.clone()), ); diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 29d2f8d924888..846b32a65d218 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -367,18 +367,20 @@ where F: SnapshotGenerator + Send + 'static .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshot::VERSION)?; let is_active = self.ctx.txn_mgr().lock().is_active(); let snapshot_ref = if is_active { - if let Some(previous) = self - .ctx - .txn_mgr() - .lock() - .get_table_snapshot_by_id(table_info.ident.table_id) - { - snapshot.prev_snapshot_id = previous.prev_snapshot_id; - assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + if let Some(l) = &previous_location { + if let Some(previous) = self + .ctx + .txn_mgr() + .lock() + .get_table_snapshot_by_location(l.as_str()) + { + snapshot.prev_snapshot_id = previous.prev_snapshot_id; + assert_eq!(snapshot.prev_table_seq, previous.prev_table_seq); + } } let snapshot_ref = Arc::new(snapshot); self.ctx.txn_mgr().lock().upsert_table_snapshot( - previous_location.as_ref(), + previous_location.as_ref().map(|s| s.as_str()), &location, snapshot_ref.clone(), ); diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 792432c822b20..5162b08d0be47 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -305,7 +305,7 @@ impl FuseTable { } let snapshot_location = root_snapshot_location_op.unwrap(); - let reader = MetaReaders::table_snapshot_reader(self.get_operator()); + let reader = MetaReaders::table_snapshot_reader(self.get_operator(), ctx.txn_mgr()); let ver = TableMetaLocationGenerator::snapshot_version(snapshot_location.as_str()); let params = LoadParams { location: snapshot_location.clone(), diff --git a/src/query/storages/fuse/src/operations/navigate.rs b/src/query/storages/fuse/src/operations/navigate.rs index 804aaa58e3794..d65452d16886a 100644 --- a/src/query/storages/fuse/src/operations/navigate.rs +++ b/src/query/storages/fuse/src/operations/navigate.rs @@ -28,6 +28,7 @@ use databend_storages_common_cache::LoadParams; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; use databend_storages_common_table_meta::table::OPT_KEY_SOURCE_TABLE_ID; +use databend_storages_common_txn::TxnManagerRef; use futures::TryStreamExt; use log::warn; use opendal::EntryMode; @@ -47,10 +48,11 @@ impl FuseTable { &self, point: &NavigationPoint, abort_checker: AbortChecker, + txn_mgr: TxnManagerRef, ) -> Result> { match point { NavigationPoint::SnapshotID(snapshot_id) => { - self.navigate_to_snapshot(snapshot_id.as_str(), abort_checker) + self.navigate_to_snapshot(snapshot_id.as_str(), abort_checker, txn_mgr) .await } NavigationPoint::TimePoint(time_point) => { @@ -59,15 +61,19 @@ impl FuseTable { "Empty Table has no historical data", )); }; - self.navigate_to_time_point(location, *time_point, abort_checker) + self.navigate_to_time_point(location, *time_point, abort_checker, txn_mgr) .await } - NavigationPoint::StreamInfo(info) => self.navigate_to_stream(info).await, + NavigationPoint::StreamInfo(info) => self.navigate_to_stream(info, txn_mgr).await, } } #[async_backtrace::framed] - pub async fn navigate_to_stream(&self, stream_info: &TableInfo) -> Result> { + pub async fn navigate_to_stream( + &self, + stream_info: &TableInfo, + txn_mgr: TxnManagerRef, + ) -> Result> { let options = stream_info.options(); let stream_table_id = options .get(OPT_KEY_SOURCE_TABLE_ID) @@ -88,7 +94,7 @@ impl FuseTable { return Ok(table.into()); }; let (snapshot, format_version) = - SnapshotsIO::read_snapshot(snapshot_loc.clone(), self.get_operator()).await?; + SnapshotsIO::read_snapshot(snapshot_loc.clone(), self.get_operator(), txn_mgr).await?; self.load_table_by_snapshot(snapshot.as_ref(), format_version) } @@ -98,14 +104,20 @@ impl FuseTable { location: String, time_point: DateTime, aborting: AbortChecker, + txn_mgr: TxnManagerRef, ) -> Result> { - self.find(location, aborting, |snapshot| { - if let Some(ts) = snapshot.timestamp { - ts <= time_point - } else { - false - } - }) + self.find( + location, + aborting, + |snapshot| { + if let Some(ts) = snapshot.timestamp { + ts <= time_point + } else { + false + } + }, + txn_mgr, + ) .await } @@ -114,6 +126,7 @@ impl FuseTable { &self, snapshot_id: &str, abort_checker: AbortChecker, + txn_mgr: TxnManagerRef, ) -> Result> { let Some(location) = self.snapshot_loc().await? else { return Err(ErrorCode::TableHistoricalDataNotFound( @@ -121,14 +134,19 @@ impl FuseTable { )); }; - self.find(location, abort_checker, |snapshot| { - snapshot - .snapshot_id - .simple() - .to_string() - .as_str() - .starts_with(snapshot_id) - }) + self.find( + location, + abort_checker, + |snapshot| { + snapshot + .snapshot_id + .simple() + .to_string() + .as_str() + .starts_with(snapshot_id) + }, + txn_mgr, + ) .await } @@ -138,12 +156,13 @@ impl FuseTable { location: String, abort_checker: AbortChecker, mut pred: P, + txn_mgr: TxnManagerRef, ) -> Result> where P: FnMut(&TableSnapshot) -> bool, { let snapshot_version = TableMetaLocationGenerator::snapshot_version(location.as_str()); - let reader = MetaReaders::table_snapshot_reader(self.get_operator()); + let reader = MetaReaders::table_snapshot_reader(self.get_operator(), txn_mgr); // grab the table history as stream // snapshots are order by timestamp DESC. let mut snapshot_stream = reader.snapshot_history( @@ -236,18 +255,18 @@ impl FuseTable { let (location, files) = match instant { Some(NavigationPoint::TimePoint(point)) => { time_point = std::cmp::min(point, time_point); - self.list_by_time_point(time_point).await + self.list_by_time_point(time_point, ctx.txn_mgr()).await } Some(NavigationPoint::SnapshotID(snapshot_id)) => { self.list_by_snapshot_id(snapshot_id.as_str(), time_point) .await } Some(NavigationPoint::StreamInfo(info)) => self.list_by_stream(info, time_point).await, - None => self.list_by_time_point(time_point).await, + None => self.list_by_time_point(time_point, ctx.txn_mgr()).await, }?; let table = self - .navigate_to_time_point(location, time_point, ctx.clone().get_abort_checker()) + .navigate_to_time_point(location, time_point, ctx.clone().get_abort_checker(),ctx.txn_mgr()) .await?; Ok((table, files)) @@ -257,6 +276,7 @@ impl FuseTable { pub async fn list_by_time_point( &self, time_point: DateTime, + txn_mgr: TxnManagerRef, ) -> Result<(String, Vec)> { let prefix = format!( "{}/{}/", @@ -274,7 +294,7 @@ impl FuseTable { } let location = files[0].clone(); - let reader = MetaReaders::table_snapshot_reader(self.get_operator()); + let reader = MetaReaders::table_snapshot_reader(self.get_operator(), txn_mgr); let ver = TableMetaLocationGenerator::snapshot_version(location.as_str()); let load_params = LoadParams { location, diff --git a/src/query/storages/fuse/src/operations/revert.rs b/src/query/storages/fuse/src/operations/revert.rs index ead9ac4e60fed..853be49e8a89b 100644 --- a/src/query/storages/fuse/src/operations/revert.rs +++ b/src/query/storages/fuse/src/operations/revert.rs @@ -36,6 +36,7 @@ impl FuseTable { .navigate_to_point( &navigation_descriptor.point, ctx.clone().get_abort_checker(), + ctx.txn_mgr(), ) .await?; let table_reverting_to = FuseTable::try_from_table(table.as_ref())?; diff --git a/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs b/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs index 0aa6626d229b3..af323daaabd22 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_columns/fuse_column.rs @@ -75,7 +75,8 @@ impl<'a> FuseColumn<'a> { let snapshot_location = tbl .meta_location_generator .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; - let reader = MetaReaders::table_snapshot_reader(tbl.get_operator()); + let reader = + MetaReaders::table_snapshot_reader(tbl.get_operator(), self.ctx.txn_mgr()); let mut snapshot_stream = reader.snapshot_history( snapshot_location, snapshot_version, diff --git a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs index 4e5a5c946da86..6913a3aeda310 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_segments/fuse_segment.rs @@ -71,7 +71,7 @@ impl<'a> FuseSegment<'a> { .meta_location_generator .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; let reader = - MetaReaders::table_snapshot_reader((tbl.get_operator(), self.ctx.txn_mgr())); + MetaReaders::table_snapshot_reader(tbl.get_operator(), self.ctx.txn_mgr()); let mut snapshot_stream = reader.snapshot_history( snapshot_location, snapshot_version, diff --git a/src/query/storages/system/src/streams_table.rs b/src/query/storages/system/src/streams_table.rs index 5734f2a2d6624..9e6b608199325 100644 --- a/src/query/storages/system/src/streams_table.rs +++ b/src/query/storages/system/src/streams_table.rs @@ -243,7 +243,7 @@ impl AsyncSystemTable for StreamsTable { // safe unwrap. let stream_table = StreamTable::try_from_table(table.as_ref()).unwrap(); - match stream_table.source_table(ctx).await { + match stream_table.source_table(ctx.clone()).await { Ok(source) => { // safe unwrap, has been checked in source_table. let fuse_table = From 6622d9cfe620336292610dca8f5bd73e601d4684 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 22:32:50 +0800 Subject: [PATCH 08/13] update --- src/query/ee/src/attach_table/handler.rs | 3 +- src/query/ee/src/fail_safe/handler.rs | 5 ++- .../storages/fuse/operations/vacuum_table.rs | 2 +- src/query/ee/src/stream/handler.rs | 2 +- .../interpreters/interpreter_txn_commit.rs | 39 ++++++++----------- src/query/storages/common/txn/src/manager.rs | 8 ++++ .../fuse_amend/fuse_amend_table.rs | 7 +++- 7 files changed, 38 insertions(+), 28 deletions(-) diff --git a/src/query/ee/src/attach_table/handler.rs b/src/query/ee/src/attach_table/handler.rs index df3a61d6e6872..72461d8dfcf62 100644 --- a/src/query/ee/src/attach_table/handler.rs +++ b/src/query/ee/src/attach_table/handler.rs @@ -29,6 +29,7 @@ use databend_storages_common_cache::LoadParams; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; +use databend_storages_common_txn::TxnManager; pub struct RealAttachTableHandler {} #[async_trait::async_trait] @@ -43,7 +44,7 @@ impl AttachTableHandler for RealAttachTableHandler { let sp = plan.storage_params.as_ref().unwrap(); let operator = DataOperator::try_create(sp).await?; let operator = operator.operator(); - let reader = MetaReaders::table_snapshot_reader(operator.clone()); + let reader = MetaReaders::table_snapshot_reader(operator.clone(),TxnManager::init()); let hint = format!("{}/{}", storage_prefix, FUSE_TBL_LAST_SNAPSHOT_HINT); let snapshot_loc = operator.read(&hint).await?.to_vec(); let snapshot_loc = String::from_utf8(snapshot_loc)?; diff --git a/src/query/ee/src/fail_safe/handler.rs b/src/query/ee/src/fail_safe/handler.rs index e62a4733b887f..9ccef8e34525c 100644 --- a/src/query/ee/src/fail_safe/handler.rs +++ b/src/query/ee/src/fail_safe/handler.rs @@ -142,7 +142,10 @@ impl Amender { } async fn recover_snapshot(&self, table: Box) -> Result<()> { - match table.read_table_snapshot_without_cache().await { + match table + .read_table_snapshot_without_cache(TxnManager::init()) + .await + { Ok(Some(snapshot)) => { let schema = table.schema(); let operator = table.get_operator(); diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs index 26381be5018f9..2f2c9e653d0ea 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs @@ -72,7 +72,7 @@ pub async fn get_snapshot_referenced_files( } let root_snapshot_location = root_snapshot_location_op.unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(),ctx.txn_mgr()); let ver = TableMetaLocationGenerator::snapshot_version(root_snapshot_location.as_str()); let params = LoadParams { location: root_snapshot_location.clone(), diff --git a/src/query/ee/src/stream/handler.rs b/src/query/ee/src/stream/handler.rs index 9cf5f75a8e8ba..31d48dccfc1b2 100644 --- a/src/query/ee/src/stream/handler.rs +++ b/src/query/ee/src/stream/handler.rs @@ -96,7 +96,7 @@ impl StreamHandler for RealStreamHandler { } let table = FuseTable::try_from_table(table.as_ref())?; - let abort_checker = ctx.get_abort_checker(); + let abort_checker = ctx.clone().get_abort_checker(); let change_desc = table .get_change_descriptor( plan.append_only, diff --git a/src/query/service/src/interpreters/interpreter_txn_commit.rs b/src/query/service/src/interpreters/interpreter_txn_commit.rs index 8e98dc4107545..3b01fe6c8a3a7 100644 --- a/src/query/service/src/interpreters/interpreter_txn_commit.rs +++ b/src/query/service/src/interpreters/interpreter_txn_commit.rs @@ -60,29 +60,22 @@ impl Interpreter for CommitInterpreter { let catalog = self.ctx.get_default_catalog()?; { - let snapshots = self.ctx.txn_mgr().lock().snapshots(); - for (table_id, snapshot) in snapshots { - // let table_info = self - // .ctx - // .txn_mgr() - // .lock() - // .get_table_from_buffer_by_id(table_id) - // .ok_or_else(|| { - // ErrorCode::UnknownTable(format!( - // "Unknown table id in txn manager: {}", - // table_id - // )) - // })?; - // let table = catalog.get_table_by_info(&table_info)?; - // let fuse_table = FuseTable::try_from_table(table.as_ref())?; - // let dal = fuse_table.get_operator(); - // let location = fuse_table.snapshot_loc().await?.ok_or_else(|| { - // ErrorCode::Internal(format!( - // "Table {} has no snapshot location", - // table_info.name - // )) - // })?; - // dal.write(&location, snapshot.to_bytes()?).await?; + let mutated_tables = self.ctx.txn_mgr().lock().mutated_tables(); + for table_info in mutated_tables.values() { + let table = catalog.get_table_by_info(&table_info)?; + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let dal = fuse_table.get_operator(); + if let Some(location) = fuse_table.snapshot_loc().await? { + let snapshot = self + .ctx + .txn_mgr() + .lock() + .get_table_snapshot_by_location(&location); + if let Some(snapshot) = snapshot + { + dal.write(&location, snapshot.to_bytes()?).await?; + } + } } } let req = self.ctx.txn_mgr().lock().req(); diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 30d830d200466..7db5ad53389ad 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -129,6 +129,10 @@ impl TxnBuffer { fn snapshots(&mut self) -> HashMap> { std::mem::take(&mut self.snapshots) } + + fn mutated_tables(&mut self) -> HashMap { + std::mem::take(&mut self.mutated_tables) + } } impl TxnManager { @@ -324,4 +328,8 @@ impl TxnManager { pub fn get_table_snapshot_by_location(&self, location: &str) -> Option> { self.txn_buffer.get_table_snapshot_by_location(location) } + + pub fn mutated_tables(&mut self) -> HashMap { + self.txn_buffer.mutated_tables() + } } diff --git a/src/query/storages/fuse/src/table_functions/fuse_amend/fuse_amend_table.rs b/src/query/storages/fuse/src/table_functions/fuse_amend/fuse_amend_table.rs index 826e2ccb3de24..d8de45f683959 100644 --- a/src/query/storages/fuse/src/table_functions/fuse_amend/fuse_amend_table.rs +++ b/src/query/storages/fuse/src/table_functions/fuse_amend/fuse_amend_table.rs @@ -104,10 +104,15 @@ impl Table for FuseAmendTable { #[async_backtrace::framed] async fn read_partitions( &self, - _ctx: Arc, + ctx: Arc, _push_downs: Option, _dry_run: bool, ) -> Result<(PartStatistics, Partitions)> { + if ctx.txn_mgr().lock().is_active() { + return Err(ErrorCode::StorageOther( + "Cannot amend table in active transaction", + )); + } Ok((PartStatistics::default(), Partitions::default())) } From 61938b3c20305991046497de257fdb9f0132a606 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Thu, 11 Jul 2024 23:56:52 +0800 Subject: [PATCH 09/13] make lint --- src/query/catalog/src/table.rs | 2 +- src/query/ee/src/attach_table/handler.rs | 2 +- .../storages/fuse/operations/vacuum_table.rs | 2 +- .../interpreters/interpreter_txn_commit.rs | 5 ++-- .../storages/fuse/operations/alter_table.rs | 4 ++- .../it/storages/fuse/operations/clustering.rs | 5 ++-- .../tests/it/storages/fuse/operations/gc.rs | 7 ++++- .../fuse/operations/internal_column.rs | 3 ++- .../it/storages/fuse/operations/navigate.rs | 26 ++++++++++++++----- .../service/tests/it/storages/fuse/pruning.rs | 3 ++- .../processors/multi_table_insert_commit.rs | 2 +- .../common/processors/sink_commit.rs | 2 +- .../storages/fuse/src/operations/navigate.rs | 7 ++++- 13 files changed, 48 insertions(+), 22 deletions(-) diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index 03bbca0b2e3cf..306c1eee6a04b 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -299,7 +299,7 @@ pub trait Table: Sync + Send { &self, navigation: &TimeNavigation, abort_checker: AbortChecker, - _txn_mgr:TxnManagerRef, + _txn_mgr: TxnManagerRef, ) -> Result> { let _ = navigation; let _ = abort_checker; diff --git a/src/query/ee/src/attach_table/handler.rs b/src/query/ee/src/attach_table/handler.rs index 72461d8dfcf62..ad88dc1cd2322 100644 --- a/src/query/ee/src/attach_table/handler.rs +++ b/src/query/ee/src/attach_table/handler.rs @@ -44,7 +44,7 @@ impl AttachTableHandler for RealAttachTableHandler { let sp = plan.storage_params.as_ref().unwrap(); let operator = DataOperator::try_create(sp).await?; let operator = operator.operator(); - let reader = MetaReaders::table_snapshot_reader(operator.clone(),TxnManager::init()); + let reader = MetaReaders::table_snapshot_reader(operator.clone(), TxnManager::init()); let hint = format!("{}/{}", storage_prefix, FUSE_TBL_LAST_SNAPSHOT_HINT); let snapshot_loc = operator.read(&hint).await?.to_vec(); let snapshot_loc = String::from_utf8(snapshot_loc)?; diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs index 2f2c9e653d0ea..a39396b51f8c3 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs @@ -72,7 +72,7 @@ pub async fn get_snapshot_referenced_files( } let root_snapshot_location = root_snapshot_location_op.unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(),ctx.txn_mgr()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), ctx.txn_mgr()); let ver = TableMetaLocationGenerator::snapshot_version(root_snapshot_location.as_str()); let params = LoadParams { location: root_snapshot_location.clone(), diff --git a/src/query/service/src/interpreters/interpreter_txn_commit.rs b/src/query/service/src/interpreters/interpreter_txn_commit.rs index 3b01fe6c8a3a7..036350c0c50ff 100644 --- a/src/query/service/src/interpreters/interpreter_txn_commit.rs +++ b/src/query/service/src/interpreters/interpreter_txn_commit.rs @@ -62,7 +62,7 @@ impl Interpreter for CommitInterpreter { { let mutated_tables = self.ctx.txn_mgr().lock().mutated_tables(); for table_info in mutated_tables.values() { - let table = catalog.get_table_by_info(&table_info)?; + let table = catalog.get_table_by_info(table_info)?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; let dal = fuse_table.get_operator(); if let Some(location) = fuse_table.snapshot_loc().await? { @@ -71,8 +71,7 @@ impl Interpreter for CommitInterpreter { .txn_mgr() .lock() .get_table_snapshot_by_location(&location); - if let Some(snapshot) = snapshot - { + if let Some(snapshot) = snapshot { dal.write(&location, snapshot.to_bytes()?).await?; } } diff --git a/src/query/service/tests/it/storages/fuse/operations/alter_table.rs b/src/query/service/tests/it/storages/fuse/operations/alter_table.rs index ac82d441b8567..b3d9258253693 100644 --- a/src/query/service/tests/it/storages/fuse/operations/alter_table.rs +++ b/src/query/service/tests/it/storages/fuse/operations/alter_table.rs @@ -44,6 +44,7 @@ use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; +use databend_storages_common_txn::TxnManager; use futures_util::TryStreamExt; use ordered_float::OrderedFloat; @@ -73,7 +74,8 @@ async fn check_segment_column_ids( .unwrap(); let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let snapshot_reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let snapshot_reader = + MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let params = LoadParams { location: snapshot_loc.clone(), len_hint: None, diff --git a/src/query/service/tests/it/storages/fuse/operations/clustering.rs b/src/query/service/tests/it/storages/fuse/operations/clustering.rs index 03667872a2d7c..1445239fe4daa 100644 --- a/src/query/service/tests/it/storages/fuse/operations/clustering.rs +++ b/src/query/service/tests/it/storages/fuse/operations/clustering.rs @@ -30,6 +30,7 @@ use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::OPT_KEY_DATABASE_ID; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; +use databend_storages_common_txn::TxnManager; #[tokio::test(flavor = "multi_thread")] async fn test_fuse_alter_table_cluster_key() -> databend_common_exception::Result<()> { @@ -87,7 +88,7 @@ async fn test_fuse_alter_table_cluster_key() -> databend_common_exception::Resul .options() .get(OPT_KEY_SNAPSHOT_LOCATION) .unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let load_params = LoadParams { location: snapshot_loc.clone(), @@ -123,7 +124,7 @@ async fn test_fuse_alter_table_cluster_key() -> databend_common_exception::Resul .options() .get(OPT_KEY_SNAPSHOT_LOCATION) .unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let params = LoadParams { location: snapshot_loc.clone(), diff --git a/src/query/service/tests/it/storages/fuse/operations/gc.rs b/src/query/service/tests/it/storages/fuse/operations/gc.rs index 244c42444fc2e..e1df9dfc71d6e 100644 --- a/src/query/service/tests/it/storages/fuse/operations/gc.rs +++ b/src/query/service/tests/it/storages/fuse/operations/gc.rs @@ -273,7 +273,12 @@ async fn test_fuse_purge_older_version() -> Result<()> { let time_point = now - Duration::hours(12); let snapshot_loc = fuse_table.snapshot_loc().await?.unwrap(); let table = fuse_table - .navigate_to_time_point(snapshot_loc, time_point, ctx.clone().get_abort_checker(),ctx.txn_mgr()) + .navigate_to_time_point( + snapshot_loc, + time_point, + ctx.clone().get_abort_checker(), + ctx.txn_mgr(), + ) .await?; let keep_last_snapshot = true; table diff --git a/src/query/service/tests/it/storages/fuse/operations/internal_column.rs b/src/query/service/tests/it/storages/fuse/operations/internal_column.rs index 5527734d4cb91..833a6c3086e41 100644 --- a/src/query/service/tests/it/storages/fuse/operations/internal_column.rs +++ b/src/query/service/tests/it/storages/fuse/operations/internal_column.rs @@ -36,6 +36,7 @@ use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; +use databend_storages_common_txn::TxnManager; use futures::TryStreamExt; fn expected_data_block( @@ -98,7 +99,7 @@ async fn check_partitions(parts: &Partitions, fixture: &TestFixture) -> Result<( .options() .get(OPT_KEY_SNAPSHOT_LOCATION) .unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let load_params = LoadParams { location: snapshot_name.clone(), diff --git a/src/query/service/tests/it/storages/fuse/operations/navigate.rs b/src/query/service/tests/it/storages/fuse/operations/navigate.rs index d2fa1690a0f94..0e910c348d054 100644 --- a/src/query/service/tests/it/storages/fuse/operations/navigate.rs +++ b/src/query/service/tests/it/storages/fuse/operations/navigate.rs @@ -25,8 +25,8 @@ use databend_query::storages::fuse::io::MetaReaders; use databend_query::storages::fuse::io::TableMetaLocationGenerator; use databend_query::storages::fuse::FuseTable; use databend_query::test_kits::*; +use databend_storages_common_txn::TxnManager; use futures::TryStreamExt; - #[tokio::test(flavor = "multi_thread")] async fn test_fuse_navigate() -> Result<()> { // - perform two insertions, which will left 2 snapshots @@ -80,7 +80,7 @@ async fn test_fuse_navigate() -> Result<()> { // 2. grab the history let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let loc = fuse_table.snapshot_loc().await?.unwrap(); assert_eq!(second_snapshot, loc); let version = TableMetaLocationGenerator::snapshot_version(loc.as_str()); @@ -107,7 +107,12 @@ async fn test_fuse_navigate() -> Result<()> { let ctx = fixture.new_query_ctx().await?; // navigate from the instant that is just one ms before the timestamp of the latest snapshot let tbl = fuse_table - .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker(),ctx.txn_mgr()) + .navigate_to_time_point( + loc.clone(), + instant, + ctx.clone().get_abort_checker(), + ctx.txn_mgr(), + ) .await?; // check we got the snapshot of the first insertion @@ -121,7 +126,12 @@ async fn test_fuse_navigate() -> Result<()> { .sub(chrono::Duration::milliseconds(1)); // navigate from the instant that is just one ms before the timestamp of the last insertion let res = fuse_table - .navigate_to_time_point(loc.clone(), instant, ctx.clone().get_abort_checker(),ctx.txn_mgr()) + .navigate_to_time_point( + loc.clone(), + instant, + ctx.clone().get_abort_checker(), + ctx.txn_mgr(), + ) .await; match res { Ok(_) => panic!("historical data should not exist"), @@ -134,7 +144,7 @@ async fn test_fuse_navigate() -> Result<()> { let checker = ctx.clone().get_abort_checker(); assert!(checker.is_aborting()); let res = fuse_table - .navigate_to_time_point(loc, instant, ctx.get_abort_checker(),ctx.txn_mgr()) + .navigate_to_time_point(loc, instant, ctx.clone().get_abort_checker(), ctx.txn_mgr()) .await; assert!(res.is_err()); @@ -209,7 +219,7 @@ async fn test_navigate_for_purge() -> Result<()> { // 2. grab the history let table = fixture.latest_default_table().await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(),TxnManager::init()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let loc = fuse_table.snapshot_loc().await?.unwrap(); assert_eq!(third_snapshot, loc); let version = TableMetaLocationGenerator::snapshot_version(loc.as_str()); @@ -231,7 +241,9 @@ async fn test_navigate_for_purge() -> Result<()> { assert!(modified.is_some()); let time_point = modified.unwrap().sub(chrono::Duration::milliseconds(1)); // navigate from the instant that is just one ms before the timestamp of the latest snapshot. - let (navigate, files) = fuse_table.list_by_time_point(time_point).await?; + let (navigate, files) = fuse_table + .list_by_time_point(time_point, TxnManager::init()) + .await?; assert_eq!(2, files.len()); assert_eq!(navigate, first_snapshot); diff --git a/src/query/service/tests/it/storages/fuse/pruning.rs b/src/query/service/tests/it/storages/fuse/pruning.rs index 0baddad87dedf..9502a5b67e319 100644 --- a/src/query/service/tests/it/storages/fuse/pruning.rs +++ b/src/query/service/tests/it/storages/fuse/pruning.rs @@ -50,6 +50,7 @@ use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::table::OPT_KEY_DATABASE_ID; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; +use databend_storages_common_txn::TxnManager; use opendal::Operator; async fn apply_block_pruning( @@ -164,7 +165,7 @@ async fn test_block_pruner() -> Result<()> { .get(OPT_KEY_SNAPSHOT_LOCATION) .unwrap(); - let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator(), TxnManager::init()); let load_params = LoadParams { location: snapshot_loc.clone(), diff --git a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs index e768671c68769..0f33f3d4c4cae 100644 --- a/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/multi_table_insert_commit.rs @@ -255,7 +255,7 @@ async fn build_update_table_meta_req( } ctx.txn_mgr().lock().upsert_table_snapshot( - old_location.as_ref().map(|l| l.as_str()), + old_location.as_deref(), &location, Arc::new(snapshot.clone()), ); diff --git a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs index 846b32a65d218..50710e55d50b7 100644 --- a/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs +++ b/src/query/storages/fuse/src/operations/common/processors/sink_commit.rs @@ -380,7 +380,7 @@ where F: SnapshotGenerator + Send + 'static } let snapshot_ref = Arc::new(snapshot); self.ctx.txn_mgr().lock().upsert_table_snapshot( - previous_location.as_ref().map(|s| s.as_str()), + previous_location.as_deref(), &location, snapshot_ref.clone(), ); diff --git a/src/query/storages/fuse/src/operations/navigate.rs b/src/query/storages/fuse/src/operations/navigate.rs index d65452d16886a..4eac1716c823d 100644 --- a/src/query/storages/fuse/src/operations/navigate.rs +++ b/src/query/storages/fuse/src/operations/navigate.rs @@ -266,7 +266,12 @@ impl FuseTable { }?; let table = self - .navigate_to_time_point(location, time_point, ctx.clone().get_abort_checker(),ctx.txn_mgr()) + .navigate_to_time_point( + location, + time_point, + ctx.clone().get_abort_checker(), + ctx.txn_mgr(), + ) .await?; Ok((table, files)) From 4c773bcc9b6a00f50aa907c61d8fb98d041c3d8f Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 12 Jul 2024 00:15:11 +0800 Subject: [PATCH 10/13] add more test --- .../09_0100_insert_multi_table.sql | 31 +++++++++++++++++++ .../14_transaction/14_0003_merge_into.test | 25 +++++++++++++++ 2 files changed, 56 insertions(+) diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0100_insert_multi_table.sql b/tests/sqllogictests/suites/base/09_fuse_engine/09_0100_insert_multi_table.sql index 0a91c746a10d5..e944cc8ee81b5 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0100_insert_multi_table.sql +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0100_insert_multi_table.sql @@ -325,6 +325,11 @@ begin transaction; statement ok insert into s values(3,4),(1,2),(5,6); +query I +select count(*) from fuse_snapshot('default', 's'); +---- +1 + query II INSERT FIRST WHEN c3 = 5 THEN @@ -335,6 +340,16 @@ SELECT * from s; ---- 1 2 +query I +select count(*) from fuse_snapshot('default', 't1'); +---- +1 + +query I +select count(*) from fuse_snapshot('default', 't2'); +---- +1 + query II select * from t1 order by c1; ---- @@ -349,6 +364,22 @@ select * from t2; statement ok rollback; +query I +select count(*) from fuse_snapshot('default', 's'); +---- +0 + +query I +select count(*) from fuse_snapshot('default', 't1'); +---- +0 + +query I +select count(*) from fuse_snapshot('default', 't2'); +---- +0 + + query II select * from t1 order by c1; ---- diff --git a/tests/sqllogictests/suites/base/14_transaction/14_0003_merge_into.test b/tests/sqllogictests/suites/base/14_transaction/14_0003_merge_into.test index 14862533d6850..c75a63c214fce 100644 --- a/tests/sqllogictests/suites/base/14_transaction/14_0003_merge_into.test +++ b/tests/sqllogictests/suites/base/14_transaction/14_0003_merge_into.test @@ -30,11 +30,21 @@ INSERT INTO employees VALUES (3, 'Charlie', 'Finance'), (4, 'David', 'HR'); +query I +select count(*) from fuse_snapshot('test_txn_merge_into','employees'); +---- +1 + statement ok INSERT INTO salaries VALUES (1, 50000.00), (2, 60000.00); +query I +select count(*) from fuse_snapshot('test_txn_merge_into','salaries'); +---- +1 + statement ok MERGE INTO salaries USING (SELECT * FROM employees) AS employees @@ -49,6 +59,11 @@ MERGE INTO salaries INSERT (employee_id, salary) VALUES (employees.employee_id, 55000.00); +query I +select count(*) from fuse_snapshot('test_txn_merge_into','salaries'); +---- +1 + query IF SELECT employee_id, salary FROM salaries order by employee_id; ---- @@ -60,6 +75,16 @@ SELECT employee_id, salary FROM salaries order by employee_id; statement ok COMMIT; +query I +select count(*) from fuse_snapshot('test_txn_merge_into','salaries'); +---- +1 + +query I +select count(*) from fuse_snapshot('test_txn_merge_into','employees'); +---- +1 + query IF SELECT employee_id, salary FROM salaries order by employee_id; ---- From e524131b76ec85b91868856e7ffc5c510257e9b1 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 12 Jul 2024 01:15:31 +0800 Subject: [PATCH 11/13] fix --- src/query/storages/common/txn/src/manager.rs | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/src/query/storages/common/txn/src/manager.rs b/src/query/storages/common/txn/src/manager.rs index 7db5ad53389ad..9a90aa54f4f6b 100644 --- a/src/query/storages/common/txn/src/manager.rs +++ b/src/query/storages/common/txn/src/manager.rs @@ -126,12 +126,8 @@ impl TxnBuffer { self.snapshots.get(location).cloned() } - fn snapshots(&mut self) -> HashMap> { - std::mem::take(&mut self.snapshots) - } - - fn mutated_tables(&mut self) -> HashMap { - std::mem::take(&mut self.mutated_tables) + fn mutated_tables(&self) -> HashMap { + self.mutated_tables.clone() } } @@ -321,15 +317,11 @@ impl TxnManager { .upsert_table_snapshot(old_location, new_location, snapshot); } - pub fn snapshots(&mut self) -> HashMap> { - self.txn_buffer.snapshots() - } - pub fn get_table_snapshot_by_location(&self, location: &str) -> Option> { self.txn_buffer.get_table_snapshot_by_location(location) } - pub fn mutated_tables(&mut self) -> HashMap { + pub fn mutated_tables(&self) -> HashMap { self.txn_buffer.mutated_tables() } } From cc8997880519bc82fcf968d13f0711ef1d8b6d93 Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 12 Jul 2024 08:28:38 +0800 Subject: [PATCH 12/13] fix ut --- src/query/service/tests/it/sql/exec/get_table_bind_test.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs index 169e3ba8dd6a2..13f8605b2460a 100644 --- a/src/query/service/tests/it/sql/exec/get_table_bind_test.rs +++ b/src/query/service/tests/it/sql/exec/get_table_bind_test.rs @@ -446,7 +446,7 @@ impl TableContext for CtxDelegation { } fn txn_mgr(&self) -> TxnManagerRef { - todo!() + self.ctx.txn_mgr() } fn incr_total_scan_value(&self, _value: ProgressValues) { From a8173568f24d5491257a965df0b96a60c5fa276c Mon Sep 17 00:00:00 2001 From: sky <3374614481@qq.com> Date: Fri, 12 Jul 2024 08:52:54 +0800 Subject: [PATCH 13/13] fix stream --- .../interpreters/interpreter_txn_commit.rs | 26 ++++++++++++------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/src/query/service/src/interpreters/interpreter_txn_commit.rs b/src/query/service/src/interpreters/interpreter_txn_commit.rs index 036350c0c50ff..24a7975635dc3 100644 --- a/src/query/service/src/interpreters/interpreter_txn_commit.rs +++ b/src/query/service/src/interpreters/interpreter_txn_commit.rs @@ -63,17 +63,23 @@ impl Interpreter for CommitInterpreter { let mutated_tables = self.ctx.txn_mgr().lock().mutated_tables(); for table_info in mutated_tables.values() { let table = catalog.get_table_by_info(table_info)?; - let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let dal = fuse_table.get_operator(); - if let Some(location) = fuse_table.snapshot_loc().await? { - let snapshot = self - .ctx - .txn_mgr() - .lock() - .get_table_snapshot_by_location(&location); - if let Some(snapshot) = snapshot { - dal.write(&location, snapshot.to_bytes()?).await?; + match table.engine().to_uppercase().as_str() { + "FUSE" => { + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let dal = fuse_table.get_operator(); + if let Some(location) = fuse_table.snapshot_loc().await? { + let snapshot = self + .ctx + .txn_mgr() + .lock() + .get_table_snapshot_by_location(&location); + if let Some(snapshot) = snapshot { + dal.write(&location, snapshot.to_bytes()?).await?; + } + } } + "STREAM" => {} + _ => unreachable!(), } } }