diff --git a/Cargo.lock b/Cargo.lock index 40f348ed57ba..a88b8b77cf6b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4313,6 +4313,7 @@ dependencies = [ "databend-common-pipeline-core", "databend-common-pipeline-sinks", "databend-common-pipeline-sources", + "databend-storages-common-table-meta", ] [[package]] @@ -4754,6 +4755,7 @@ dependencies = [ "opendal", "tantivy", "tempfile", + "uuid", ] [[package]] diff --git a/src/query/catalog/src/catalog/interface.rs b/src/query/catalog/src/catalog/interface.rs index 21c0210d60ff..bdc7975c81db 100644 --- a/src/query/catalog/src/catalog/interface.rs +++ b/src/query/catalog/src/catalog/interface.rs @@ -21,6 +21,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_meta_app::schema::database_name_ident::DatabaseNameIdent; use databend_common_meta_app::schema::dictionary_name_ident::DictionaryNameIdent; +use databend_common_meta_app::schema::least_visible_time_ident::LeastVisibleTimeIdent; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::CommitTableMetaReply; use databend_common_meta_app::schema::CommitTableMetaReq; @@ -63,6 +64,7 @@ use databend_common_meta_app::schema::GetSequenceReq; use databend_common_meta_app::schema::GetTableCopiedFileReply; use databend_common_meta_app::schema::GetTableCopiedFileReq; use databend_common_meta_app::schema::IndexMeta; +use databend_common_meta_app::schema::LeastVisibleTime; use databend_common_meta_app::schema::ListDictionaryReq; use databend_common_meta_app::schema::ListDroppedTableReq; use databend_common_meta_app::schema::ListIndexesByIdReq; @@ -523,5 +525,13 @@ pub trait Catalog: DynClone + Send + Sync + Debug { req: ListDictionaryReq, ) -> Result>; + async fn set_table_lvt( + &self, + _name_ident: &LeastVisibleTimeIdent, + _value: &LeastVisibleTime, + ) -> Result { + unimplemented!() + } + async fn rename_dictionary(&self, req: RenameDictionaryReq) -> Result<()>; } diff --git a/src/query/catalog/src/table.rs b/src/query/catalog/src/table.rs index 1ae950ae984e..9c0d083ef63d 100644 --- a/src/query/catalog/src/table.rs +++ b/src/query/catalog/src/table.rs @@ -38,6 +38,7 @@ use databend_common_pipeline_core::Pipeline; use databend_common_storage::Histogram; use databend_common_storage::StorageMetrics; use databend_storages_common_table_meta::meta::SnapshotId; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::ChangeType; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; @@ -234,7 +235,12 @@ pub trait Table: Sync + Send { } /// Assembly the pipeline of appending data to storage - fn append_data(&self, ctx: Arc, pipeline: &mut Pipeline) -> Result<()> { + fn append_data( + &self, + ctx: Arc, + pipeline: &mut Pipeline, + _table_meta_timestamps: TableMetaTimestamps, + ) -> Result<()> { let (_, _) = (ctx, pipeline); Err(ErrorCode::Unimplemented(format!( @@ -253,6 +259,7 @@ pub trait Table: Sync + Send { overwrite: bool, prev_snapshot_id: Option, _deduplicated_label: Option, + _table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { let (_, _, _, _, _, _) = ( ctx, diff --git a/src/query/catalog/src/table_context.rs b/src/query/catalog/src/table_context.rs index c7f45b570184..f49658c4039b 100644 --- a/src/query/catalog/src/table_context.rs +++ b/src/query/catalog/src/table_context.rs @@ -59,6 +59,8 @@ use databend_common_users::GrantObjectVisibilityChecker; use databend_storages_common_session::SessionState; use databend_storages_common_session::TxnManagerRef; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::TableSnapshot; use parking_lot::Mutex; use parking_lot::RwLock; use xorf::BinaryFuse16; @@ -328,6 +330,11 @@ pub trait TableContext: Send + Sync { fn has_bloom_runtime_filters(&self, id: usize) -> bool; fn txn_mgr(&self) -> TxnManagerRef; + fn get_table_meta_timestamps( + &self, + table_id: u64, + previous_snapshot: Option>, + ) -> Result; fn get_read_block_thresholds(&self) -> BlockThresholds; fn set_read_block_thresholds(&self, _thresholds: BlockThresholds); diff --git a/src/query/ee/Cargo.toml b/src/query/ee/Cargo.toml index def5aeb0bc62..4568b4c7d57a 100644 --- a/src/query/ee/Cargo.toml +++ b/src/query/ee/Cargo.toml @@ -65,6 +65,7 @@ jwt-simple = { workspace = true } log = { workspace = true } opendal = { workspace = true } tempfile = { workspace = true } +uuid = { workspace = true } # aws sdk aws-config = { workspace = true, features = ["behavior-version-latest"] } diff --git a/src/query/ee/src/storages/fuse/mod.rs b/src/query/ee/src/storages/fuse/mod.rs index 7b49add957fe..342c19d67ca1 100644 --- a/src/query/ee/src/storages/fuse/mod.rs +++ b/src/query/ee/src/storages/fuse/mod.rs @@ -18,4 +18,5 @@ pub mod operations; pub use io::snapshots::get_snapshot_referenced_segments; pub use operations::vacuum_drop_tables::vacuum_drop_tables; pub use operations::vacuum_table::do_vacuum; +pub use operations::vacuum_table_v2::do_vacuum2; pub use operations::virtual_columns::do_refresh_virtual_column; diff --git a/src/query/ee/src/storages/fuse/operations/handler.rs b/src/query/ee/src/storages/fuse/operations/handler.rs index 9de42f9aecbf..b4dc70cc04a1 100644 --- a/src/query/ee/src/storages/fuse/operations/handler.rs +++ b/src/query/ee/src/storages/fuse/operations/handler.rs @@ -28,6 +28,7 @@ use databend_enterprise_vacuum_handler::VacuumHandler; use databend_enterprise_vacuum_handler::VacuumHandlerWrapper; use crate::storages::fuse::do_vacuum; +use crate::storages::fuse::operations::vacuum_table_v2::do_vacuum2; use crate::storages::fuse::operations::vacuum_temporary_files::do_vacuum_temporary_files; use crate::storages::fuse::vacuum_drop_tables; pub struct RealVacuumHandler {} @@ -44,6 +45,15 @@ impl VacuumHandler for RealVacuumHandler { do_vacuum(fuse_table, ctx, retention_time, dry_run).await } + async fn do_vacuum2( + &self, + fuse_table: &FuseTable, + ctx: Arc, + respect_flash_back: bool, + ) -> Result> { + do_vacuum2(fuse_table, ctx, respect_flash_back).await + } + async fn do_vacuum_drop_tables( &self, threads_nums: usize, diff --git a/src/query/ee/src/storages/fuse/operations/mod.rs b/src/query/ee/src/storages/fuse/operations/mod.rs index 161c7a91567f..95e551c45f35 100644 --- a/src/query/ee/src/storages/fuse/operations/mod.rs +++ b/src/query/ee/src/storages/fuse/operations/mod.rs @@ -15,7 +15,7 @@ pub mod handler; pub mod vacuum_drop_tables; pub mod vacuum_table; +pub mod vacuum_table_v2; pub mod vacuum_temporary_files; pub mod virtual_columns; - pub use handler::RealVacuumHandler; 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 9f4f1e33c5ea..7b67a49cc02b 100644 --- a/src/query/ee/src/storages/fuse/operations/vacuum_table.rs +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table.rs @@ -177,7 +177,7 @@ pub async fn do_gc_orphan_files( let location_gen = fuse_table.meta_location_generator(); let segment_locations_to_be_purged = get_orphan_files_to_be_purged( fuse_table, - location_gen.segment_info_prefix(), + location_gen.segment_location_prefix(), referenced_files.segments, retention_time, ) @@ -298,7 +298,7 @@ pub async fn do_dry_run_orphan_files( // 2. Get purge orphan segment files. let segment_locations_to_be_purged = get_orphan_files_to_be_purged( fuse_table, - location_gen.segment_info_prefix(), + location_gen.segment_location_prefix(), referenced_files.segments, retention_time, ) diff --git a/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs new file mode 100644 index 000000000000..9e99d541aeb2 --- /dev/null +++ b/src/query/ee/src/storages/fuse/operations/vacuum_table_v2.rs @@ -0,0 +1,428 @@ +// Copyright 2023 Databend Cloud +// +// Licensed under the Elastic License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// https://www.elastic.co/licensing/elastic-license +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashSet; +use std::sync::Arc; + +use chrono::DateTime; +use chrono::Days; +use chrono::Utc; +use databend_common_base::base::uuid::Uuid; +use databend_common_catalog::table::Table; +use databend_common_catalog::table_context::AbortChecker; +use databend_common_catalog::table_context::TableContext; +use databend_common_exception::Result; +use databend_common_meta_app::schema::least_visible_time_ident::LeastVisibleTimeIdent; +use databend_common_meta_app::schema::LeastVisibleTime; +use databend_common_meta_app::schema::ListIndexesByIdReq; +use databend_common_storages_fuse::io::MetaReaders; +use databend_common_storages_fuse::io::SegmentsIO; +use databend_common_storages_fuse::io::TableMetaLocationGenerator; +use databend_common_storages_fuse::FuseTable; +use databend_storages_common_cache::LoadParams; +use databend_storages_common_io::Files; +use databend_storages_common_table_meta::meta::uuid_from_date_time; +use databend_storages_common_table_meta::meta::CompactSegmentInfo; +use databend_storages_common_table_meta::meta::TableSnapshot; +use databend_storages_common_table_meta::meta::VACUUM2_OBJECT_KEY_PREFIX; +use futures_util::TryStreamExt; +use log::info; +use uuid::Version; +#[async_backtrace::framed] +pub async fn do_vacuum2( + fuse_table: &FuseTable, + ctx: Arc, + respect_flash_back: bool, +) -> Result> { + let start = std::time::Instant::now(); + let retention_period_in_days = ctx.get_settings().get_data_retention_time_in_days()?; + let Some(lvt) = set_lvt(fuse_table, ctx.as_ref(), retention_period_in_days).await? else { + return Ok(vec![]); + }; + + ctx.set_status_info(&format!( + "set lvt for table {} takes {:?}, lvt: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + lvt + )); + + let start = std::time::Instant::now(); + let snapshots_before_lvt = list_until_timestamp( + fuse_table, + fuse_table + .meta_location_generator() + .snapshot_location_prefix(), + lvt, + true, + ) + .await?; + let elapsed = start.elapsed(); + ctx.set_status_info(&format!( + "list snapshots before lvt for table {} takes {:?}, snapshots_dir: {:?}, lvt: {:?}, snapshots: {:?}", + fuse_table.get_table_info().desc, + elapsed, + fuse_table.meta_location_generator().snapshot_location_prefix(), + lvt, + slice_summary(&snapshots_before_lvt) + )); + + let start = std::time::Instant::now(); + let is_vacuum_all = retention_period_in_days == 0; + let Some((gc_root, snapshots_to_gc)) = select_gc_root( + fuse_table, + &snapshots_before_lvt, + is_vacuum_all, + respect_flash_back, + ctx.clone().get_abort_checker(), + lvt, + ) + .await? + else { + return Ok(vec![]); + }; + ctx.set_status_info(&format!( + "select gc_root for table {} takes {:?}, gc_root: {:?}, snapshots_to_gc: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + gc_root, + slice_summary(snapshots_to_gc) + )); + + let start = std::time::Instant::now(); + let gc_root_timestamp = gc_root.timestamp.unwrap(); + let gc_root_segments = gc_root + .segments + .iter() + .map(|(path, _)| path) + .collect::>(); + let segments_before_gc_root = list_until_timestamp( + fuse_table, + fuse_table + .meta_location_generator() + .segment_location_prefix(), + gc_root_timestamp, + false, + ) + .await?; + ctx.set_status_info(&format!( + "list segments before gc_root for table {} takes {:?}, segment_dir: {:?}, gc_root_timestamp: {:?}, segments: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + fuse_table.meta_location_generator().segment_location_prefix(), + gc_root_timestamp, + slice_summary(&segments_before_gc_root) + )); + + let start = std::time::Instant::now(); + let segments_to_gc: Vec = segments_before_gc_root + .into_iter() + .filter(|s| !gc_root_segments.contains(s)) + .collect(); + ctx.set_status_info(&format!( + "Filter segments to gc for table {} takes {:?}, segments_to_gc: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + slice_summary(&segments_to_gc) + )); + + let start = std::time::Instant::now(); + let segments_io = + SegmentsIO::create(ctx.clone(), fuse_table.get_operator(), fuse_table.schema()); + let segments = segments_io + .read_segments::>(&gc_root.segments, false) + .await?; + let mut gc_root_blocks = HashSet::new(); + for segment in segments { + gc_root_blocks.extend(segment?.block_metas()?.iter().map(|b| b.location.0.clone())); + } + ctx.set_status_info(&format!( + "read segments for table {} takes {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + )); + + let start = std::time::Instant::now(); + let blocks_before_gc_root = list_until_timestamp( + fuse_table, + fuse_table.meta_location_generator().block_location_prefix(), + gc_root_timestamp, + false, + ) + .await?; + ctx.set_status_info(&format!( + "list blocks before gc_root for table {} takes {:?}, block_dir: {:?}, least_visible_timestamp: {:?}, blocks: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + fuse_table.meta_location_generator().block_location_prefix(), + gc_root_timestamp, + slice_summary(&blocks_before_gc_root) + )); + + let start = std::time::Instant::now(); + let blocks_to_gc: Vec = blocks_before_gc_root + .into_iter() + .filter(|b| !gc_root_blocks.contains(b)) + .collect(); + ctx.set_status_info(&format!( + "Filter blocks to gc for table {} takes {:?}, blocks_to_gc: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + slice_summary(&blocks_to_gc) + )); + + let start = std::time::Instant::now(); + let catalog = ctx.get_default_catalog()?; + let table_agg_index_ids = catalog + .list_index_ids_by_table_id(ListIndexesByIdReq::new( + ctx.get_tenant(), + fuse_table.get_id(), + )) + .await?; + let inverted_indexes = &fuse_table.get_table_info().meta.indexes; + let mut indexes_to_gc = Vec::with_capacity( + blocks_to_gc.len() * (table_agg_index_ids.len() + inverted_indexes.len() + 1), + ); + for loc in &blocks_to_gc { + for index_id in &table_agg_index_ids { + indexes_to_gc.push( + TableMetaLocationGenerator::gen_agg_index_location_from_block_location( + loc, *index_id, + ), + ); + } + for idx in inverted_indexes.values() { + indexes_to_gc.push( + TableMetaLocationGenerator::gen_inverted_index_location_from_block_location( + loc, + idx.name.as_str(), + idx.version.as_str(), + ), + ); + } + indexes_to_gc + .push(TableMetaLocationGenerator::gen_bloom_index_location_from_block_location(loc)); + } + + ctx.set_status_info(&format!( + "collect indexes to gc for table {} takes {:?}, indexes_to_gc: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + slice_summary(&indexes_to_gc) + )); + + let start = std::time::Instant::now(); + let subject_files_to_gc: Vec<_> = segments_to_gc + .into_iter() + .chain(blocks_to_gc.into_iter()) + .collect(); + let op = Files::create(ctx.clone(), fuse_table.get_operator()); + + // order is important + // indexes should be removed before blocks, because index locations to gc are generated from block locations + // subject_files should be removed before snapshots, because gc of subject_files depend on gc root + op.remove_file_in_batch(&indexes_to_gc, true).await?; + op.remove_file_in_batch(&subject_files_to_gc, true).await?; + op.remove_file_in_batch(snapshots_to_gc, true).await?; + + let files_to_gc: Vec<_> = subject_files_to_gc + .into_iter() + .chain(snapshots_to_gc.iter().cloned()) + .chain(indexes_to_gc.into_iter()) + .collect(); + ctx.set_status_info(&format!( + "remove files for table {} takes {:?}, files_to_gc: {:?}", + fuse_table.get_table_info().desc, + start.elapsed(), + slice_summary(&files_to_gc) + )); + Ok(files_to_gc) +} + +/// Try set lvt as min(latest_snapshot.timestamp, now - retention_time). +/// +/// Return `None` means we stop vacuumming, but don't want to report error to user. +async fn set_lvt( + fuse_table: &FuseTable, + ctx: &dyn TableContext, + retention: u64, +) -> Result>> { + let Some(latest_snapshot) = fuse_table.read_table_snapshot().await? else { + info!( + "Table {} has no snapshot, stop vacuuming", + fuse_table.get_table_info().desc + ); + return Ok(None); + }; + if !is_uuid_v7(&latest_snapshot.snapshot_id) { + info!( + "latest snapshot {:?} is not v7, stop vacuuming", + latest_snapshot + ); + return Ok(None); + } + let cat = ctx.get_default_catalog()?; + // safe to unwrap, as we have checked the version is v5 + let latest_ts = latest_snapshot.timestamp.unwrap(); + let lvt_point_candidate = if retention == 0 { + // when retention=0, only latest snapshot is reserved + latest_ts + } else { + std::cmp::min(Utc::now() - Days::new(retention), latest_ts) + }; + + let lvt_point = cat + .set_table_lvt( + &LeastVisibleTimeIdent::new(ctx.get_tenant(), fuse_table.get_id()), + &LeastVisibleTime::new(lvt_point_candidate), + ) + .await? + .time; + Ok(Some(lvt_point)) +} + +fn is_uuid_v7(uuid: &Uuid) -> bool { + let version = uuid.get_version(); + version.is_some_and(|v| matches!(v, Version::SortRand)) +} + +async fn list_until_prefix( + fuse_table: &FuseTable, + path: &str, + until: &str, + need_one_more: bool, +) -> Result> { + info!("list until prefix: {}", until); + let mut lister = fuse_table.get_operator().lister(path).await?; + let mut paths = vec![]; + while let Some(entry) = lister.try_next().await? { + if entry.path() >= until { + info!("entry path: {} >= until: {}", entry.path(), until); + if need_one_more { + paths.push(entry.path().to_string()); + } + break; + } + paths.push(entry.path().to_string()); + } + Ok(paths) +} + +async fn list_until_timestamp( + fuse_table: &FuseTable, + path: &str, + until: DateTime, + need_one_more: bool, +) -> Result> { + let uuid = uuid_from_date_time(until); + let uuid_str = uuid.simple().to_string(); + + // extract the most significant 48 bits, which is 12 characters + let timestamp_component = &uuid_str[..12]; + let until = format!( + "{}{}{}", + path, VACUUM2_OBJECT_KEY_PREFIX, timestamp_component + ); + list_until_prefix(fuse_table, path, &until, need_one_more).await +} + +async fn read_snapshot_from_location( + fuse_table: &FuseTable, + path: &str, +) -> Result> { + let reader = MetaReaders::table_snapshot_reader(fuse_table.get_operator()); + let ver = TableMetaLocationGenerator::snapshot_version(path); + info!("read snapshot from location: {}, version: {}", path, ver); + let params = LoadParams { + location: path.to_owned(), + len_hint: None, + ver, + put_cache: false, + }; + reader.read(¶ms).await +} + +async fn select_gc_root<'a>( + fuse_table: &FuseTable, + snapshots_before_lvt: &'a [String], + is_vacuum_all: bool, + respect_flash_back: bool, + abort_checker: AbortChecker, + lvt: DateTime, +) -> Result, &'a [String])>> { + let gc_root_path = if is_vacuum_all { + // safe to unwrap, or we should have stopped vacuuming in set_lvt() + fuse_table.snapshot_loc().unwrap() + } else if respect_flash_back { + let latest_location = fuse_table.snapshot_loc().unwrap(); + let gc_root = fuse_table + .find(latest_location, abort_checker, |snapshot| { + snapshot.timestamp.is_some_and(|ts| ts <= lvt) + }) + .await? + .snapshot_loc(); + let Some(gc_root) = gc_root else { + info!("no gc_root found, stop vacuuming"); + return Ok(None); + }; + gc_root + } else { + if snapshots_before_lvt.is_empty() { + info!("no snapshots before lvt, stop vacuuming"); + return Ok(None); + } + let anchor = + read_snapshot_from_location(fuse_table, snapshots_before_lvt.last().unwrap()).await?; + let Some((gc_root_id, gc_root_ver)) = anchor.prev_snapshot_id else { + info!("anchor has no prev_snapshot_id, stop vacuuming"); + return Ok(None); + }; + let gc_root_path = fuse_table + .meta_location_generator() + .snapshot_location_from_uuid(&gc_root_id, gc_root_ver)?; + if !is_uuid_v7(&gc_root_id) { + info!("gc_root {} is not v7", gc_root_path); + return Ok(None); + } + gc_root_path + }; + let gc_root = read_snapshot_from_location(fuse_table, &gc_root_path).await; + match gc_root { + Ok(gc_root) => { + info!("gc_root found: {:?}", gc_root); + let gc_root_idx = snapshots_before_lvt.binary_search(&gc_root_path).unwrap(); + let snapshots_to_gc = &snapshots_before_lvt[..gc_root_idx]; + Ok(Some((gc_root, snapshots_to_gc))) + } + Err(e) => { + info!("read gc_root {} failed: {:?}", gc_root_path, e); + Ok(None) + } + } +} + +fn slice_summary(s: &[T]) -> String { + if s.len() > 10 { + let first_five = &s[..5]; + let last_five = &s[s.len() - 5..]; + format!( + "First five: {:?}, Last five: {:?},Len: {}", + first_five, + last_five, + s.len() + ) + } else { + format!("{:?}", s) + } +} diff --git a/src/query/ee/tests/it/aggregating_index/index_refresh.rs b/src/query/ee/tests/it/aggregating_index/index_refresh.rs index 28157ae63eb1..bf4dd7993bf0 100644 --- a/src/query/ee/tests/it/aggregating_index/index_refresh.rs +++ b/src/query/ee/tests/it/aggregating_index/index_refresh.rs @@ -40,6 +40,7 @@ use databend_enterprise_query::test_kits::context::EESetup; use databend_query::interpreters::InterpreterFactory; use databend_query::sessions::QueryContext; use databend_query::test_kits::*; +use databend_storages_common_table_meta::meta::VACUUM2_OBJECT_KEY_PREFIX; use derive_visitor::DriveMut; use futures_util::TryStreamExt; @@ -84,7 +85,15 @@ async fn test_refresh_agg_index() -> Result<()> { blocks.sort(); indexes.sort(); - assert_eq!(blocks, indexes); + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes); // Check aggregating index is correct. { @@ -122,7 +131,15 @@ async fn test_refresh_agg_index() -> Result<()> { blocks.sort(); indexes.sort(); - assert_eq!(blocks, indexes); + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes); let new_block = { blocks.retain(|s| s != &pre_block); @@ -264,7 +281,15 @@ async fn test_sync_agg_index_after_update() -> Result<()> { blocks.sort(); indexes_0.sort(); - assert_eq!(blocks, indexes_0); + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_0); // Check aggregating index_0 is correct. { @@ -303,7 +328,17 @@ async fn test_sync_agg_index_after_update() -> Result<()> { blocks.sort(); indexes_0.sort(); - assert_eq!(blocks, indexes_0); + let indexes_0 = collect_file_names(&agg_index_path_0)?; + + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_0); // Check aggregating index_0 is correct after update. { @@ -388,7 +423,15 @@ async fn test_sync_agg_index_after_insert() -> Result<()> { blocks.sort(); indexes_1.sort(); - assert_eq!(blocks, indexes_1); + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_1); // Check aggregating index_0 is correct. { @@ -446,13 +489,31 @@ async fn test_sync_agg_index_after_insert() -> Result<()> { blocks.sort(); indexes_0.sort(); - assert_eq!(blocks, indexes_0); + let indexes_0 = collect_file_names(&agg_index_path_0)?; + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_0); // check index1 let mut indexes_1 = collect_file_names(&agg_index_path_1)?; indexes_1.sort(); - assert_eq!(blocks, indexes_1); + let indexes_1 = collect_file_names(&agg_index_path_1)?; + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_1); Ok(()) } @@ -494,7 +555,15 @@ async fn test_sync_agg_index_after_copy_into() -> Result<()> { blocks.sort(); indexes_0.sort(); - assert_eq!(blocks, indexes_0); + let blocks_remove_prefix_g = blocks + .iter() + .map(|b| { + b.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX) + .unwrap_or(b) + .to_string() + }) + .collect::>(); + assert_eq!(blocks_remove_prefix_g, indexes_0); // Check aggregating index_0 is correct. { diff --git a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs index d7430eab6825..d25581784e4d 100644 --- a/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs +++ b/src/query/ee/tests/it/storages/fuse/operations/vacuum.rs @@ -525,7 +525,7 @@ async fn test_remove_files_in_batch_do_not_swallow_errors() -> Result<()> { let file_util = Files::create(ctx, operator); // files to be deleted does not matter, faulty_accessor will always fail to delete - let r = file_util.remove_file_in_batch(vec!["1", "2"]).await; + let r = file_util.remove_file_in_batch(vec!["1", "2"], false).await; assert!(r.is_err()); // verify that accessor.delete() was called diff --git a/src/query/ee_features/vacuum_handler/src/vacuum_handler.rs b/src/query/ee_features/vacuum_handler/src/vacuum_handler.rs index e83b94181de9..29090f889232 100644 --- a/src/query/ee_features/vacuum_handler/src/vacuum_handler.rs +++ b/src/query/ee_features/vacuum_handler/src/vacuum_handler.rs @@ -40,6 +40,13 @@ pub trait VacuumHandler: Sync + Send { dry_run: bool, ) -> Result>>; + async fn do_vacuum2( + &self, + fuse_table: &FuseTable, + ctx: Arc, + respect_flash_back: bool, + ) -> Result>; + async fn do_vacuum_drop_tables( &self, threads_nums: usize, @@ -85,6 +92,18 @@ impl VacuumHandlerWrapper { .await } + #[async_backtrace::framed] + pub async fn do_vacuum2( + &self, + fuse_table: &FuseTable, + ctx: Arc, + respect_flash_back: bool, + ) -> Result> { + self.handler + .do_vacuum2(fuse_table, ctx, respect_flash_back) + .await + } + #[async_backtrace::framed] pub async fn do_vacuum_drop_tables( &self, diff --git a/src/query/service/src/catalogs/default/database_catalog.rs b/src/query/service/src/catalogs/default/database_catalog.rs index 70e78bb8508c..50d84e81cb4a 100644 --- a/src/query/service/src/catalogs/default/database_catalog.rs +++ b/src/query/service/src/catalogs/default/database_catalog.rs @@ -27,6 +27,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_meta_app::schema::database_name_ident::DatabaseNameIdent; use databend_common_meta_app::schema::dictionary_name_ident::DictionaryNameIdent; +use databend_common_meta_app::schema::least_visible_time_ident::LeastVisibleTimeIdent; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::CommitTableMetaReply; use databend_common_meta_app::schema::CommitTableMetaReq; @@ -68,6 +69,7 @@ use databend_common_meta_app::schema::GetSequenceReq; use databend_common_meta_app::schema::GetTableCopiedFileReply; use databend_common_meta_app::schema::GetTableCopiedFileReq; use databend_common_meta_app::schema::IndexMeta; +use databend_common_meta_app::schema::LeastVisibleTime; use databend_common_meta_app::schema::ListDictionaryReq; use databend_common_meta_app::schema::ListDroppedTableReq; use databend_common_meta_app::schema::ListIndexesByIdReq; @@ -836,6 +838,14 @@ impl Catalog for DatabaseCatalog { self.mutable_catalog.list_dictionaries(req).await } + async fn set_table_lvt( + &self, + name_ident: &LeastVisibleTimeIdent, + value: &LeastVisibleTime, + ) -> Result { + self.mutable_catalog.set_table_lvt(name_ident, value).await + } + async fn rename_dictionary(&self, req: RenameDictionaryReq) -> Result<()> { self.mutable_catalog.rename_dictionary(req).await } diff --git a/src/query/service/src/catalogs/default/mutable_catalog.rs b/src/query/service/src/catalogs/default/mutable_catalog.rs index d492b6b2b3f0..d3b9c7ae84ea 100644 --- a/src/query/service/src/catalogs/default/mutable_catalog.rs +++ b/src/query/service/src/catalogs/default/mutable_catalog.rs @@ -30,6 +30,7 @@ use databend_common_meta_app::schema::database_name_ident::DatabaseNameIdent; use databend_common_meta_app::schema::dictionary_name_ident::DictionaryNameIdent; use databend_common_meta_app::schema::index_id_ident::IndexId; use databend_common_meta_app::schema::index_id_ident::IndexIdIdent; +use databend_common_meta_app::schema::least_visible_time_ident::LeastVisibleTimeIdent; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::CommitTableMetaReply; use databend_common_meta_app::schema::CommitTableMetaReq; @@ -76,6 +77,7 @@ use databend_common_meta_app::schema::GetSequenceReq; use databend_common_meta_app::schema::GetTableCopiedFileReply; use databend_common_meta_app::schema::GetTableCopiedFileReq; use databend_common_meta_app::schema::IndexMeta; +use databend_common_meta_app::schema::LeastVisibleTime; use databend_common_meta_app::schema::ListDatabaseReq; use databend_common_meta_app::schema::ListDictionaryReq; use databend_common_meta_app::schema::ListDroppedTableReq; @@ -793,6 +795,14 @@ impl Catalog for MutableCatalog { Ok(self.ctx.meta.list_dictionaries(req).await?) } + async fn set_table_lvt( + &self, + name_ident: &LeastVisibleTimeIdent, + value: &LeastVisibleTime, + ) -> Result { + Ok(self.ctx.meta.set_table_lvt(name_ident, value).await?) + } + #[async_backtrace::framed] async fn rename_dictionary(&self, req: RenameDictionaryReq) -> Result<()> { let res = self.ctx.meta.rename_dictionary(req).await?; diff --git a/src/query/service/src/catalogs/default/session_catalog.rs b/src/query/service/src/catalogs/default/session_catalog.rs index d035933f843a..39f417759605 100644 --- a/src/query/service/src/catalogs/default/session_catalog.rs +++ b/src/query/service/src/catalogs/default/session_catalog.rs @@ -25,6 +25,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_meta_app::schema::database_name_ident::DatabaseNameIdent; use databend_common_meta_app::schema::dictionary_name_ident::DictionaryNameIdent; +use databend_common_meta_app::schema::least_visible_time_ident::LeastVisibleTimeIdent; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::CommitTableMetaReply; use databend_common_meta_app::schema::CommitTableMetaReq; @@ -66,6 +67,7 @@ use databend_common_meta_app::schema::GetSequenceReq; use databend_common_meta_app::schema::GetTableCopiedFileReply; use databend_common_meta_app::schema::GetTableCopiedFileReq; use databend_common_meta_app::schema::IndexMeta; +use databend_common_meta_app::schema::LeastVisibleTime; use databend_common_meta_app::schema::ListDictionaryReq; use databend_common_meta_app::schema::ListDroppedTableReq; use databend_common_meta_app::schema::ListIndexesByIdReq; @@ -699,6 +701,14 @@ impl Catalog for SessionCatalog { self.inner.list_dictionaries(req).await } + async fn set_table_lvt( + &self, + name_ident: &LeastVisibleTimeIdent, + value: &LeastVisibleTime, + ) -> Result { + self.inner.set_table_lvt(name_ident, value).await + } + async fn rename_dictionary(&self, req: RenameDictionaryReq) -> Result<()> { self.inner.rename_dictionary(req).await } diff --git a/src/query/service/src/interpreters/interpreter_copy_into_table.rs b/src/query/service/src/interpreters/interpreter_copy_into_table.rs index 2b7a4f5d08ff..7d6394d811bc 100644 --- a/src/query/service/src/interpreters/interpreter_copy_into_table.rs +++ b/src/query/service/src/interpreters/interpreter_copy_into_table.rs @@ -33,6 +33,7 @@ use databend_common_sql::executor::physical_plans::TableScan; use databend_common_sql::executor::table_read_plan::ToReadDataSourcePlan; use databend_common_sql::executor::PhysicalPlan; use databend_common_storage::StageFileInfo; +use databend_common_storages_fuse::FuseTable; use databend_common_storages_stage::StageTable; use log::debug; use log::info; @@ -105,6 +106,12 @@ impl CopyIntoTableInterpreter { &plan.table_name, ) .await?; + let snapshot = FuseTable::try_from_table(to_table.as_ref())? + .read_table_snapshot() + .await?; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(to_table.get_id(), snapshot)?; let mut update_stream_meta_reqs = vec![]; let (source, project_columns) = if let Some(ref query) = plan.query { let query = if plan.enable_distributed { @@ -160,6 +167,7 @@ impl CopyIntoTableInterpreter { project_columns, source, is_transform: plan.is_transform, + table_meta_timestamps, })); if plan.enable_distributed { @@ -251,6 +259,11 @@ impl CopyIntoTableInterpreter { &plan.stage_table_info.copy_into_table_options, )?; + let fuse_table = FuseTable::try_from_table(to_table.as_ref())?; + let table_meta_timestamps = ctx.get_table_meta_timestamps( + to_table.get_id(), + fuse_table.read_table_snapshot().await?, + )?; to_table.commit_insertion( ctx.clone(), main_pipeline, @@ -259,6 +272,7 @@ impl CopyIntoTableInterpreter { plan.write_mode.is_overwrite(), None, deduplicated_label, + table_meta_timestamps, )?; } diff --git a/src/query/service/src/interpreters/interpreter_insert.rs b/src/query/service/src/interpreters/interpreter_insert.rs index c218c481351a..809ffca778ca 100644 --- a/src/query/service/src/interpreters/interpreter_insert.rs +++ b/src/query/service/src/interpreters/interpreter_insert.rs @@ -107,6 +107,15 @@ impl Interpreter for InsertInterpreter { // check mutability table.check_mutable()?; + let table_meta_timestamps = if table.engine() == "FUSE" { + let fuse_table = + databend_common_storages_fuse::FuseTable::try_from_table(table.as_ref())?; + let snapshot = fuse_table.read_table_snapshot().await?; + self.ctx + .get_table_meta_timestamps(table.get_id(), snapshot)? + } else { + Default::default() + }; let mut build_res = PipelineBuildResult::create(); @@ -186,6 +195,7 @@ impl Interpreter for InsertInterpreter { select_column_bindings, insert_schema: self.plan.dest_schema(), cast_needed: self.check_schema_cast(plan)?, + table_meta_timestamps, }, ))); PhysicalPlan::Exchange(exchange.clone()) @@ -202,6 +212,7 @@ impl Interpreter for InsertInterpreter { select_column_bindings, insert_schema: self.plan.dest_schema(), cast_needed: self.check_schema_cast(plan)?, + table_meta_timestamps, })) } }; @@ -218,6 +229,7 @@ impl Interpreter for InsertInterpreter { self.plan.overwrite, None, unsafe { self.ctx.get_settings().get_deduplicate_label()? }, + table_meta_timestamps, )?; // Execute the hook operator. @@ -246,6 +258,7 @@ impl Interpreter for InsertInterpreter { vec![], self.plan.overwrite, unsafe { self.ctx.get_settings().get_deduplicate_label()? }, + table_meta_timestamps, )?; // Execute the hook operator. diff --git a/src/query/service/src/interpreters/interpreter_insert_multi_table.rs b/src/query/service/src/interpreters/interpreter_insert_multi_table.rs index 9b021672ad12..ca651899da40 100644 --- a/src/query/service/src/interpreters/interpreter_insert_multi_table.rs +++ b/src/query/service/src/interpreters/interpreter_insert_multi_table.rs @@ -48,6 +48,7 @@ use databend_common_sql::plans::Into; use databend_common_sql::plans::Plan; use databend_common_sql::MetadataRef; use databend_common_sql::ScalarExpr; +use databend_common_storages_fuse::FuseTable; use super::HookOperator; use crate::interpreters::common::dml_build_update_stream_req; @@ -380,9 +381,14 @@ impl InsertIntoBranches { for table in &self.tables { let table_info = table.get_table_info(); let catalog_info = ctx.get_catalog(table_info.catalog()).await?.info(); + let snapshot = FuseTable::try_from_table(table.as_ref())? + .read_table_snapshot() + .await?; + let table_meta_timestamps = ctx.get_table_meta_timestamps(table.get_id(), snapshot)?; serializable_tables.push(SerializableTable { target_catalog_info: catalog_info, target_table_info: table_info.clone(), + table_meta_timestamps, }); } Ok(serializable_tables) @@ -402,9 +408,14 @@ impl InsertIntoBranches { } last_table_id = Some(table_id); let catalog_info = ctx.get_catalog(table_info.catalog()).await?.info(); + let snapshot = FuseTable::try_from_table(table.as_ref())? + .read_table_snapshot() + .await?; + let table_meta_timestamps = ctx.get_table_meta_timestamps(table.get_id(), snapshot)?; serializable_tables.push(SerializableTable { target_catalog_info: catalog_info, target_table_info: table_info.clone(), + table_meta_timestamps, }); } Ok(serializable_tables) diff --git a/src/query/service/src/interpreters/interpreter_mutation.rs b/src/query/service/src/interpreters/interpreter_mutation.rs index c22ccdf93d19..9d3302974a29 100644 --- a/src/query/service/src/interpreters/interpreter_mutation.rs +++ b/src/query/service/src/interpreters/interpreter_mutation.rs @@ -194,11 +194,15 @@ impl MutationInterpreter { let partitions = self .mutation_source_partitions(mutation, fuse_table, table_snapshot.clone()) .await?; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(fuse_table.get_id(), table_snapshot.clone())?; Ok(MutationBuildInfo { table_info, table_snapshot, update_stream_meta, partitions, + table_meta_timestamps, }) } diff --git a/src/query/service/src/interpreters/interpreter_replace.rs b/src/query/service/src/interpreters/interpreter_replace.rs index 9d35f62e3818..cfd3257e7071 100644 --- a/src/query/service/src/interpreters/interpreter_replace.rs +++ b/src/query/service/src/interpreters/interpreter_replace.rs @@ -170,6 +170,10 @@ impl ReplaceInterpreter { let table_info = fuse_table.get_table_info(); let base_snapshot = fuse_table.read_table_snapshot().await?; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(table_info.ident.table_id, base_snapshot.clone())?; + let is_multi_node = !self.ctx.get_cluster().is_empty(); let is_value_source = matches!(self.plan.source, InsertInputSource::Values(_)); let is_distributed = is_multi_node @@ -333,6 +337,7 @@ impl ReplaceInterpreter { block_slots: None, need_insert: true, plan_id: u32::MAX, + table_meta_timestamps, }))); if is_distributed { @@ -355,6 +360,7 @@ impl ReplaceInterpreter { merge_meta: false, deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, plan_id: u32::MAX, + table_meta_timestamps, recluster_info: None, }))); root.adjust_plan_id(&mut 0); 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 e2493d9d3438..b8dd3ebae297 100644 --- a/src/query/service/src/interpreters/interpreter_table_add_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_add_column.rs @@ -109,7 +109,7 @@ impl Interpreter for AddTableColumnInterpreter { }; new_table_meta.add_column(&field, &self.plan.comment, index)?; - let _ = generate_new_snapshot(self.ctx.as_ref(), tbl.as_ref(), &mut new_table_meta).await?; + generate_new_snapshot(tbl.as_ref(), &mut new_table_meta, self.ctx.as_ref()).await?; let table_id = table_info.ident.table_id; let table_version = table_info.ident.seq; @@ -146,16 +146,17 @@ impl Interpreter for AddTableColumnInterpreter { } pub(crate) async fn generate_new_snapshot( - ctx: &QueryContext, 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? { - let mut new_snapshot = TableSnapshot::from_previous( - snapshot.as_ref(), + let mut new_snapshot = TableSnapshot::try_from_previous( + snapshot.clone(), Some(fuse_table.get_table_info().ident.seq), - ); + ctx.get_table_meta_timestamps(table.get_id(), Some(snapshot))?, + )?; // replace schema new_snapshot.schema = new_table_meta.schema.as_ref().clone(); 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 8c86699fd737..36d2d4267464 100644 --- a/src/query/service/src/interpreters/interpreter_table_drop_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_drop_column.rs @@ -129,7 +129,7 @@ impl Interpreter for DropTableColumnInterpreter { let table_id = table_info.ident.table_id; let table_version = table_info.ident.seq; - generate_new_snapshot(self.ctx.as_ref(), 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 d749c81bcc95..49222df271f1 100644 --- a/src/query/service/src/interpreters/interpreter_table_modify_column.rs +++ b/src/query/service/src/interpreters/interpreter_table_modify_column.rs @@ -165,10 +165,10 @@ impl ModifyTableColumnInterpreter { let catalog = self.ctx.get_catalog(catalog_name).await?; let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let prev_snapshot_id = fuse_table - .read_table_snapshot() - .await - .map_or(None, |v| v.map(|snapshot| snapshot.snapshot_id)); + let base_snapshot = fuse_table.read_table_snapshot().await?; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(fuse_table.get_id(), base_snapshot.clone())?; let mut bloom_index_cols = vec![]; if let Some(v) = table_info.options().get(OPT_KEY_BLOOM_INDEX_COLUMNS) { @@ -391,6 +391,7 @@ impl ModifyTableColumnInterpreter { select_column_bindings, insert_schema: Arc::new(Arc::new(new_schema).into()), cast_needed: true, + table_meta_timestamps, })); let mut build_res = build_query_pipeline_without_render_result_set(&self.ctx, &insert_plan).await?; @@ -402,8 +403,9 @@ impl ModifyTableColumnInterpreter { None, vec![], true, - prev_snapshot_id, + base_snapshot.as_ref().map(|b| b.snapshot_id), None, + table_meta_timestamps, )?; Ok(build_res) diff --git a/src/query/service/src/interpreters/interpreter_user_stage_remove.rs b/src/query/service/src/interpreters/interpreter_user_stage_remove.rs index 94fcf52dc825..5a395ee09bde 100644 --- a/src/query/service/src/interpreters/interpreter_user_stage_remove.rs +++ b/src/query/service/src/interpreters/interpreter_user_stage_remove.rs @@ -82,7 +82,7 @@ impl Interpreter for RemoveUserStageInterpreter { while let Some(chunk) = chunks.next().await { let chunk: Result> = chunk.into_iter().collect(); let chunk = chunk?.into_iter().map(|x| x.path).collect::>(); - if let Err(e) = file_op.remove_file_in_batch(&chunk).await { + if let Err(e) = file_op.remove_file_in_batch(&chunk, false).await { error!("Failed to delete file: {:?}, error: {}", chunk, e); } diff --git a/src/query/service/src/pipelines/builders/builder_append_table.rs b/src/query/service/src/pipelines/builders/builder_append_table.rs index 90e151df2550..c4c3347c0d9a 100644 --- a/src/query/service/src/pipelines/builders/builder_append_table.rs +++ b/src/query/service/src/pipelines/builders/builder_append_table.rs @@ -20,6 +20,7 @@ use databend_common_expression::DataSchemaRef; use databend_common_meta_app::schema::UpdateStreamMetaReq; use databend_common_meta_app::schema::UpsertTableCopiedFileReq; use databend_common_pipeline_core::Pipeline; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::pipelines::PipelineBuilder; use crate::sessions::QueryContext; @@ -36,11 +37,11 @@ impl PipelineBuilder { update_stream_meta: Vec, overwrite: bool, deduplicated_label: Option, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { Self::fill_and_reorder_columns(ctx.clone(), main_pipeline, table.clone(), source_schema)?; - table.append_data(ctx.clone(), main_pipeline)?; - + table.append_data(ctx.clone(), main_pipeline, table_meta_timestamps)?; table.commit_insertion( ctx, main_pipeline, @@ -49,6 +50,7 @@ impl PipelineBuilder { overwrite, None, deduplicated_label, + table_meta_timestamps, )?; Ok(()) @@ -59,10 +61,11 @@ impl PipelineBuilder { main_pipeline: &mut Pipeline, table: Arc, source_schema: DataSchemaRef, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { Self::fill_and_reorder_columns(ctx.clone(), main_pipeline, table.clone(), source_schema)?; - table.append_data(ctx, main_pipeline)?; + table.append_data(ctx, main_pipeline, table_meta_timestamps)?; Ok(()) } diff --git a/src/query/service/src/pipelines/builders/builder_column_mutation.rs b/src/query/service/src/pipelines/builders/builder_column_mutation.rs index 225fef28c138..a55412e78be5 100644 --- a/src/query/service/src/pipelines/builders/builder_column_mutation.rs +++ b/src/query/service/src/pipelines/builders/builder_column_mutation.rs @@ -62,6 +62,7 @@ impl PipelineBuilder { table, cluster_stats_gen.clone(), MutationKind::Delete, + column_mutation.table_meta_timestamps, )?; proc.into_processor() })?; @@ -81,6 +82,7 @@ impl PipelineBuilder { table, cluster_stats_gen.clone(), MutationKind::Update, + column_mutation.table_meta_timestamps, )?; proc.into_processor() })?; diff --git a/src/query/service/src/pipelines/builders/builder_commit.rs b/src/query/service/src/pipelines/builders/builder_commit.rs index e4becfd76f55..6123d6d9b84f 100644 --- a/src/query/service/src/pipelines/builders/builder_commit.rs +++ b/src/query/service/src/pipelines/builders/builder_commit.rs @@ -58,6 +58,7 @@ impl PipelineBuilder { recluster_info.removed_segment_indexes, recluster_info.removed_statistics, plan.mutation_kind, + plan.table_meta_timestamps, ) }); } @@ -74,6 +75,7 @@ impl PipelineBuilder { None, None, plan.deduplicated_label.clone(), + plan.table_meta_timestamps, ) }) } diff --git a/src/query/service/src/pipelines/builders/builder_compact.rs b/src/query/service/src/pipelines/builders/builder_compact.rs index 03fadf872e31..415769c70a7e 100644 --- a/src/query/service/src/pipelines/builders/builder_compact.rs +++ b/src/query/service/src/pipelines/builders/builder_compact.rs @@ -146,6 +146,7 @@ impl PipelineBuilder { table, cluster_stats_gen.clone(), MutationKind::Compact, + compact_block.table_meta_timestamps, )?; proc.into_processor() })?; @@ -161,6 +162,7 @@ impl PipelineBuilder { vec![], Default::default(), MutationKind::Compact, + compact_block.table_meta_timestamps, ) }); } diff --git a/src/query/service/src/pipelines/builders/builder_copy_into_location.rs b/src/query/service/src/pipelines/builders/builder_copy_into_location.rs index 5ebfa3b7d882..691b738c74ff 100644 --- a/src/query/service/src/pipelines/builders/builder_copy_into_location.rs +++ b/src/query/service/src/pipelines/builders/builder_copy_into_location.rs @@ -42,6 +42,7 @@ impl PipelineBuilder { vec![], false, unsafe { self.ctx.get_settings().get_deduplicate_label()? }, + Default::default(), ) } } diff --git a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs b/src/query/service/src/pipelines/builders/builder_copy_into_table.rs index a843cbba728b..11373fb8041d 100644 --- a/src/query/service/src/pipelines/builders/builder_copy_into_table.rs +++ b/src/query/service/src/pipelines/builders/builder_copy_into_table.rs @@ -179,6 +179,7 @@ impl PipelineBuilder { main_pipeline, to_table.clone(), plan_required_values_schema.clone(), + plan.table_meta_timestamps, )? } CopyIntoTableMode::Replace => {} @@ -187,6 +188,7 @@ impl PipelineBuilder { main_pipeline, to_table.clone(), plan_required_values_schema.clone(), + plan.table_meta_timestamps, )?, } Ok(()) diff --git a/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs b/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs index ef2bc29e131b..f90e8fbd3ac6 100644 --- a/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs +++ b/src/query/service/src/pipelines/builders/builder_distributed_insert_select.rs @@ -60,7 +60,11 @@ impl PipelineBuilder { source_schema.clone(), )?; - table.append_data(self.ctx.clone(), &mut self.main_pipeline)?; + table.append_data( + self.ctx.clone(), + &mut self.main_pipeline, + insert_select.table_meta_timestamps, + )?; Ok(()) } diff --git a/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs b/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs index c1772090b835..604276059587 100644 --- a/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs +++ b/src/query/service/src/pipelines/builders/builder_insert_multi_table.rs @@ -240,7 +240,11 @@ impl PipelineBuilder { sort_builders.push(Box::new(self.dummy_transform_builder()?)); } serialize_block_builders.push(Box::new( - self.with_tid_serialize_block_transform_builder(table, cluster_stats_gen)?, + self.with_tid_serialize_block_transform_builder( + table, + cluster_stats_gen, + append_data.table_meta_timestamps, + )?, )); } self.main_pipeline @@ -268,6 +272,7 @@ impl PipelineBuilder { deduplicated_label, targets, } = plan; + let mut table_meta_timestampss = HashMap::new(); self.build_pipeline(input)?; let mut serialize_segment_builders: Vec = Vec::with_capacity(targets.len()); @@ -279,12 +284,18 @@ impl PipelineBuilder { .ctx .build_table_by_table_info(&target.target_table_info, None)?; let block_thresholds = table.get_block_thresholds(); - serialize_segment_builders.push(Box::new( - self.serialize_segment_transform_builder(table.clone(), block_thresholds)?, - )); + serialize_segment_builders.push(Box::new(self.serialize_segment_transform_builder( + table.clone(), + block_thresholds, + target.table_meta_timestamps, + )?)); mutation_aggregator_builders.push(Box::new( - self.mutation_aggregator_transform_builder(table.clone())?, + self.mutation_aggregator_transform_builder( + table.clone(), + target.table_meta_timestamps, + )?, )); + table_meta_timestampss.insert(table.get_id(), target.table_meta_timestamps); tables.insert(table.get_id(), table); } self.main_pipeline @@ -306,6 +317,7 @@ impl PipelineBuilder { update_stream_meta.clone(), deduplicated_label.clone(), catalog.clone(), + table_meta_timestampss.clone(), ), ))) })?; diff --git a/src/query/service/src/pipelines/builders/builder_mutation.rs b/src/query/service/src/pipelines/builders/builder_mutation.rs index 3cd64c367fc5..7425152d3ed5 100644 --- a/src/query/service/src/pipelines/builders/builder_mutation.rs +++ b/src/query/service/src/pipelines/builders/builder_mutation.rs @@ -66,6 +66,7 @@ impl PipelineBuilder { OutputPort::create(), table, block_thresholds, + merge_into.table_meta_timestamps, ); // For row_id port, create rowid_aggregate_mutator @@ -105,6 +106,7 @@ impl PipelineBuilder { io_request_semaphore, merge_into.segments.clone(), false, + merge_into.table_meta_timestamps, )?); } @@ -117,6 +119,7 @@ impl PipelineBuilder { table, cluster_stats_gen.clone(), MutationKind::MergeInto, + merge_into.table_meta_timestamps, )?; pipe_items.push(serialize_block_transform.into_pipe_item()); } diff --git a/src/query/service/src/pipelines/builders/builder_on_finished.rs b/src/query/service/src/pipelines/builders/builder_on_finished.rs index a3b69a8a5a4d..8b74415abe59 100644 --- a/src/query/service/src/pipelines/builders/builder_on_finished.rs +++ b/src/query/service/src/pipelines/builders/builder_on_finished.rs @@ -118,7 +118,7 @@ impl PipelineBuilder { match op { Ok(op) => { let file_op = Files::create(table_ctx, op); - if let Err(e) = file_op.remove_file_in_batch(files).await { + if let Err(e) = file_op.remove_file_in_batch(files, false).await { error!("Failed to delete file: {:?}, error: {}", files, e); } } diff --git a/src/query/service/src/pipelines/builders/builder_recluster.rs b/src/query/service/src/pipelines/builders/builder_recluster.rs index dfc414490f9b..cad5888bfd1d 100644 --- a/src/query/service/src/pipelines/builders/builder_recluster.rs +++ b/src/query/service/src/pipelines/builders/builder_recluster.rs @@ -158,6 +158,7 @@ impl PipelineBuilder { table, cluster_stats_gen.clone(), MutationKind::Recluster, + recluster.table_meta_timestamps, )?; proc.into_processor() }) diff --git a/src/query/service/src/pipelines/builders/builder_replace_into.rs b/src/query/service/src/pipelines/builders/builder_replace_into.rs index ad3fd8854904..5e7dedd0e32e 100644 --- a/src/query/service/src/pipelines/builders/builder_replace_into.rs +++ b/src/query/service/src/pipelines/builders/builder_replace_into.rs @@ -128,6 +128,7 @@ impl PipelineBuilder { table, cluster_stats_gen, MutationKind::Replace, + replace.table_meta_timestamps, )?; let mut block_builder = serialize_block_transform.get_block_builder(); block_builder.source_schema = table.schema_with_stream(); @@ -137,6 +138,7 @@ impl PipelineBuilder { OutputPort::create(), table, *block_thresholds, + replace.table_meta_timestamps, ); if !*need_insert { if segment_partition_num == 0 { diff --git a/src/query/service/src/pipelines/builders/transform_builder.rs b/src/query/service/src/pipelines/builders/transform_builder.rs index a78a87ba62d9..1d4aa0ec84eb 100644 --- a/src/query/service/src/pipelines/builders/transform_builder.rs +++ b/src/query/service/src/pipelines/builders/transform_builder.rs @@ -39,7 +39,9 @@ use databend_common_storages_fuse::operations::TransformSerializeBlock; use databend_common_storages_fuse::operations::TransformSerializeSegment; use databend_common_storages_fuse::statistics::ClusterStatsGenerator; use databend_common_storages_fuse::FuseTable; +use databend_storages_common_table_meta::meta; use databend_storages_common_table_meta::meta::Statistics; +use meta::TableMetaTimestamps; use crate::pipelines::processors::transforms::TransformFilter; use crate::pipelines::processors::InputPort; @@ -117,6 +119,7 @@ impl PipelineBuilder { &self, table: Arc, cluster_stats_gen: ClusterStatsGenerator, + table_meta_timestamps: TableMetaTimestamps, ) -> Result, Arc) -> Result> { let ctx = self.ctx.clone(); Ok(move |input, output| { @@ -128,6 +131,7 @@ impl PipelineBuilder { fuse_table, cluster_stats_gen.clone(), MutationKind::Insert, + table_meta_timestamps, )?; proc.into_processor() }) @@ -137,10 +141,17 @@ impl PipelineBuilder { &self, table: Arc, block_thresholds: BlockThresholds, + table_meta_timestamps: TableMetaTimestamps, ) -> Result, Arc) -> Result> { Ok(move |input, output| { let fuse_table = FuseTable::try_from_table(table.as_ref())?; - let proc = TransformSerializeSegment::new(input, output, fuse_table, block_thresholds); + let proc = TransformSerializeSegment::new( + input, + output, + fuse_table, + block_thresholds, + table_meta_timestamps, + ); proc.into_processor() }) } @@ -148,6 +159,7 @@ impl PipelineBuilder { pub(crate) fn mutation_aggregator_transform_builder( &self, table: Arc, + table_meta_timestamps: TableMetaTimestamps, ) -> Result, Arc) -> Result> { let ctx = self.ctx.clone(); Ok(move |input, output| { @@ -160,6 +172,7 @@ impl PipelineBuilder { vec![], Statistics::default(), MutationKind::Insert, + table_meta_timestamps, ); Ok(ProcessorPtr::create(AsyncAccumulatingTransformer::create( input, output, aggregator, diff --git a/src/query/service/src/sessions/query_ctx.rs b/src/query/service/src/sessions/query_ctx.rs index c372651b9dd7..21c8d0080e5a 100644 --- a/src/query/service/src/sessions/query_ctx.rs +++ b/src/query/service/src/sessions/query_ctx.rs @@ -112,6 +112,8 @@ use databend_storages_common_session::drop_table_by_id; use databend_storages_common_session::SessionState; use databend_storages_common_session::TxnManagerRef; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; use jiff::tz::TimeZone; use jiff::Zoned; @@ -1349,6 +1351,29 @@ impl TableContext for QueryContext { self.shared.session.session_ctx.session_state() } + fn get_table_meta_timestamps( + &self, + table_id: u64, + previous_snapshot: Option>, + ) -> Result { + let cache = self.shared.get_table_meta_timestamps(); + let cached_item = cache.lock().get(&table_id).copied(); + match cached_item { + Some(ts) => Ok(ts), + None => { + let ts = self.txn_mgr().lock().get_table_meta_timestamps( + table_id, + previous_snapshot, + self.get_settings() + .get_max_retryable_transaction_duration_in_hours()? + as i64, + ); + cache.lock().insert(table_id, ts); + Ok(ts) + } + } + } + fn get_read_block_thresholds(&self) -> BlockThresholds { *self.block_threshold.read() } diff --git a/src/query/service/src/sessions/query_ctx_shared.rs b/src/query/service/src/sessions/query_ctx_shared.rs index d5da13802173..99e8fa8c828b 100644 --- a/src/query/service/src/sessions/query_ctx_shared.rs +++ b/src/query/service/src/sessions/query_ctx_shared.rs @@ -53,6 +53,7 @@ use databend_common_storage::MutationStatus; use databend_common_storage::StorageMetrics; use databend_common_storages_stream::stream_table::StreamTable; use databend_common_users::UserApiProvider; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use parking_lot::Mutex; use parking_lot::RwLock; use uuid::Uuid; @@ -142,6 +143,7 @@ pub struct QueryContextShared { pub(in crate::sessions) query_cache_metrics: DataCacheMetrics, pub(in crate::sessions) query_queued_duration: Arc>, + pub(in crate::sessions) table_meta_timestamps: Arc>>, pub(in crate::sessions) spilled_files: Arc>>, @@ -201,6 +203,7 @@ impl QueryContextShared { merge_into_join: Default::default(), multi_table_insert_status: Default::default(), query_queued_duration: Arc::new(RwLock::new(Duration::from_secs(0))), + table_meta_timestamps: Arc::new(Mutex::new(HashMap::new())), spilled_files: Default::default(), })) } @@ -643,6 +646,10 @@ impl QueryContextShared { }; } } + + pub fn get_table_meta_timestamps(&self) -> Arc>> { + self.table_meta_timestamps.clone() + } } impl Drop for QueryContextShared { diff --git a/src/query/service/src/table_functions/fuse_vacuum2/fuse_vacuum2_table.rs b/src/query/service/src/table_functions/fuse_vacuum2/fuse_vacuum2_table.rs new file mode 100644 index 000000000000..4de5cc879014 --- /dev/null +++ b/src/query/service/src/table_functions/fuse_vacuum2/fuse_vacuum2_table.rs @@ -0,0 +1,204 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use databend_common_catalog::catalog::Catalog; +use databend_common_catalog::catalog_kind::CATALOG_DEFAULT; +use databend_common_catalog::plan::DataSourcePlan; +use databend_common_catalog::table::TableExt; +use databend_common_catalog::table_args::TableArgs; +use databend_common_exception::ErrorCode; +use databend_common_exception::Result; +use databend_common_expression::types::StringType; +use databend_common_expression::DataBlock; +use databend_common_expression::FromData; +use databend_common_expression::TableDataType; +use databend_common_expression::TableField; +use databend_common_expression::TableSchemaRef; +use databend_common_expression::TableSchemaRefExt; +use databend_common_license::license::Feature::Vacuum; +use databend_common_license::license_manager::LicenseManagerSwitch; +use databend_common_storages_fuse::table_functions::bool_literal; +use databend_common_storages_fuse::table_functions::bool_value; +use databend_common_storages_fuse::table_functions::parse_db_tb_args; +use databend_common_storages_fuse::table_functions::string_literal; +use databend_common_storages_fuse::table_functions::string_value; +use databend_common_storages_fuse::table_functions::SimpleTableFunc; +use databend_common_storages_fuse::FuseTable; +use databend_enterprise_vacuum_handler::get_vacuum_handler; +use databend_enterprise_vacuum_handler::VacuumHandlerWrapper; + +use crate::sessions::TableContext; + +enum Vacuum2TableArgs { + SingleTable { + arg_database_name: String, + arg_table_name: String, + respect_flash_back: Option, + }, + All, +} + +impl From<&Vacuum2TableArgs> for TableArgs { + fn from(value: &Vacuum2TableArgs) -> Self { + match value { + Vacuum2TableArgs::SingleTable { + arg_database_name, + arg_table_name, + respect_flash_back, + } => TableArgs::new_positioned(vec![ + string_literal(arg_database_name.as_str()), + string_literal(arg_table_name.as_str()), + bool_literal(respect_flash_back.unwrap_or_default()), + ]), + Vacuum2TableArgs::All => TableArgs::new_positioned(vec![]), + } + } +} + +pub struct FuseVacuum2Table { + args: Vacuum2TableArgs, + handler: Arc, +} + +#[async_trait::async_trait] +impl SimpleTableFunc for FuseVacuum2Table { + fn table_args(&self) -> Option { + Some((&self.args).into()) + } + + fn schema(&self) -> TableSchemaRef { + TableSchemaRefExt::create(vec![TableField::new("vacuumed", TableDataType::String)]) + } + + async fn apply( + &self, + ctx: &Arc, + _: &DataSourcePlan, + ) -> Result> { + LicenseManagerSwitch::instance().check_enterprise_enabled(ctx.get_license_key(), Vacuum)?; + + let catalog = ctx.get_catalog(CATALOG_DEFAULT).await?; + let res = match &self.args { + Vacuum2TableArgs::SingleTable { + arg_database_name, + arg_table_name, + respect_flash_back, + } => { + self.apply_single_table( + ctx, + catalog.as_ref(), + arg_database_name, + arg_table_name, + respect_flash_back.unwrap_or_default(), + ) + .await? + } + Vacuum2TableArgs::All => self.apply_all_tables(ctx, catalog.as_ref()).await?, + }; + Ok(Some(DataBlock::new_from_columns(vec![ + StringType::from_data(res), + ]))) + } + + fn create(func_name: &str, table_args: TableArgs) -> Result + where Self: Sized { + let args = match table_args.positioned.len() { + 0 => Vacuum2TableArgs::All, + 2 => { + let (arg_database_name, arg_table_name) = parse_db_tb_args(&table_args, func_name)?; + Vacuum2TableArgs::SingleTable { + arg_database_name, + arg_table_name, + respect_flash_back: None, + } + } + 3 => { + let args = table_args.expect_all_positioned(func_name, None)?; + + let arg_database_name = string_value(&args[0])?; + let arg_table_name = string_value(&args[1])?; + let arg_respect_flash_back = bool_value(&args[2])?; + Vacuum2TableArgs::SingleTable { + arg_database_name, + arg_table_name, + respect_flash_back: Some(arg_respect_flash_back), + } + } + _ => { + return Err(ErrorCode::NumberArgumentsNotMatch( + "Expected 0 or 2 arguments".to_string(), + )); + } + }; + Ok(Self { + args, + handler: get_vacuum_handler(), + }) + } +} + +impl FuseVacuum2Table { + async fn apply_single_table( + &self, + ctx: &Arc, + catalog: &dyn Catalog, + database_name: &str, + table_name: &str, + respect_flash_back: bool, + ) -> Result> { + let tbl = catalog + .get_table(&ctx.get_tenant(), database_name, table_name) + .await?; + + let tbl = FuseTable::try_from_table(tbl.as_ref()).map_err(|_| { + ErrorCode::StorageOther("Invalid table engine, only fuse table is supported") + })?; + + tbl.check_mutable()?; + + self.handler + .do_vacuum2(tbl, ctx.clone(), respect_flash_back) + .await + } + + async fn apply_all_tables( + &self, + ctx: &Arc, + catalog: &dyn Catalog, + ) -> Result> { + let tenant_id = ctx.get_tenant(); + let dbs = catalog.list_databases(&tenant_id).await?; + for db in dbs { + if db.engine() != "DEFAULT" { + continue; + } + let tables = catalog.list_tables(&tenant_id, db.name()).await?; + for table in tables { + let tbl = FuseTable::try_from_table(table.as_ref()).map_err(|_| { + ErrorCode::StorageOther("Invalid table engine, only fuse table is supported") + })?; + + if table.is_read_only() { + continue; + } + + let _ = self.handler.do_vacuum2(tbl, ctx.clone(), false).await?; + } + } + + Ok(vec![]) + } +} diff --git a/src/query/service/src/table_functions/fuse_vacuum2/mod.rs b/src/query/service/src/table_functions/fuse_vacuum2/mod.rs new file mode 100644 index 000000000000..6560a4acec71 --- /dev/null +++ b/src/query/service/src/table_functions/fuse_vacuum2/mod.rs @@ -0,0 +1,17 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod fuse_vacuum2_table; + +pub use fuse_vacuum2_table::FuseVacuum2Table; diff --git a/src/query/service/src/table_functions/mod.rs b/src/query/service/src/table_functions/mod.rs index 2bef2a7cf428..fc672e3af1d6 100644 --- a/src/query/service/src/table_functions/mod.rs +++ b/src/query/service/src/table_functions/mod.rs @@ -14,6 +14,7 @@ mod async_crash_me; mod cloud; +mod fuse_vacuum2; mod infer_schema; mod inspect_parquet; mod list_stage; diff --git a/src/query/service/src/table_functions/table_function_factory.rs b/src/query/service/src/table_functions/table_function_factory.rs index 019d01f4e7e3..87e8d4dabcae 100644 --- a/src/query/service/src/table_functions/table_function_factory.rs +++ b/src/query/service/src/table_functions/table_function_factory.rs @@ -46,6 +46,7 @@ use crate::table_functions::async_crash_me::AsyncCrashMeTable; use crate::table_functions::cloud::TaskDependentsEnableTable; use crate::table_functions::cloud::TaskDependentsTable; use crate::table_functions::cloud::TaskHistoryTable; +use crate::table_functions::fuse_vacuum2::FuseVacuum2Table; use crate::table_functions::infer_schema::InferSchemaTable; use crate::table_functions::inspect_parquet::InspectParquetTable; use crate::table_functions::list_stage::ListStageTable; @@ -56,7 +57,6 @@ use crate::table_functions::srf::RangeTable; use crate::table_functions::sync_crash_me::SyncCrashMeTable; use crate::table_functions::GPT2SQLTable; use crate::table_functions::TableFunction; - type TableFunctionCreators = RwLock)>>; pub trait TableFunctionCreator: Send + Sync { @@ -146,6 +146,14 @@ impl TableFunctionFactory { ), ); + creators.insert( + "fuse_vacuum2".to_string(), + ( + next_id(), + Arc::new(TableFunctionTemplate::::create), + ), + ); + creators.insert( "fuse_block".to_string(), ( diff --git a/src/query/service/src/test_kits/block_writer.rs b/src/query/service/src/test_kits/block_writer.rs index 39912a48534f..68a5d829e1a8 100644 --- a/src/query/service/src/test_kits/block_writer.rs +++ b/src/query/service/src/test_kits/block_writer.rs @@ -31,24 +31,32 @@ use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::Compression; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::StatisticsOfColumns; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; use opendal::Operator; use parquet::format::FileMetaData; use uuid::Uuid; +use super::old_version_generator; pub struct BlockWriter<'a> { location_generator: &'a TableMetaLocationGenerator, data_accessor: &'a Operator, + table_meta_timestamps: TableMetaTimestamps, + is_greater_than_v5: bool, } impl<'a> BlockWriter<'a> { pub fn new( data_accessor: &'a Operator, location_generator: &'a TableMetaLocationGenerator, + table_meta_timestamps: TableMetaTimestamps, + is_greater_than_v5: bool, ) -> Self { Self { location_generator, data_accessor, + table_meta_timestamps, + is_greater_than_v5, } } @@ -60,7 +68,15 @@ impl<'a> BlockWriter<'a> { col_stats: StatisticsOfColumns, cluster_stats: Option, ) -> Result<(BlockMeta, Option)> { - let (location, block_id) = self.location_generator.gen_block_location(); + let (location, block_id) = if !self.is_greater_than_v5 { + let location_generator = old_version_generator::TableMetaLocationGenerator::with_prefix( + self.location_generator.prefix().to_string(), + ); + location_generator.gen_block_location(self.table_meta_timestamps) + } else { + self.location_generator + .gen_block_location(self.table_meta_timestamps) + }; let data_accessor = &self.data_accessor; let row_count = block.num_rows() as u64; diff --git a/src/query/service/src/test_kits/fixture.rs b/src/query/service/src/test_kits/fixture.rs index 7c8f66beb993..023fb2a37572 100644 --- a/src/query/service/src/test_kits/fixture.rs +++ b/src/query/service/src/test_kits/fixture.rs @@ -841,7 +841,11 @@ impl TestFixture { data_schema, )?; - table.append_data(ctx.clone(), &mut build_res.main_pipeline)?; + table.append_data( + ctx.clone(), + &mut build_res.main_pipeline, + Default::default(), + )?; if commit { table.commit_insertion( ctx.clone(), @@ -851,6 +855,7 @@ impl TestFixture { overwrite, None, None, + Default::default(), )?; } else { build_res diff --git a/src/query/service/src/test_kits/fuse.rs b/src/query/service/src/test_kits/fuse.rs index cc4996a8754d..747aed7ad32e 100644 --- a/src/query/service/src/test_kits/fuse.rs +++ b/src/query/service/src/test_kits/fuse.rs @@ -37,10 +37,12 @@ use databend_common_storages_fuse::FuseTable; use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; use databend_storages_common_table_meta::meta::testing::SegmentInfoV2; use databend_storages_common_table_meta::meta::testing::TableSnapshotV2; +use databend_storages_common_table_meta::meta::testing::TableSnapshotV4; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use futures_util::TryStreamExt; @@ -49,6 +51,7 @@ use serde::Serialize; use uuid::Uuid; use super::block_writer::BlockWriter; +use super::old_version_generator; use super::TestFixture; use crate::interpreters::Interpreter; use crate::interpreters::MutationInterpreter; @@ -63,10 +66,11 @@ pub async fn generate_snapshot_with_segments( let current_snapshot = fuse_table.read_table_snapshot().await?.unwrap(); let operator = fuse_table.get_operator(); let location_gen = fuse_table.meta_location_generator(); - let mut new_snapshot = TableSnapshot::from_previous( - current_snapshot.as_ref(), + let mut new_snapshot = TableSnapshot::try_from_previous( + current_snapshot, Some(fuse_table.get_table_info().ident.seq), - ); + Default::default(), + )?; new_snapshot.segments = segment_locations; let new_snapshot_location = location_gen .snapshot_location_from_uuid(&new_snapshot.snapshot_id, TableSnapshot::VERSION)?; @@ -102,7 +106,8 @@ pub async fn generate_segments_v2( let mut segs = vec![]; for _ in 0..number_of_segments { let dal = fuse_table.get_operator_ref(); - let block_metas = generate_blocks(fuse_table, blocks_per_segment).await?; + let block_metas = + generate_blocks(fuse_table, blocks_per_segment, false, Default::default()).await?; let summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); let segment_info = SegmentInfoV2::new(block_metas, summary); let uuid = Uuid::new_v4(); @@ -123,26 +128,51 @@ pub async fn generate_segments( fuse_table: &FuseTable, number_of_segments: usize, blocks_per_segment: usize, + is_greater_than_v5: bool, + table_meta_timestamps: TableMetaTimestamps, ) -> Result> { let mut segs = vec![]; + let location_generator = fuse_table.meta_location_generator(); for _ in 0..number_of_segments { let dal = fuse_table.get_operator_ref(); - let block_metas = generate_blocks(fuse_table, blocks_per_segment).await?; + let block_metas = generate_blocks( + fuse_table, + blocks_per_segment, + is_greater_than_v5, + table_meta_timestamps, + ) + .await?; let summary = reduce_block_metas(&block_metas, BlockThresholds::default(), None); let segment_info = SegmentInfo::new(block_metas, summary); - let location = fuse_table - .meta_location_generator() - .gen_segment_info_location(); - segment_info.write_meta(dal, &location).await?; + let location = if is_greater_than_v5 { + location_generator.gen_segment_info_location(table_meta_timestamps) + } else { + let location_generator = old_version_generator::TableMetaLocationGenerator::with_prefix( + location_generator.prefix().to_string(), + ); + location_generator.gen_segment_info_location(table_meta_timestamps) + }; + segment_info.write_meta(dal, location.as_str()).await?; segs.push(((location, SegmentInfo::VERSION), segment_info)) } Ok(segs) } -async fn generate_blocks(fuse_table: &FuseTable, num_blocks: usize) -> Result>> { +async fn generate_blocks( + fuse_table: &FuseTable, + num_blocks: usize, + is_greater_than_v5: bool, + table_meta_timestamps: TableMetaTimestamps, +) -> Result>> { let dal = fuse_table.get_operator_ref(); let schema = fuse_table.schema(); - let block_writer = BlockWriter::new(dal, fuse_table.meta_location_generator()); + let location_generator = fuse_table.meta_location_generator(); + let block_writer = BlockWriter::new( + dal, + location_generator, + table_meta_timestamps, + is_greater_than_v5, + ); let mut block_metas = vec![]; // does not matter in this suite @@ -197,21 +227,27 @@ pub async fn generate_snapshots(fixture: &TestFixture) -> Result<()> { // generate 2 segments, 4 blocks. let num_of_segments = 2; let blocks_per_segment = 2; - let segments_v3 = generate_segments(fuse_table, num_of_segments, blocks_per_segment).await?; + let segments_v3 = generate_segments( + fuse_table, + num_of_segments, + blocks_per_segment, + false, + Default::default(), + ) + .await?; // create snapshot 1, the format version is 3. let locations = vec![segments_v3[0].0.clone(), segments_v2[0].0.clone()]; - let mut snapshot_1 = TableSnapshot::new( - Uuid::new_v4(), + let mut snapshot_1 = TableSnapshot::try_new( + None, None, - &snapshot_0.timestamp, - Some((snapshot_0.snapshot_id, TableSnapshotV2::VERSION)), schema.as_ref().clone(), Statistics::default(), locations, None, None, - ); + Default::default(), + )?; snapshot_1.timestamp = Some(now - Duration::hours(12)); snapshot_1.summary = merge_statistics(snapshot_0.summary.clone(), &segments_v3[0].1.summary, None); @@ -227,7 +263,8 @@ pub async fn generate_snapshots(fixture: &TestFixture) -> Result<()> { segments_v3[0].0.clone(), segments_v2[0].0.clone(), ]; - let mut snapshot_2 = TableSnapshot::from_previous(&snapshot_1, None); + let mut snapshot_2 = + TableSnapshot::try_from_previous(Arc::new(snapshot_1.clone()), None, Default::default())?; snapshot_2.segments = locations; snapshot_2.timestamp = Some(now); snapshot_2.summary = @@ -335,3 +372,74 @@ pub async fn append_computed_sample_data(num_blocks: usize, fixture: &TestFixtur .append_commit_blocks(table.clone(), blocks, true, true) .await } + +pub async fn generate_snapshot_v2( + fixture: &TestFixture, + number_of_segments: usize, + blocks_per_segment: usize, + prev: Option<&TableSnapshot>, +) -> Result { + let schema = TestFixture::default_table_schema(); + let table = fixture.latest_default_table().await?; + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let location_gen = fuse_table.meta_location_generator(); + let operator = fuse_table.get_operator(); + + let segments = generate_segments_v2(fuse_table, number_of_segments, blocks_per_segment).await?; + + let id = Uuid::new_v4(); + let snapshot = TableSnapshotV2::new( + id, + &prev.as_ref().and_then(|p| p.timestamp), + prev.map(|p| (p.snapshot_id, p.format_version)), + schema.as_ref().clone(), + Statistics::default(), + segments.iter().map(|s| s.0.clone()).collect(), + None, + None, + ); + + let new_snapshot_location = location_gen + .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshotV2::VERSION)?; + write_v2_to_storage(&operator, &new_snapshot_location, &snapshot).await?; + Ok(snapshot.into()) +} + +pub async fn generate_snapshot_v4( + fixture: &TestFixture, + number_of_segments: usize, + blocks_per_segment: usize, + prev: Option<&TableSnapshot>, +) -> Result { + let schema = TestFixture::default_table_schema(); + let table = fixture.latest_default_table().await?; + let fuse_table = FuseTable::try_from_table(table.as_ref())?; + let location_gen = fuse_table.meta_location_generator(); + let operator = fuse_table.get_operator(); + + let segments = generate_segments( + fuse_table, + number_of_segments, + blocks_per_segment, + false, + Default::default(), + ) + .await?; + + let snapshot = TableSnapshotV4::try_new( + None, + prev.map(|p| Arc::new(p.clone())), + schema.as_ref().clone(), + Statistics::default(), + segments.iter().map(|s| s.0.clone()).collect(), + None, + None, + Default::default(), + )?; + let new_snapshot_location = location_gen + .snapshot_location_from_uuid(&snapshot.snapshot_id, TableSnapshotV4::VERSION)?; + operator + .write(&new_snapshot_location, snapshot.to_bytes()?) + .await?; + Ok(snapshot) +} diff --git a/src/query/service/src/test_kits/mod.rs b/src/query/service/src/test_kits/mod.rs index 0ee44f0d8dec..e5a1f7102d2f 100644 --- a/src/query/service/src/test_kits/mod.rs +++ b/src/query/service/src/test_kits/mod.rs @@ -21,6 +21,7 @@ pub mod config; mod context; mod fixture; mod fuse; +mod old_version_generator; pub use block_writer::BlockWriter; pub use check::*; diff --git a/src/query/service/src/test_kits/old_version_generator.rs b/src/query/service/src/test_kits/old_version_generator.rs new file mode 100644 index 000000000000..f56988a9e951 --- /dev/null +++ b/src/query/service/src/test_kits/old_version_generator.rs @@ -0,0 +1,63 @@ +// Copyright 2021 Datafuse Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use databend_common_expression::DataBlock; +use databend_common_storages_fuse::FUSE_TBL_BLOCK_PREFIX; +use databend_common_storages_fuse::FUSE_TBL_SEGMENT_PREFIX; +use databend_storages_common_table_meta::meta::uuid_from_date_time; +use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::Versioned; +use databend_storages_common_table_meta::meta::VACUUM2_OBJECT_KEY_PREFIX; +use uuid::Uuid; +#[derive(Clone)] +pub struct TableMetaLocationGenerator { + prefix: String, +} + +impl TableMetaLocationGenerator { + pub fn with_prefix(prefix: String) -> Self { + Self { prefix } + } + + pub fn gen_block_location( + &self, + table_meta_timestamps: TableMetaTimestamps, + ) -> (Location, Uuid) { + let part_uuid = uuid_from_date_time(table_meta_timestamps.segment_block_timestamp); + let location_path = format!( + "{}/{}/{}{}_v{}.parquet", + &self.prefix, + FUSE_TBL_BLOCK_PREFIX, + VACUUM2_OBJECT_KEY_PREFIX, + part_uuid.as_simple(), + DataBlock::VERSION, + ); + + ((location_path, DataBlock::VERSION), part_uuid) + } + + pub fn gen_segment_info_location(&self, table_meta_timestamps: TableMetaTimestamps) -> String { + let segment_uuid = uuid_from_date_time(table_meta_timestamps.segment_block_timestamp); + format!( + "{}/{}/{}{}_v{}.mpk", + &self.prefix, + FUSE_TBL_SEGMENT_PREFIX, + VACUUM2_OBJECT_KEY_PREFIX, + segment_uuid.as_simple(), + SegmentInfo::VERSION, + ) + } +} 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 e8679f31baf2..bcb34a89a5c8 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 @@ -146,6 +146,8 @@ use databend_query::test_kits::*; use databend_storages_common_session::SessionState; use databend_storages_common_session::TxnManagerRef; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::TableSnapshot; use parking_lot::Mutex; use parking_lot::RwLock; use xorf::BinaryFuse16; @@ -984,6 +986,15 @@ impl TableContext for CtxDelegation { todo!() } + fn get_table_meta_timestamps( + &self, + table_id: u64, + previous_snapshot: Option>, + ) -> Result { + self.ctx + .get_table_meta_timestamps(table_id, previous_snapshot) + } + fn get_temp_table_prefix(&self) -> Result { todo!() } diff --git a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs index 9f4997981964..d6cbb0b757e9 100644 --- a/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs +++ b/src/query/service/tests/it/storages/fuse/bloom_index_meta_size.rs @@ -125,7 +125,7 @@ use uuid::Uuid; #[ignore] async fn test_random_location_memory_size() -> databend_common_exception::Result<()> { // generate random location of Type Location - let location_gen = TableMetaLocationGenerator::with_prefix("/root".to_string()); + let location_gen = TableMetaLocationGenerator::new("/root".to_string()); let num_segments = 5_000_000; let sys = System::new_all(); @@ -141,7 +141,7 @@ async fn test_random_location_memory_size() -> databend_common_exception::Result let mut locations: HashSet = HashSet::new(); for _ in 0..num_segments { - let segment_path = location_gen.gen_segment_info_location(); + let segment_path = location_gen.gen_segment_info_location(Default::default()); let segment_location = (segment_path, SegmentInfo::VERSION); locations.insert(segment_location); } @@ -320,9 +320,9 @@ fn build_test_segment_info( } assert_eq!(num_number_columns + num_string_columns, col_stats.len()); - let location_gen = TableMetaLocationGenerator::with_prefix("/root/12345/67890".to_owned()); + let location_gen = TableMetaLocationGenerator::new("/root/12345/67890".to_owned()); - let (block_location, block_uuid) = location_gen.gen_block_location(); + let (block_location, block_uuid) = location_gen.gen_block_location(Default::default()); let block_meta = BlockMeta { row_count: 0, block_size: 0, @@ -382,9 +382,9 @@ async fn setup() -> databend_common_exception::Result { let block = DataBlock::new_from_columns(columns); let operator = Operator::new(opendal::services::Memory::default())?.finish(); - let loc_generator = TableMetaLocationGenerator::with_prefix("/".to_owned()); + let loc_generator = TableMetaLocationGenerator::new("/".to_owned()); let col_stats = gen_columns_statistics(&block, None, &schema)?; - let block_writer = BlockWriter::new(&operator, &loc_generator); + let block_writer = BlockWriter::new(&operator, &loc_generator, Default::default(), true); let (_block_meta, thrift_file_meta) = block_writer .write(FuseStorageFormat::Parquet, &schema, block, col_stats, None) .await?; diff --git a/src/query/service/tests/it/storages/fuse/conflict.rs b/src/query/service/tests/it/storages/fuse/conflict.rs index b615a9a56667..5978574deee6 100644 --- a/src/query/service/tests/it/storages/fuse/conflict.rs +++ b/src/query/service/tests/it/storages/fuse/conflict.rs @@ -26,7 +26,6 @@ use databend_storages_common_session::TxnManager; use databend_storages_common_table_meta::meta::Statistics; use databend_storages_common_table_meta::meta::TableSnapshot; -#[test] /// base snapshot contains segments 1, 2, 3, /// /// a delete operation wants to remove segment 2, @@ -36,6 +35,7 @@ use databend_storages_common_table_meta::meta::TableSnapshot; /// i.e. in this test, segment 2 and 3 are compacted into segment 4 /// /// so the delete operation cannot be applied +#[test] fn test_unresolvable_delete_conflict() { let mut base_snapshot = TableSnapshot::new_empty_snapshot(TableSchema::default(), None); base_snapshot.segments = vec![ @@ -65,6 +65,7 @@ fn test_unresolvable_delete_conflict() { None, TxnManager::init(), 0, + Default::default(), "test", ); assert!(result.is_err()); @@ -157,6 +158,7 @@ fn test_resolvable_delete_conflict() { None, TxnManager::init(), 0, + Default::default(), "test", ); let snapshot = result.unwrap(); @@ -265,6 +267,7 @@ fn test_resolvable_replace_conflict() { None, TxnManager::init(), 0, + Default::default(), "test", ); let snapshot = result.unwrap(); diff --git a/src/query/service/tests/it/storages/fuse/io.rs b/src/query/service/tests/it/storages/fuse/io.rs index 380aca84034b..524bf89e551c 100644 --- a/src/query/service/tests/it/storages/fuse/io.rs +++ b/src/query/service/tests/it/storages/fuse/io.rs @@ -25,10 +25,10 @@ use uuid::Uuid; #[test] fn test_meta_locations() -> Result<()> { let test_prefix = "test_pref"; - let locs = TableMetaLocationGenerator::with_prefix(test_prefix.to_owned()); - let ((path, _ver), _id) = locs.gen_block_location(); + let locs = TableMetaLocationGenerator::new(test_prefix.to_owned()); + let ((path, _ver), _id) = locs.gen_block_location(Default::default()); assert!(path.starts_with(test_prefix)); - let seg_loc = locs.gen_segment_info_location(); + let seg_loc = locs.gen_segment_info_location(Default::default()); assert!(seg_loc.starts_with(test_prefix)); let uuid = Uuid::new_v4(); let snapshot_loc = locs.snapshot_location_from_uuid(&uuid, TableSnapshot::VERSION)?; diff --git a/src/query/service/tests/it/storages/fuse/meta/snapshot.rs b/src/query/service/tests/it/storages/fuse/meta/snapshot.rs index b0d6b35413d3..7dcac3061063 100644 --- a/src/query/service/tests/it/storages/fuse/meta/snapshot.rs +++ b/src/query/service/tests/it/storages/fuse/meta/snapshot.rs @@ -14,19 +14,29 @@ use std::collections::HashMap; use std::ops::Add; +use std::sync::Arc; use databend_common_expression::TableSchema; use databend_storages_common_table_meta::meta::testing::StatisticsV0; use databend_storages_common_table_meta::meta::testing::TableSnapshotV1; use databend_storages_common_table_meta::meta::testing::TableSnapshotV2; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use uuid::Uuid; - fn default_snapshot() -> TableSnapshot { - let uuid = Uuid::new_v4(); let schema = TableSchema::empty(); let stats = Default::default(); - TableSnapshot::new(uuid, None, &None, None, schema, stats, vec![], None, None) + TableSnapshot::try_new( + None, + None, + schema, + stats, + vec![], + None, + None, + Default::default(), + ) + .unwrap() } #[test] @@ -39,18 +49,17 @@ fn snapshot_timestamp_is_some() { fn snapshot_timestamp_monotonic_increase() { let prev = default_snapshot(); let schema = TableSchema::empty(); - let uuid = Uuid::new_v4(); - let current = TableSnapshot::new( - uuid, + let current = TableSnapshot::try_new( None, - &prev.timestamp, - prev.prev_snapshot_id, + Some(Arc::new(prev.clone())), schema, Default::default(), vec![], None, None, - ); + Default::default(), + ) + .unwrap(); let current_ts = current.timestamp.unwrap(); let prev_ts = prev.timestamp.unwrap(); assert!(current_ts > prev_ts) @@ -60,22 +69,22 @@ fn snapshot_timestamp_monotonic_increase() { fn snapshot_timestamp_time_skew_tolerance() { let mut prev = default_snapshot(); let schema = TableSchema::empty(); - let uuid = Uuid::new_v4(); // simulating a stalled clock prev.timestamp = Some(prev.timestamp.unwrap().add(chrono::Duration::days(1))); + let table_meta_timestamps = TableMetaTimestamps::new(None, 72); - let current = TableSnapshot::new( - uuid, + let current = TableSnapshot::try_new( None, - &prev.timestamp, - prev.prev_snapshot_id, + Some(Arc::new(prev.clone())), schema, Default::default(), vec![], None, None, - ); + table_meta_timestamps, + ) + .unwrap(); let current_ts = current.timestamp.unwrap(); let prev_ts = prev.timestamp.unwrap(); assert!(current_ts > prev_ts) diff --git a/src/query/service/tests/it/storages/fuse/operations/commit.rs b/src/query/service/tests/it/storages/fuse/operations/commit.rs index be1090d0c933..24ddc5988761 100644 --- a/src/query/service/tests/it/storages/fuse/operations/commit.rs +++ b/src/query/service/tests/it/storages/fuse/operations/commit.rs @@ -151,12 +151,12 @@ use databend_storages_common_session::TxnManagerRef; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use futures::TryStreamExt; use parking_lot::Mutex; use parking_lot::RwLock; -use uuid::Uuid; use walkdir::WalkDir; use xorf::BinaryFuse16; @@ -279,17 +279,17 @@ async fn test_commit_to_meta_server() -> Result<()> { let fuse_table = FuseTable::try_from_table(table.as_ref())?; let new_segments = vec![("do not care".to_string(), SegmentInfo::VERSION)]; - let new_snapshot = TableSnapshot::new( - Uuid::new_v4(), + let new_snapshot = TableSnapshot::try_new( None, - &None, None, table.schema().as_ref().clone(), Statistics::default(), new_segments, None, None, - ); + Default::default(), + ) + .unwrap(); let faked_catalog = FakedCatalog { cat: catalog, @@ -882,6 +882,14 @@ impl TableContext for CtxDelegation { async fn drop_m_cte_temp_table(&self) -> Result<()> { todo!() } + fn get_table_meta_timestamps( + &self, + table_id: u64, + previous_snapshot: Option>, + ) -> Result { + self.ctx + .get_table_meta_timestamps(table_id, previous_snapshot) + } } #[derive(Clone, Debug)] 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 794073b1da9e..476cd21cac13 100644 --- a/src/query/service/tests/it/storages/fuse/operations/gc.rs +++ b/src/query/service/tests/it/storages/fuse/operations/gc.rs @@ -91,7 +91,8 @@ async fn test_fuse_purge_normal_orphan_snapshot() -> Result<()> { .snapshot_location_from_uuid(&orphan_snapshot_id, TableSnapshot::VERSION)?; // orphan_snapshot is created by using `from_previous`, which guarantees // that the timestamp of snapshot returned is larger than `current_snapshot`'s. - let orphan_snapshot = TableSnapshot::from_previous(current_snapshot.as_ref(), None); + let orphan_snapshot = + TableSnapshot::try_from_previous(current_snapshot.clone(), None, Default::default())?; orphan_snapshot .write_meta(&operator, &orphan_snapshot_location) .await?; @@ -196,7 +197,14 @@ async fn test_fuse_purge_orphan_retention() -> Result<()> { // 2. prepare `seg_2` let num_of_segments = 1; let blocks_per_segment = 1; - let segments = generate_segments(fuse_table, num_of_segments, blocks_per_segment).await?; + let segments = generate_segments( + fuse_table, + num_of_segments, + blocks_per_segment, + false, + Default::default(), + ) + .await?; let (segment_locations, _segment_info): (Vec<_>, Vec<_>) = segments.into_iter().unzip(); // 2. prepare S_2 @@ -209,7 +217,14 @@ async fn test_fuse_purge_orphan_retention() -> Result<()> { { let num_of_segments = 1; let blocks_per_segment = 1; - let segments = generate_segments(fuse_table, num_of_segments, blocks_per_segment).await?; + let segments = generate_segments( + fuse_table, + num_of_segments, + blocks_per_segment, + false, + Default::default(), + ) + .await?; let segment_locations: Vec = segments.into_iter().map(|(l, _)| l).collect(); let new_timestamp = base_timestamp - Duration::days(1); let _snapshot_location = generate_snapshot_with_segments( diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs index 0ac5a4b2c355..a8512c9f548b 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/block_compact_mutator.rs @@ -44,7 +44,6 @@ use databend_storages_common_table_meta::meta::TableSnapshot; use opendal::Operator; use rand::thread_rng; use rand::Rng; -use uuid::Uuid; use crate::storages::fuse::operations::mutation::segments_compact_mutator::CompactSegmentTestFixture; @@ -121,12 +120,15 @@ async fn do_compact(ctx: Arc, table: Arc) -> Result, table: Arc) -> Result Result<()> { merge_statistics_mut(&mut summary, &seg.summary, None); } - let id = Uuid::new_v4(); - let snapshot = TableSnapshot::new( - id, + let snapshot = TableSnapshot::try_new( None, - &None, None, schema.as_ref().clone(), summary, locations.clone(), None, None, - ); + Default::default(), + )?; let limit: usize = rand.gen_range(1..15); let compact_params = CompactOptions { diff --git a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs index 8fe26e8947f3..ef5e86193042 100644 --- a/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs +++ b/src/query/service/tests/it/storages/fuse/operations/mutation/recluster_mutator.rs @@ -58,7 +58,7 @@ use crate::storages::fuse::operations::mutation::CompactSegmentTestFixture; async fn test_recluster_mutator_block_select() -> Result<()> { let fixture = TestFixture::setup().await?; let ctx = fixture.new_query_ctx().await?; - let location_generator = TableMetaLocationGenerator::with_prefix("_prefix".to_owned()); + let location_generator = TableMetaLocationGenerator::new("_prefix".to_owned()); let data_accessor = ctx.get_application_level_data_operator()?.operator(); @@ -88,7 +88,7 @@ async fn test_recluster_mutator_block_select() -> Result<()> { ); let segment = SegmentInfo::new(vec![test_block_meta], statistics); - let segment_location = location_generator.gen_segment_info_location(); + let segment_location = location_generator.gen_segment_info_location(Default::default()); segment .write_meta(&data_accessor, &segment_location) .await?; @@ -247,18 +247,16 @@ async fn test_safety_for_recluster() -> Result<()> { merge_statistics_mut(&mut summary, &seg.summary, Some(cluster_key_id)); } - let id = Uuid::new_v4(); - let snapshot = Arc::new(TableSnapshot::new( - id, + let snapshot = Arc::new(TableSnapshot::try_new( None, - &None, None, schema.as_ref().clone(), summary, locations.clone(), None, None, - )); + Default::default(), + )?); let mut block_ids = HashSet::new(); for seg in &segment_infos { 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 74fe1c57bed5..88058e09617e 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 @@ -638,7 +638,7 @@ pub struct CompactSegmentTestFixture { impl CompactSegmentTestFixture { fn try_new(ctx: &Arc, block_per_seg: u64) -> Result { - let location_gen = TableMetaLocationGenerator::with_prefix("test/".to_owned()); + let location_gen = TableMetaLocationGenerator::new("test/".to_owned()); let data_accessor = ctx.get_application_level_data_operator()?; Ok(Self { ctx: ctx.clone(), @@ -706,7 +706,7 @@ impl CompactSegmentTestFixture { block_per_seg: usize, unclustered: bool, ) -> Result<(Vec, Vec, Vec)> { - let location_gen = TableMetaLocationGenerator::with_prefix("test/".to_owned()); + let location_gen = TableMetaLocationGenerator::new("test/".to_owned()); let data_accessor = ctx.get_application_level_data_operator()?.operator(); let threads_nums = ctx.get_settings().get_max_threads()? as usize; @@ -747,7 +747,7 @@ impl CompactSegmentTestFixture { }) }; - let (location, _) = location_gen.gen_block_location(); + let (location, _) = location_gen.gen_block_location(Default::default()); let row_count = block.num_rows() as u64; let block_size = block.memory_size() as u64; @@ -782,7 +782,7 @@ impl CompactSegmentTestFixture { } let summary = stats_acc.summary(thresholds, cluster_key_id); let segment_info = SegmentInfo::new(stats_acc.blocks_metas, summary); - let path = location_gen.gen_segment_info_location(); + let path = location_gen.gen_segment_info_location(Default::default()); segment_info.write_meta(&data_accessor, &path).await?; Ok::<_, ErrorCode>(((path, SegmentInfo::VERSION), collected_blocks, segment_info)) }); @@ -964,7 +964,7 @@ async fn test_compact_segment_with_cluster() -> Result<()> { let fixture = TestFixture::setup().await?; let ctx = fixture.new_query_ctx().await?; - let location_gen = TableMetaLocationGenerator::with_prefix("test/".to_owned()); + let location_gen = TableMetaLocationGenerator::new("test/".to_owned()); let data_accessor = ctx.get_application_level_data_operator()?.operator(); let schema = TestFixture::default_table_schema(); 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 854baa2b19d4..0df8a5f3e857 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 @@ -205,7 +205,8 @@ async fn test_table_analyze_without_prev_table_seq() -> Result<()> { // genenrate snapshot without prev_table_seq let snapshot_0 = fuse_table.read_table_snapshot().await?.unwrap(); - let snapshot_1 = TableSnapshot::from_previous(&snapshot_0, None); + let snapshot_1 = + TableSnapshot::try_from_previous(snapshot_0.clone(), None, Default::default())?; let snapshot_loc_1 = location_gen .snapshot_location_from_uuid(&snapshot_1.snapshot_id, TableSnapshot::VERSION)?; snapshot_1.write_meta(&operator, &snapshot_loc_1).await?; @@ -220,7 +221,8 @@ async fn test_table_analyze_without_prev_table_seq() -> Result<()> { table_statistics.format_version(), )?; // genenrate snapshot without prev_table_seq - let mut snapshot_2 = TableSnapshot::from_previous(&snapshot_1, None); + let mut snapshot_2 = + TableSnapshot::try_from_previous(Arc::new(snapshot_1.clone()), None, Default::default())?; snapshot_2.table_statistics_location = Some(table_statistics_location); FuseTable::commit_to_meta_server( fixture.new_query_ctx().await?.as_ref(), diff --git a/src/query/service/tests/it/storages/fuse/statistics.rs b/src/query/service/tests/it/storages/fuse/statistics.rs index 6224b218faa3..459ceea8f459 100644 --- a/src/query/service/tests/it/storages/fuse/statistics.rs +++ b/src/query/service/tests/it/storages/fuse/statistics.rs @@ -317,11 +317,11 @@ async fn test_accumulator() -> databend_common_exception::Result<()> { let mut stats_acc = StatisticsAccumulator::default(); let operator = Operator::new(opendal::services::Memory::default())?.finish(); - let loc_generator = TableMetaLocationGenerator::with_prefix("/".to_owned()); + let loc_generator = TableMetaLocationGenerator::new("/".to_owned()); for item in blocks { let block = item?; let col_stats = gen_columns_statistics(&block, None, &schema)?; - let block_writer = BlockWriter::new(&operator, &loc_generator); + let block_writer = BlockWriter::new(&operator, &loc_generator, Default::default(), true); let (block_meta, _index_meta) = block_writer .write(FuseStorageFormat::Parquet, &schema, block, col_stats, None) .await?; diff --git a/src/query/settings/src/settings_default.rs b/src/query/settings/src/settings_default.rs index 2738858242e0..b5393585ce0d 100644 --- a/src/query/settings/src/settings_default.rs +++ b/src/query/settings/src/settings_default.rs @@ -179,6 +179,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(0..=data_retention_time_in_days_max)), }), + ("max_retryable_transaction_duration_in_hours", DefaultSettingValue { + value: UserSettingValue::UInt64(72), + desc: "Sets the maximum retryable transaction duration in hours.", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(0..=240)), + }), ("max_spill_io_requests", DefaultSettingValue { value: UserSettingValue::UInt64(default_max_spill_io_requests), desc: "Sets the maximum number of concurrent spill I/O requests.", @@ -193,6 +200,13 @@ impl DefaultSettings { scope: SettingScope::Both, range: Some(SettingRange::Numeric(1..=1024)), }), + ("max_delete_requests", DefaultSettingValue { + value: UserSettingValue::UInt64(1), + desc: "Sets the maximum number of concurrent delete files requests in fuse_vacuum2().", + mode: SettingMode::Both, + scope: SettingScope::Both, + range: Some(SettingRange::Numeric(1..=1024)), + }), ("storage_io_min_bytes_for_seek", DefaultSettingValue { value: UserSettingValue::UInt64(48), desc: "Sets the minimum byte size of data that must be read from storage in a single I/O operation \ diff --git a/src/query/settings/src/settings_getter_setter.rs b/src/query/settings/src/settings_getter_setter.rs index 214206bf0887..70652fb066ae 100644 --- a/src/query/settings/src/settings_getter_setter.rs +++ b/src/query/settings/src/settings_getter_setter.rs @@ -223,6 +223,14 @@ impl Settings { self.try_get_u64("data_retention_time_in_days") } + pub fn get_max_retryable_transaction_duration_in_hours(&self) -> Result { + self.try_get_u64("max_retryable_transaction_duration_in_hours") + } + + pub fn set_max_retryable_transaction_duration_in_hours(&self, val: u64) -> Result<()> { + self.try_set_u64("max_retryable_transaction_duration_in_hours", val) + } + pub fn get_max_storage_io_requests(&self) -> Result { self.try_get_u64("max_storage_io_requests") } @@ -816,6 +824,14 @@ impl Settings { self.try_set_u64("short_sql_max_length", val) } + pub fn get_max_delete_requests(&self) -> Result { + self.try_get_u64("max_delete_requests") + } + + pub fn set_max_delete_requests(&self, val: u64) -> Result<()> { + self.try_set_u64("max_delete_requests", val) + } + pub fn get_enable_prune_pipeline(&self) -> Result { Ok(self.try_get_u64("enable_prune_pipeline")? == 1) } diff --git a/src/query/sql/src/executor/physical_plan_builder.rs b/src/query/sql/src/executor/physical_plan_builder.rs index 1b28a32d5f87..148ffce2027c 100644 --- a/src/query/sql/src/executor/physical_plan_builder.rs +++ b/src/query/sql/src/executor/physical_plan_builder.rs @@ -20,6 +20,7 @@ use databend_common_exception::Result; use databend_common_expression::FunctionContext; use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::UpdateStreamMetaReq; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use crate::executor::explain::PlanStatsInfo; @@ -142,4 +143,5 @@ pub struct MutationBuildInfo { pub table_snapshot: Option>, pub update_stream_meta: Vec, pub partitions: Option, + pub table_meta_timestamps: TableMetaTimestamps, } diff --git a/src/query/sql/src/executor/physical_plan_visitor.rs b/src/query/sql/src/executor/physical_plan_visitor.rs index 7f53a14b515f..9b9818540b57 100644 --- a/src/query/sql/src/executor/physical_plan_visitor.rs +++ b/src/query/sql/src/executor/physical_plan_visitor.rs @@ -409,13 +409,8 @@ pub trait PhysicalPlanReplacer { Ok(PhysicalPlan::DistributedInsertSelect(Box::new( DistributedInsertSelect { - plan_id: plan.plan_id, input: Box::new(input), - table_info: plan.table_info.clone(), - select_schema: plan.select_schema.clone(), - insert_schema: plan.insert_schema.clone(), - select_column_bindings: plan.select_column_bindings.clone(), - cast_needed: plan.cast_needed, + ..plan.clone() }, ))) } diff --git a/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs b/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs index b3e167a802dd..2fc0ee59907c 100644 --- a/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs +++ b/src/query/sql/src/executor/physical_plans/physical_column_mutation.rs @@ -16,6 +16,7 @@ use std::collections::HashMap; use databend_common_expression::RemoteExpr; use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::binder::MutationType; use crate::executor::physical_plan::PhysicalPlan; @@ -31,4 +32,5 @@ pub struct ColumnMutation { pub field_id_to_schema_index: HashMap, pub input_num_columns: usize, pub has_filter_column: bool, + pub table_meta_timestamps: TableMetaTimestamps, } diff --git a/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs b/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs index 33bd4f06f7ca..76d2b938d828 100644 --- a/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs +++ b/src/query/sql/src/executor/physical_plans/physical_commit_sink.rs @@ -18,6 +18,7 @@ use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::UpdateStreamMetaReq; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use crate::executor::physical_plans::common::MutationKind; @@ -35,6 +36,7 @@ pub struct CommitSink { pub update_stream_meta: Vec, pub merge_meta: bool, pub deduplicated_label: Option, + pub table_meta_timestamps: TableMetaTimestamps, // Used for recluster. pub recluster_info: Option, diff --git a/src/query/sql/src/executor/physical_plans/physical_compact_source.rs b/src/query/sql/src/executor/physical_plans/physical_compact_source.rs index 333db8087154..3ffd6f971a9d 100644 --- a/src/query/sql/src/executor/physical_plans/physical_compact_source.rs +++ b/src/query/sql/src/executor/physical_plans/physical_compact_source.rs @@ -21,6 +21,7 @@ use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::ColumnId; use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::executor::physical_plans::CommitSink; use crate::executor::physical_plans::Exchange; @@ -35,6 +36,7 @@ pub struct CompactSource { pub parts: Partitions, pub table_info: TableInfo, pub column_ids: HashSet, + pub table_meta_timestamps: TableMetaTimestamps, } impl PhysicalPlanBuilder { @@ -64,12 +66,17 @@ impl PhysicalPlanBuilder { ))); }; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(table_info.ident.table_id, Some(snapshot.clone()))?; + let merge_meta = parts.partitions_type() == PartInfoType::LazyLevel; let mut root = PhysicalPlan::CompactSource(Box::new(CompactSource { parts, table_info: table_info.clone(), column_ids: snapshot.schema.to_leaf_column_id_set(), plan_id: u32::MAX, + table_meta_timestamps, })); let is_distributed = (!self.ctx.get_cluster().is_empty()) @@ -95,6 +102,7 @@ impl PhysicalPlanBuilder { deduplicated_label: None, plan_id: u32::MAX, recluster_info: None, + table_meta_timestamps, })); root.adjust_plan_id(&mut 0); diff --git a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs b/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs index 1fd92f5c3ce5..6d678ad12efc 100644 --- a/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs +++ b/src/query/sql/src/executor/physical_plans/physical_copy_into_table.rs @@ -18,6 +18,7 @@ use databend_common_expression::DataSchemaRef; use databend_common_expression::DataSchemaRefExt; use databend_common_expression::Scalar; use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use enum_as_inner::EnumAsInner; use crate::executor::physical_plan::PhysicalPlan; @@ -40,6 +41,7 @@ pub struct CopyIntoTable { pub project_columns: Option>, pub source: CopyIntoTableSource, pub is_transform: bool, + pub table_meta_timestamps: TableMetaTimestamps, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize, EnumAsInner)] diff --git a/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs b/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs index abe0def948db..678cf7b21063 100644 --- a/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs +++ b/src/query/sql/src/executor/physical_plans/physical_distributed_insert_select.rs @@ -14,6 +14,7 @@ use databend_common_expression::DataSchemaRef; use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::executor::PhysicalPlan; use crate::ColumnBinding; @@ -29,4 +30,5 @@ pub struct DistributedInsertSelect { pub select_schema: DataSchemaRef, pub select_column_bindings: Vec, pub cast_needed: bool, + pub table_meta_timestamps: TableMetaTimestamps, } diff --git a/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs b/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs index 5e6f423c05c0..13669a9a1a5f 100644 --- a/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs +++ b/src/query/sql/src/executor/physical_plans/physical_multi_table_insert.rs @@ -22,6 +22,7 @@ use databend_common_expression::RemoteExpr; use databend_common_meta_app::schema::CatalogInfo; use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::UpdateStreamMetaReq; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::executor::PhysicalPlan; #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] @@ -122,6 +123,7 @@ pub struct ChunkAppendData { pub struct SerializableTable { pub target_catalog_info: Arc, pub target_table_info: TableInfo, + pub table_meta_timestamps: TableMetaTimestamps, } #[derive(Clone, Debug, serde::Serialize, serde::Deserialize)] diff --git a/src/query/sql/src/executor/physical_plans/physical_mutation.rs b/src/query/sql/src/executor/physical_plans/physical_mutation.rs index df8573418c55..62ff0038dfbd 100644 --- a/src/query/sql/src/executor/physical_plans/physical_mutation.rs +++ b/src/query/sql/src/executor/physical_plans/physical_mutation.rs @@ -39,6 +39,7 @@ use databend_common_expression::ROW_ID_COL_NAME; use databend_common_functions::BUILTIN_FUNCTIONS; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::NUM_BLOCK_ID_BITS; use databend_storages_common_table_meta::readers::snapshot_reader::TableSnapshotAccessor; use itertools::Itertools; @@ -90,6 +91,7 @@ pub struct Mutation { pub need_match: bool, pub distributed: bool, pub target_build_optimization: bool, + pub table_meta_timestamps: TableMetaTimestamps, } impl PhysicalPlanBuilder { @@ -190,6 +192,7 @@ impl PhysicalPlanBuilder { field_id_to_schema_index, input_num_columns: mutation_input_schema.fields().len(), has_filter_column: predicate_column_index.is_some(), + table_meta_timestamps: mutation_build_info.table_meta_timestamps, }); if *distributed { @@ -214,6 +217,7 @@ impl PhysicalPlanBuilder { deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, plan_id: u32::MAX, recluster_info: None, + table_meta_timestamps: mutation_build_info.table_meta_timestamps, })); plan.adjust_plan_id(&mut 0); @@ -414,6 +418,7 @@ impl PhysicalPlanBuilder { need_match: !is_not_matched_only, target_build_optimization: false, plan_id: u32::MAX, + table_meta_timestamps: mutation_build_info.table_meta_timestamps, })); let commit_input = if !distributed { @@ -451,6 +456,7 @@ impl PhysicalPlanBuilder { deduplicated_label: unsafe { self.ctx.get_settings().get_deduplicate_label()? }, plan_id: u32::MAX, recluster_info: None, + table_meta_timestamps: mutation_build_info.table_meta_timestamps, })); physical_plan.adjust_plan_id(&mut 0); Ok(physical_plan) diff --git a/src/query/sql/src/executor/physical_plans/physical_recluster.rs b/src/query/sql/src/executor/physical_plans/physical_recluster.rs index 6fdb94eb97c9..2217a58ca52e 100644 --- a/src/query/sql/src/executor/physical_plans/physical_recluster.rs +++ b/src/query/sql/src/executor/physical_plans/physical_recluster.rs @@ -20,6 +20,7 @@ use databend_common_catalog::table::TableExt; use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_meta_app::schema::TableInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::executor::physical_plans::physical_commit_sink::ReclusterInfoSideCar; use crate::executor::physical_plans::CommitSink; @@ -35,6 +36,7 @@ pub struct Recluster { pub plan_id: u32, pub tasks: Vec, pub table_info: TableInfo, + pub table_meta_timestamps: TableMetaTimestamps, } impl PhysicalPlanBuilder { @@ -86,6 +88,9 @@ impl PhysicalPlanBuilder { "No need to do recluster for '{database}'.'{table}'" ))); }; + let table_meta_timestamps = self + .ctx + .get_table_meta_timestamps(tbl.get_id(), Some(snapshot.clone()))?; if parts.is_empty() { return Err(ErrorCode::NoNeedToRecluster(format!( "No need to do recluster for '{database}'.'{table}'" @@ -105,6 +110,7 @@ impl PhysicalPlanBuilder { tasks, table_info: table_info.clone(), plan_id: u32::MAX, + table_meta_timestamps, })); if is_distributed { @@ -126,6 +132,7 @@ impl PhysicalPlanBuilder { merge_meta: false, deduplicated_label: None, plan_id: u32::MAX, + table_meta_timestamps, recluster_info: Some(ReclusterInfoSideCar { merged_blocks: remained_blocks, removed_segment_indexes, @@ -140,6 +147,7 @@ impl PhysicalPlanBuilder { table_info: table_info.clone(), column_ids: snapshot.schema.to_leaf_column_id_set(), plan_id: u32::MAX, + table_meta_timestamps, })); if is_distributed { @@ -162,6 +170,7 @@ impl PhysicalPlanBuilder { merge_meta, deduplicated_label: None, plan_id: u32::MAX, + table_meta_timestamps, recluster_info: None, })) } diff --git a/src/query/sql/src/executor/physical_plans/physical_replace_into.rs b/src/query/sql/src/executor/physical_plans/physical_replace_into.rs index 477a8a81d46e..75d0af7ac2cd 100644 --- a/src/query/sql/src/executor/physical_plans/physical_replace_into.rs +++ b/src/query/sql/src/executor/physical_plans/physical_replace_into.rs @@ -17,6 +17,7 @@ use databend_common_expression::FieldIndex; use databend_common_meta_app::schema::TableInfo; use databend_storages_common_table_meta::meta::BlockSlotDescription; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::executor::physical_plans::common::OnConflictField; use crate::executor::PhysicalPlan; @@ -34,4 +35,5 @@ pub struct ReplaceInto { pub segments: Vec<(usize, Location)>, pub block_slots: Option, pub need_insert: bool, + pub table_meta_timestamps: TableMetaTimestamps, } diff --git a/src/query/sql/src/planner/binder/binder.rs b/src/query/sql/src/planner/binder/binder.rs index 70279db2dc46..bd961eb807f1 100644 --- a/src/query/sql/src/planner/binder/binder.rs +++ b/src/query/sql/src/planner/binder/binder.rs @@ -1060,7 +1060,7 @@ async fn try_purge_files(ctx: Arc, stage_info: &StageInfo, fil match op { Ok(op) => { let file_op = Files::create(ctx, op); - if let Err(e) = file_op.remove_file_in_batch(files).await { + if let Err(e) = file_op.remove_file_in_batch(files, false).await { error!("Failed to delete file: {:?}, error: {}", files, e); } } diff --git a/src/query/storages/common/io/src/files.rs b/src/query/storages/common/io/src/files.rs index 57b30512f9c4..6d60a7b1b9bb 100644 --- a/src/query/storages/common/io/src/files.rs +++ b/src/query/storages/common/io/src/files.rs @@ -39,6 +39,7 @@ impl Files { pub async fn remove_file_in_batch( &self, file_locations: impl IntoIterator>, + limit_max_delete_requests: bool, ) -> Result<()> { let locations = Vec::from_iter(file_locations.into_iter().map(|v| v.as_ref().to_string())); @@ -48,7 +49,10 @@ impl Files { // adjusts batch_size according to the `max_threads` settings, // limits its min/max value to 1 and 1000. - let threads_nums = self.ctx.get_settings().get_max_threads()? as usize; + let threads_nums = match limit_max_delete_requests { + true => self.ctx.get_settings().get_max_delete_requests()?, + false => self.ctx.get_settings().get_max_threads()?, + } as usize; let batch_size = (locations.len() / threads_nums).clamp(1, 1000); info!( diff --git a/src/query/storages/common/session/src/transaction.rs b/src/query/storages/common/session/src/transaction.rs index b1121ba2e5a4..8836bda5eda4 100644 --- a/src/query/storages/common/session/src/transaction.rs +++ b/src/query/storages/common/session/src/transaction.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::hash_map::Entry; use std::collections::BTreeMap; use std::collections::HashMap; use std::collections::HashSet; @@ -28,6 +29,8 @@ use databend_common_meta_app::schema::UpdateTempTableReq; 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::TableMetaTimestamps; +use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::table_id_ranges::is_temp_table_id; use parking_lot::Mutex; use serde::Deserialize; @@ -61,6 +64,8 @@ pub struct TxnBuffer { stream_tables: HashMap, need_purge_files: Vec<(StageInfo, Vec)>, + pub table_meta_timestamps: HashMap, + temp_table_desc_to_id: HashMap, mutated_temp_tables: HashMap, } @@ -348,4 +353,25 @@ impl TxnManager { pub fn need_purge_files(&mut self) -> Vec<(StageInfo, Vec)> { std::mem::take(&mut self.txn_buffer.need_purge_files) } + + pub fn get_table_meta_timestamps( + &mut self, + table_id: u64, + previous_snapshot: Option>, + delta: i64, + ) -> TableMetaTimestamps { + if !self.is_active() { + return TableMetaTimestamps::new(previous_snapshot, delta); + } + + let entry = self.txn_buffer.table_meta_timestamps.entry(table_id); + match entry { + Entry::Occupied(e) => *e.get(), + Entry::Vacant(e) => { + let timestamps = TableMetaTimestamps::new(previous_snapshot, delta); + e.insert(timestamps); + timestamps + } + } + } } diff --git a/src/query/storages/common/table_meta/src/meta/mod.rs b/src/query/storages/common/table_meta/src/meta/mod.rs index 165c8bfe758f..23f0c96a36a5 100644 --- a/src/query/storages/common/table_meta/src/meta/mod.rs +++ b/src/query/storages/common/table_meta/src/meta/mod.rs @@ -25,7 +25,6 @@ mod v2; mod v3; mod v4; mod versions; - pub use compression::Compression; // table meta types of current version pub use current::*; @@ -38,8 +37,10 @@ pub use statistics::*; // currently, used by versioned readers only pub(crate) use testing::*; pub use utils::parse_storage_prefix; -pub use utils::trim_vacuum2_object_prefix; +pub use utils::trim_object_prefix; pub use utils::try_extract_uuid_str_from_path; +pub use utils::uuid_from_date_time; +pub use utils::TableMetaTimestamps; pub use utils::TEMP_TABLE_STORAGE_PREFIX; pub use utils::VACUUM2_OBJECT_KEY_PREFIX; pub(crate) use utils::*; @@ -60,4 +61,5 @@ pub mod testing { pub use super::v2::TableSnapshot as TableSnapshotV2; pub use super::v3::SegmentInfo as SegmentInfoV3; pub use super::v3::TableSnapshot as TableSnapshotV3; + pub use super::v4::TableSnapshot as TableSnapshotV4; } diff --git a/src/query/storages/common/table_meta/src/meta/utils.rs b/src/query/storages/common/table_meta/src/meta/utils.rs index f0d6f6edef11..8ffcc89bdf91 100644 --- a/src/query/storages/common/table_meta/src/meta/utils.rs +++ b/src/query/storages/common/table_meta/src/meta/utils.rs @@ -15,12 +15,16 @@ use std::collections::BTreeMap; use std::ops::Add; use std::path::Path; +use std::sync::Arc; use chrono::DateTime; use chrono::Datelike; use chrono::TimeZone; use chrono::Timelike; use chrono::Utc; +use databend_common_base::base::uuid; +use databend_common_base::base::uuid::NoContext; +use databend_common_base::base::uuid::Uuid; use databend_common_exception::ErrorCode; use databend_common_exception::Result; @@ -30,9 +34,11 @@ use crate::table::OPT_KEY_STORAGE_PREFIX; use crate::table::OPT_KEY_TEMP_PREFIX; pub const TEMP_TABLE_STORAGE_PREFIX: &str = "_tmp_tbl"; -pub const VACUUM2_OBJECT_KEY_PREFIX: &str = "g"; +use crate::meta::TableSnapshot; +use crate::readers::snapshot_reader::TableSnapshotAccessor; +pub const VACUUM2_OBJECT_KEY_PREFIX: &str = "h"; -pub fn trim_timestamp_to_micro_second(ts: DateTime) -> DateTime { +pub fn trim_timestamp_to_milli_second(ts: DateTime) -> DateTime { Utc.with_ymd_and_hms( ts.year(), ts.month(), @@ -42,7 +48,7 @@ pub fn trim_timestamp_to_micro_second(ts: DateTime) -> DateTime { ts.second(), ) .unwrap() - .with_nanosecond(ts.timestamp_subsec_micros() * 1_000) + .with_nanosecond(ts.timestamp_subsec_millis() * 1_000_000) .unwrap() } @@ -50,40 +56,59 @@ pub fn monotonically_increased_timestamp( timestamp: DateTime, previous_timestamp: &Option>, ) -> DateTime { - if let Some(prev_instant) = previous_timestamp { - // timestamp of the snapshot should always larger than the previous one's - if prev_instant > ×tamp { - // if local time is smaller, use the timestamp of previous snapshot, plus 1 ms - return prev_instant.add(chrono::Duration::milliseconds(1)); - } + let timestamp = trim_timestamp_to_milli_second(timestamp); + + let Some(prev) = previous_timestamp else { + return timestamp; + }; + + let prev = trim_timestamp_to_milli_second(*prev); + + if prev >= timestamp { + prev.add(chrono::Duration::milliseconds(1)) + } else { + timestamp } - timestamp } -pub fn parse_storage_prefix(options: &BTreeMap, table_id: u64) -> Result { - // if OPT_KE_STORAGE_PREFIX is specified, use it as storage prefix - if let Some(prefix) = options.get(OPT_KEY_STORAGE_PREFIX) { - return Ok(prefix.clone()); - } +#[derive(serde::Serialize, serde::Deserialize, Clone, Debug, Copy)] +pub struct TableMetaTimestamps { + pub segment_block_timestamp: chrono::DateTime, + pub snapshot_timestamp: chrono::DateTime, +} - // otherwise, use database id and table id as storage prefix +impl TableMetaTimestamps { + pub fn new(previous_snapshot: Option>, delta: i64) -> Self { + let snapshot_timestamp = + monotonically_increased_timestamp(chrono::Utc::now(), &previous_snapshot.timestamp()); - let db_id = options.get(OPT_KEY_DATABASE_ID).ok_or_else(|| { - ErrorCode::Internal(format!( - "Invalid fuse table, table option {} not found", - OPT_KEY_DATABASE_ID - )) - })?; - let mut prefix = table_storage_prefix(db_id, table_id); - if let Some(temp_prefix) = options.get(OPT_KEY_TEMP_PREFIX) { - prefix = format!("{}/{}/{}", TEMP_TABLE_STORAGE_PREFIX, temp_prefix, prefix); + let delta = chrono::Duration::hours(delta); + + let segment_block_timestamp = snapshot_timestamp + delta; + + Self { + snapshot_timestamp, + segment_block_timestamp, + } } - Ok(prefix) } -#[inline] -pub fn trim_vacuum2_object_prefix(key: &str) -> &str { - key.strip_prefix(VACUUM2_OBJECT_KEY_PREFIX).unwrap_or(key) +/// used in ut +impl Default for TableMetaTimestamps { + fn default() -> Self { + Self::new(None, 1) + } +} + +pub fn uuid_from_date_time(ts: DateTime) -> Uuid { + // only in this range, the order of timestamps is preserved in UUIDs + // out of range is unlikely to happen, this is just a safe guard + let range = 0..=0xFFFF_FFFF_FFFF; + assert!(range.contains(&ts.timestamp_millis())); + let seconds = ts.timestamp(); + let nanos = ts.timestamp_subsec_nanos(); + let uuid_ts = uuid::Timestamp::from_unix(NoContext, seconds as u64, nanos); + Uuid::new_v7(uuid_ts) } // Extracts the UUID part from the object key. @@ -98,7 +123,7 @@ pub fn try_extract_uuid_str_from_path(path: &str) -> databend_common_exception:: .unwrap() // path is always valid utf8 string .split('_') .collect::>(); - let uuid = trim_vacuum2_object_prefix(file_name[0]); + let uuid = trim_object_prefix(file_name[0]); Ok(uuid) } else { Err(ErrorCode::StorageOther(format!( @@ -108,23 +133,56 @@ pub fn try_extract_uuid_str_from_path(path: &str) -> databend_common_exception:: } } +pub fn parse_storage_prefix(options: &BTreeMap, table_id: u64) -> Result { + // if OPT_KE_STORAGE_PREFIX is specified, use it as storage prefix + if let Some(prefix) = options.get(OPT_KEY_STORAGE_PREFIX) { + return Ok(prefix.clone()); + } + + // otherwise, use database id and table id as storage prefix + + let db_id = options.get(OPT_KEY_DATABASE_ID).ok_or_else(|| { + ErrorCode::Internal(format!( + "Invalid fuse table, table option {} not found", + OPT_KEY_DATABASE_ID + )) + })?; + let mut prefix = table_storage_prefix(db_id, table_id); + if let Some(temp_prefix) = options.get(OPT_KEY_TEMP_PREFIX) { + prefix = format!("{}/{}/{}", TEMP_TABLE_STORAGE_PREFIX, temp_prefix, prefix); + } + Ok(prefix) +} + +#[inline] +pub fn trim_object_prefix(key: &str) -> &str { + // if object key (the file_name/stem part only) starts with a char which is larger + // than 'f', strip it off + if key > "f" { + &key[1..] + } else { + key + } +} + #[cfg(test)] +#[allow(clippy::items_after_test_module)] mod tests { + use databend_common_base::base::uuid::Uuid; - use super::*; + use crate::meta::trim_object_prefix; + use crate::meta::try_extract_uuid_str_from_path; + use crate::meta::VACUUM2_OBJECT_KEY_PREFIX; #[test] fn test_trim_vacuum2_object_prefix() { let uuid = Uuid::now_v7(); assert_eq!( - trim_vacuum2_object_prefix(&format!("g{}", uuid)), - uuid.to_string() - ); - assert_eq!( - trim_vacuum2_object_prefix(&uuid.to_string()), + trim_object_prefix(&format!("{}{}", VACUUM2_OBJECT_KEY_PREFIX, uuid)), uuid.to_string() ); + assert_eq!(trim_object_prefix(&uuid.to_string()), uuid.to_string()); } #[test] @@ -144,4 +202,25 @@ mod tests { assert_eq!(try_extract_uuid_str_from_path(input).unwrap(), expected); } } + fn assert_order_preserved( + ts1: chrono::DateTime, + ts2: chrono::DateTime, + ) { + let uuid1 = super::uuid_from_date_time(ts1); + let uuid2 = super::uuid_from_date_time(ts2); + assert_eq!(ts1.cmp(&ts2), uuid1.cmp(&uuid2)); + } + + #[test] + fn test_uuid_from_date_time() { + let now = chrono::Utc::now(); + assert_order_preserved(now, now + chrono::Duration::milliseconds(1)); + assert_order_preserved(now, now - chrono::Duration::milliseconds(1)); + assert_order_preserved(now, chrono::DateTime::default()); + + let ms = 0xFFFF_FFFF_FFFF; + let ts = chrono::DateTime::from_timestamp_millis(ms).unwrap(); + assert_order_preserved(now, ts); + assert_order_preserved(ts - chrono::Duration::milliseconds(1), ts); + } } diff --git a/src/query/storages/common/table_meta/src/meta/v1/snapshot.rs b/src/query/storages/common/table_meta/src/meta/v1/snapshot.rs index 4bb0687a03fe..e72a9aa39678 100644 --- a/src/query/storages/common/table_meta/src/meta/v1/snapshot.rs +++ b/src/query/storages/common/table_meta/src/meta/v1/snapshot.rs @@ -21,7 +21,7 @@ use serde::Serialize; use super::super::v0::statistics::Statistics; use crate::meta::monotonically_increased_timestamp; -use crate::meta::trim_timestamp_to_micro_second; +use crate::meta::trim_timestamp_to_milli_second; use crate::meta::ClusterKey; use crate::meta::FormatVersion; use crate::meta::Location; @@ -77,7 +77,7 @@ impl TableSnapshot { let adjusted_timestamp = monotonically_increased_timestamp(now, prev_timestamp); // trim timestamp to micro seconds - let trimmed_timestamp = trim_timestamp_to_micro_second(adjusted_timestamp); + let trimmed_timestamp = trim_timestamp_to_milli_second(adjusted_timestamp); let timestamp = Some(trimmed_timestamp); Self { diff --git a/src/query/storages/common/table_meta/src/meta/v2/snapshot.rs b/src/query/storages/common/table_meta/src/meta/v2/snapshot.rs index 1dbe5ccdc9f3..df262012e522 100644 --- a/src/query/storages/common/table_meta/src/meta/v2/snapshot.rs +++ b/src/query/storages/common/table_meta/src/meta/v2/snapshot.rs @@ -20,7 +20,7 @@ use serde::Deserialize; use serde::Serialize; use crate::meta::monotonically_increased_timestamp; -use crate::meta::trim_timestamp_to_micro_second; +use crate::meta::trim_timestamp_to_milli_second; use crate::meta::v0; use crate::meta::v1; use crate::meta::ClusterKey; @@ -78,7 +78,7 @@ impl TableSnapshot { let adjusted_timestamp = monotonically_increased_timestamp(now, prev_timestamp); // trim timestamp to micro seconds - let trimmed_timestamp = trim_timestamp_to_micro_second(adjusted_timestamp); + let trimmed_timestamp = trim_timestamp_to_milli_second(adjusted_timestamp); let timestamp = Some(trimmed_timestamp); Self { diff --git a/src/query/storages/common/table_meta/src/meta/v4/snapshot.rs b/src/query/storages/common/table_meta/src/meta/v4/snapshot.rs index e6fc53ae206a..86e21bf578dc 100644 --- a/src/query/storages/common/table_meta/src/meta/v4/snapshot.rs +++ b/src/query/storages/common/table_meta/src/meta/v4/snapshot.rs @@ -14,10 +14,11 @@ use std::io::Cursor; use std::io::Read; +use std::sync::Arc; use chrono::DateTime; use chrono::Utc; -use databend_common_base::base::uuid::Uuid; +use databend_common_exception::ErrorCode; use databend_common_exception::Result; use databend_common_expression::TableSchema; use databend_common_io::prelude::BinaryRead; @@ -29,7 +30,7 @@ use crate::meta::format::encode; use crate::meta::format::read_and_deserialize; use crate::meta::format::MetaCompression; use crate::meta::monotonically_increased_timestamp; -use crate::meta::trim_timestamp_to_micro_second; +use crate::meta::uuid_from_date_time; use crate::meta::v2; use crate::meta::v3; use crate::meta::ClusterKey; @@ -38,9 +39,10 @@ use crate::meta::Location; use crate::meta::MetaEncoding; use crate::meta::SnapshotId; use crate::meta::Statistics; +use crate::meta::TableMetaTimestamps; use crate::meta::Versioned; +use crate::readers::snapshot_reader::TableSnapshotAccessor; -/// The structure of the TableSnapshot is the same as that of v2, but the serialization and deserialization methods are different #[derive(Serialize, Deserialize, Clone, Debug)] pub struct TableSnapshot { /// format version of TableSnapshot meta data @@ -87,73 +89,79 @@ pub struct TableSnapshot { /// The metadata of the cluster keys. pub cluster_key_meta: Option, pub table_statistics_location: Option, - - pub least_visible_timestamp: Option>, } impl TableSnapshot { - pub fn new( - snapshot_id: SnapshotId, + /// Note that table_meta_timestamps is not always equal to prev_timestamp. + pub fn try_new( prev_table_seq: Option, - prev_timestamp: &Option>, - prev_snapshot_id: Option<(SnapshotId, FormatVersion)>, + prev_snapshot: Option>, schema: TableSchema, summary: Statistics, segments: Vec, cluster_key_meta: Option, table_statistics_location: Option, - ) -> Self { - let now = Utc::now(); - // make snapshot timestamp monotonically increased - let adjusted_timestamp = monotonically_increased_timestamp(now, prev_timestamp); - - // trim timestamp to micro seconds - let trimmed_timestamp = trim_timestamp_to_micro_second(adjusted_timestamp); - let timestamp = Some(trimmed_timestamp); + table_meta_timestamps: TableMetaTimestamps, + ) -> Result { + let TableMetaTimestamps { + segment_block_timestamp, + snapshot_timestamp, + } = table_meta_timestamps; + + let snapshot_timestamp = + monotonically_increased_timestamp(snapshot_timestamp, &prev_snapshot.timestamp()); + + if segment_block_timestamp < snapshot_timestamp { + return Err(ErrorCode::TransactionTimeout(format!( + "Snapshot is generated too late, segment_block_timestamp: {:?}, snapshot_timestamp: {:?}", + segment_block_timestamp, snapshot_timestamp + ))); + } - Self { + Ok(Self { format_version: TableSnapshot::VERSION, - snapshot_id, - timestamp, + snapshot_id: uuid_from_date_time(snapshot_timestamp), + timestamp: Some(snapshot_timestamp), prev_table_seq, - prev_snapshot_id, + prev_snapshot_id: prev_snapshot.snapshot_id(), schema, summary, segments, cluster_key_meta, table_statistics_location, - least_visible_timestamp: None, - } + }) } + /// used in ut pub fn new_empty_snapshot(schema: TableSchema, prev_table_seq: Option) -> Self { - Self::new( - Uuid::new_v4(), + Self::try_new( prev_table_seq, - &None, None, schema, Statistics::default(), vec![], None, None, + Default::default(), ) + .unwrap() } - pub fn from_previous(previous: &TableSnapshot, prev_table_seq: Option) -> Self { - let id = Uuid::new_v4(); - let clone = previous.clone(); + pub fn try_from_previous( + previous: Arc, + prev_table_seq: Option, + table_meta_timestamps: TableMetaTimestamps, + ) -> Result { // the timestamp of the new snapshot will be adjusted by the `new` method - Self::new( - id, + Self::try_new( prev_table_seq, - &clone.timestamp, - Some((clone.snapshot_id, clone.format_version)), - clone.schema, - clone.summary, - clone.segments, - clone.cluster_key_meta, - clone.table_statistics_location, + Some(previous.clone()), + previous.schema.clone(), + previous.summary.clone(), + previous.segments.clone(), + previous.cluster_key_meta.clone(), + previous.table_statistics_location.clone(), + table_meta_timestamps, ) } @@ -237,7 +245,6 @@ impl From for TableSnapshot { segments: s.segments, cluster_key_meta: s.cluster_key_meta, table_statistics_location: s.table_statistics_location, - least_visible_timestamp: None, } } } @@ -260,7 +267,6 @@ where T: Into segments: s.segments, cluster_key_meta: s.cluster_key_meta, table_statistics_location: s.table_statistics_location, - least_visible_timestamp: None, } } } diff --git a/src/query/storages/fuse/src/fuse_table.rs b/src/query/storages/fuse/src/fuse_table.rs index b63b59dc4d5b..79d66be43922 100644 --- a/src/query/storages/fuse/src/fuse_table.rs +++ b/src/query/storages/fuse/src/fuse_table.rs @@ -76,6 +76,7 @@ use databend_storages_common_table_meta::meta::ClusterKey; use databend_storages_common_table_meta::meta::CompactSegmentInfo; use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::Statistics as FuseStatistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; use databend_storages_common_table_meta::meta::Versioned; @@ -95,7 +96,6 @@ use log::info; use log::warn; use opendal::Operator; use parking_lot::Mutex; -use uuid::Uuid; use crate::fuse_column::FuseTableColumnStatisticsProvider; use crate::fuse_type::FuseTableType; @@ -221,14 +221,13 @@ impl FuseTable { .and_then(|s| s.parse::().ok()) .unwrap_or(BloomIndexColumns::All); + let meta_location_generator = TableMetaLocationGenerator::new(storage_prefix); if !table_info.meta.part_prefix.is_empty() { return Err(ErrorCode::StorageOther( "Location_prefix no longer supported. The last version that supports it is: https://github.com/databendlabs/databend/releases/tag/v1.2.653-nightly", )); } - let meta_location_generator = TableMetaLocationGenerator::with_prefix(storage_prefix); - Ok(Box::new(FuseTable { table_info, meta_location_generator, @@ -704,13 +703,10 @@ impl Table for FuseTable { let schema = self.schema().as_ref().clone(); let prev = self.read_table_snapshot().await?; - let prev_version = self.snapshot_format_version(None)?; - 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)); let prev_statistics_location = prev .as_ref() .and_then(|v| v.table_statistics_location.clone()); - let (summary, segments) = if let Some(v) = prev { + let (summary, segments) = if let Some(v) = &prev { (v.summary.clone(), v.segments.clone()) } else { (FuseStatistics::default(), vec![]) @@ -718,17 +714,16 @@ impl Table for FuseTable { let table_version = Some(self.get_table_info().ident.seq); - let new_snapshot = TableSnapshot::new( - Uuid::new_v4(), + let new_snapshot = TableSnapshot::try_new( table_version, - &prev_timestamp, - prev_snapshot_id, + prev.clone(), schema, summary, segments, cluster_key_meta, prev_statistics_location, - ); + ctx.get_table_meta_timestamps(self.get_id(), prev)?, + )?; let mut table_info = self.table_info.clone(); table_info.meta = new_table_meta; @@ -758,13 +753,10 @@ impl Table for FuseTable { let schema = self.schema().as_ref().clone(); let prev = self.read_table_snapshot().await?; - let prev_version = self.snapshot_format_version(None)?; - let prev_timestamp = prev.as_ref().and_then(|v| v.timestamp); let prev_statistics_location = prev .as_ref() .and_then(|v| v.table_statistics_location.clone()); - let prev_snapshot_id = prev.as_ref().map(|v| (v.snapshot_id, prev_version)); - let (summary, segments) = if let Some(v) = prev { + let (summary, segments) = if let Some(v) = &prev { (v.summary.clone(), v.segments.clone()) } else { (FuseStatistics::default(), vec![]) @@ -772,17 +764,16 @@ impl Table for FuseTable { let table_version = Some(self.get_table_info().ident.seq); - let new_snapshot = TableSnapshot::new( - Uuid::new_v4(), + let new_snapshot = TableSnapshot::try_new( table_version, - &prev_timestamp, - prev_snapshot_id, + prev.clone(), schema, summary, segments, None, prev_statistics_location, - ); + ctx.get_table_meta_timestamps(self.get_id(), prev)?, + )?; let mut table_info = self.table_info.clone(); table_info.meta = new_table_meta; @@ -821,6 +812,14 @@ impl Table for FuseTable { self.do_read_data(ctx, plan, pipeline, put_cache) } + fn append_data( + &self, + ctx: Arc, + pipeline: &mut Pipeline, + table_meta_timestamps: TableMetaTimestamps, + ) -> Result<()> { + self.do_append_data(ctx, pipeline, table_meta_timestamps) + } fn build_prune_pipeline( &self, table_ctx: Arc, @@ -830,10 +829,6 @@ impl Table for FuseTable { self.do_build_prune_pipeline(table_ctx, plan, source_pipeline) } - fn append_data(&self, ctx: Arc, pipeline: &mut Pipeline) -> Result<()> { - self.do_append_data(ctx, pipeline) - } - fn commit_insertion( &self, ctx: Arc, @@ -843,6 +838,7 @@ impl Table for FuseTable { overwrite: bool, prev_snapshot_id: Option, deduplicated_label: Option, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { self.do_commit( ctx, @@ -852,6 +848,7 @@ impl Table for FuseTable { overwrite, prev_snapshot_id, deduplicated_label, + table_meta_timestamps, ) } diff --git a/src/query/storages/fuse/src/io/locations.rs b/src/query/storages/fuse/src/io/locations.rs index 3de700ba04fe..c0d3b6a60149 100644 --- a/src/query/storages/fuse/src/io/locations.rs +++ b/src/query/storages/fuse/src/io/locations.rs @@ -16,10 +16,12 @@ use std::marker::PhantomData; use databend_common_exception::Result; use databend_common_expression::DataBlock; -use databend_storages_common_table_meta::meta::trim_vacuum2_object_prefix; +use databend_storages_common_table_meta::meta::trim_object_prefix; +use databend_storages_common_table_meta::meta::uuid_from_date_time; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SnapshotVersion; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshotStatisticsVersion; use databend_storages_common_table_meta::meta::Versioned; use databend_storages_common_table_meta::meta::VACUUM2_OBJECT_KEY_PREFIX; @@ -58,20 +60,22 @@ pub struct TableMetaLocationGenerator { block_location_prefix: String, segment_info_location_prefix: String, bloom_index_location_prefix: String, + snapshot_location_prefix: String, } impl TableMetaLocationGenerator { - pub fn with_prefix(prefix: String) -> Self { + pub fn new(prefix: String) -> Self { let block_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_BLOCK_PREFIX,); let bloom_index_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_XOR_BLOOM_INDEX_PREFIX); let segment_info_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_SEGMENT_PREFIX); - + let snapshot_location_prefix = format!("{}/{}/", &prefix, FUSE_TBL_SNAPSHOT_PREFIX); Self { prefix, block_location_prefix, segment_info_location_prefix, bloom_index_location_prefix, + snapshot_location_prefix, } } @@ -79,11 +83,31 @@ impl TableMetaLocationGenerator { &self.prefix } - pub fn gen_block_location(&self) -> (Location, Uuid) { - let part_uuid = Uuid::new_v4(); + pub fn block_location_prefix(&self) -> &str { + &self.block_location_prefix + } + + pub fn block_bloom_index_prefix(&self) -> &str { + &self.bloom_index_location_prefix + } + + pub fn segment_location_prefix(&self) -> &str { + &self.segment_info_location_prefix + } + + pub fn snapshot_location_prefix(&self) -> &str { + &self.snapshot_location_prefix + } + + pub fn gen_block_location( + &self, + table_meta_timestamps: TableMetaTimestamps, + ) -> (Location, Uuid) { + let part_uuid = uuid_from_date_time(table_meta_timestamps.segment_block_timestamp); let location_path = format!( - "{}{}_v{}.parquet", + "{}{}{}_v{}.parquet", self.block_location_prefix(), + VACUUM2_OBJECT_KEY_PREFIX, part_uuid.as_simple(), DataBlock::VERSION, ); @@ -91,10 +115,6 @@ impl TableMetaLocationGenerator { ((location_path, DataBlock::VERSION), part_uuid) } - pub fn block_location_prefix(&self) -> &str { - &self.block_location_prefix - } - pub fn block_bloom_index_location(&self, block_id: &Uuid) -> Location { ( format!( @@ -107,24 +127,17 @@ impl TableMetaLocationGenerator { ) } - pub fn block_bloom_index_prefix(&self) -> &str { - &self.bloom_index_location_prefix - } - - pub fn gen_segment_info_location(&self) -> String { - let segment_uuid = Uuid::new_v4().simple().to_string(); + pub fn gen_segment_info_location(&self, table_meta_timestamps: TableMetaTimestamps) -> String { + let segment_uuid = uuid_from_date_time(table_meta_timestamps.segment_block_timestamp); format!( - "{}{}_v{}.mpk", - self.segment_info_prefix(), - segment_uuid, + "{}{}{}_v{}.mpk", + &self.segment_location_prefix(), + VACUUM2_OBJECT_KEY_PREFIX, + segment_uuid.as_simple(), SegmentInfo::VERSION, ) } - pub fn segment_info_prefix(&self) -> &str { - &self.segment_info_location_prefix - } - pub fn snapshot_location_from_uuid(&self, id: &Uuid, version: u64) -> Result { let snapshot_version = SnapshotVersion::try_from(version)?; Ok(snapshot_version.create(id, &self.prefix)) @@ -181,7 +194,7 @@ impl TableMetaLocationGenerator { let splits = loc.split('/').collect::>(); let len = splits.len(); let prefix = splits[..len - 2].join("/"); - let block_name = trim_vacuum2_object_prefix(splits[len - 1]); + let block_name = trim_object_prefix(splits[len - 1]); format!("{prefix}/{FUSE_TBL_AGG_INDEX_PREFIX}/{index_id}/{block_name}") } @@ -193,7 +206,7 @@ impl TableMetaLocationGenerator { let splits = loc.split('/').collect::>(); let len = splits.len(); let prefix = splits[..len - 2].join("/"); - let block_name = trim_vacuum2_object_prefix(splits[len - 1]); + let block_name = trim_object_prefix(splits[len - 1]); let id: String = block_name.chars().take(32).collect(); let short_ver: String = index_version.chars().take(7).collect(); format!( @@ -206,6 +219,21 @@ impl TableMetaLocationGenerator { InvertedIndexFile::VERSION, ) } + + pub fn gen_bloom_index_location_from_block_location(loc: &str) -> String { + let splits = loc.split('/').collect::>(); + let len = splits.len(); + let prefix = splits[..len - 2].join("/"); + let block_name = trim_object_prefix(splits[len - 1]); + let id: String = block_name.chars().take(32).collect(); + format!( + "{}/{}/{}_v{}.parquet", + prefix, + FUSE_TBL_XOR_BLOOM_INDEX_PREFIX, + id, + BlockFilter::VERSION, + ) + } } trait SnapshotLocationCreator { @@ -214,6 +242,7 @@ trait SnapshotLocationCreator { } impl SnapshotLocationCreator for SnapshotVersion { + // todo rename this fn create(&self, id: &Uuid, prefix: impl AsRef) -> String { let vacuum_prefix = if id .get_version() diff --git a/src/query/storages/fuse/src/io/write/block_writer.rs b/src/query/storages/fuse/src/io/write/block_writer.rs index 29276b1563ba..3003caf4c688 100644 --- a/src/query/storages/fuse/src/io/write/block_writer.rs +++ b/src/query/storages/fuse/src/io/write/block_writer.rs @@ -48,6 +48,7 @@ use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::ClusterStatistics; use databend_storages_common_table_meta::meta::ColumnMeta; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::TableCompression; use opendal::Operator; @@ -343,6 +344,7 @@ pub struct BlockBuilder { pub cluster_stats_gen: ClusterStatsGenerator, pub bloom_columns_map: BTreeMap, pub inverted_index_builders: Vec, + pub table_meta_timestamps: TableMetaTimestamps, } impl BlockBuilder { @@ -350,7 +352,9 @@ impl BlockBuilder { where F: Fn(DataBlock, &ClusterStatsGenerator) -> Result<(Option, DataBlock)> { let (cluster_stats, data_block) = f(data_block, &self.cluster_stats_gen)?; - let (block_location, block_id) = self.meta_locations.gen_block_location(); + let (block_location, block_id) = self + .meta_locations + .gen_block_location(self.table_meta_timestamps); let bloom_index_location = self.meta_locations.block_bloom_index_location(&block_id); let bloom_index_state = BloomIndexState::from_data_block( diff --git a/src/query/storages/fuse/src/io/write/meta_writer.rs b/src/query/storages/fuse/src/io/write/meta_writer.rs index f38dc6dcd236..852975bd1dac 100644 --- a/src/query/storages/fuse/src/io/write/meta_writer.rs +++ b/src/query/storages/fuse/src/io/write/meta_writer.rs @@ -100,7 +100,6 @@ mod tests { use databend_common_base::runtime::catch_unwind; use databend_common_expression::TableSchema; - use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::Statistics; use super::*; @@ -127,17 +126,17 @@ mod tests { // old versions are not allowed (runtime panics) for v in 0..TableSnapshot::VERSION { let r = catch_unwind(|| { - let mut snapshot = TableSnapshot::new( - SnapshotId::new_v4(), + let mut snapshot = TableSnapshot::try_new( None, - &None, None, TableSchema::default(), Statistics::default(), vec![], None, None, - ); + Default::default(), + ) + .unwrap(); snapshot.format_version = v; let _ = snapshot.marshal(); }); @@ -145,17 +144,17 @@ mod tests { } // current version allowed - let snapshot = TableSnapshot::new( - SnapshotId::new_v4(), + let snapshot = TableSnapshot::try_new( None, - &None, None, TableSchema::default(), Statistics::default(), vec![], None, None, - ); + Default::default(), + ) + .unwrap(); snapshot.marshal().unwrap(); } } diff --git a/src/query/storages/fuse/src/operations/analyze.rs b/src/query/storages/fuse/src/operations/analyze.rs index 5cb0358d6d14..b5414dcaa110 100644 --- a/src/query/storages/fuse/src/operations/analyze.rs +++ b/src/query/storages/fuse/src/operations/analyze.rs @@ -261,8 +261,12 @@ impl SinkAnalyzeState { let (col_stats, cluster_stats) = regenerate_statistics(table, snapshot.as_ref(), &self.ctx).await?; // 4. Save table statistics - let mut new_snapshot = - TableSnapshot::from_previous(&snapshot, Some(table.get_table_info().ident.seq)); + let mut new_snapshot = TableSnapshot::try_from_previous( + snapshot.clone(), + Some(table.get_table_info().ident.seq), + self.ctx + .get_table_meta_timestamps(table.get_id(), Some(snapshot.clone()))?, + )?; new_snapshot.summary.col_stats = col_stats; new_snapshot.summary.cluster_stats = cluster_stats; new_snapshot.table_statistics_location = Some(table_statistics_location); diff --git a/src/query/storages/fuse/src/operations/append.rs b/src/query/storages/fuse/src/operations/append.rs index 0588a64b4c77..fb5fcc5feaea 100644 --- a/src/query/storages/fuse/src/operations/append.rs +++ b/src/query/storages/fuse/src/operations/append.rs @@ -33,6 +33,7 @@ use databend_common_pipeline_transforms::processors::TransformSortPartial; use databend_common_sql::evaluator::BlockOperator; use databend_common_sql::evaluator::CompoundBlockOperator; use databend_common_sql::executor::physical_plans::MutationKind; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use crate::operations::common::TransformSerializeBlock; use crate::statistics::ClusterStatsGenerator; @@ -43,6 +44,7 @@ impl FuseTable { &self, ctx: Arc, pipeline: &mut Pipeline, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { let block_thresholds = self.get_block_thresholds(); build_compact_block_pipeline(pipeline, block_thresholds)?; @@ -58,6 +60,7 @@ impl FuseTable { self, cluster_stats_gen.clone(), MutationKind::Insert, + table_meta_timestamps, )?; proc.into_processor() })?; diff --git a/src/query/storages/fuse/src/operations/commit.rs b/src/query/storages/fuse/src/operations/commit.rs index edc0a5c76124..e7d114777e0c 100644 --- a/src/query/storages/fuse/src/operations/commit.rs +++ b/src/query/storages/fuse/src/operations/commit.rs @@ -44,6 +44,7 @@ use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::SnapshotId; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::TableSnapshotStatistics; use databend_storages_common_table_meta::meta::Versioned; @@ -55,6 +56,7 @@ use log::info; use log::warn; use opendal::Operator; +use super::decorate_snapshot; use crate::io::MetaWriter; use crate::io::SegmentsIO; use crate::io::TableMetaLocationGenerator; @@ -78,13 +80,20 @@ impl FuseTable { overwrite: bool, prev_snapshot_id: Option, deduplicated_label: Option, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { let block_thresholds = self.get_block_thresholds(); pipeline.try_resize(1)?; pipeline.add_transform(|input, output| { - let proc = TransformSerializeSegment::new(input, output, self, block_thresholds); + let proc = TransformSerializeSegment::new( + input, + output, + self, + block_thresholds, + table_meta_timestamps, + ); proc.into_processor() })?; @@ -97,6 +106,7 @@ impl FuseTable { vec![], Statistics::default(), MutationKind::Insert, + table_meta_timestamps, ) }); @@ -112,6 +122,7 @@ impl FuseTable { None, prev_snapshot_id, deduplicated_label.clone(), + table_meta_timestamps, ) })?; @@ -335,12 +346,15 @@ impl FuseTable { // Status ctx.set_status_info("mutation: begin try to commit"); + let table_meta_timestamps = + ctx.get_table_meta_timestamps(self.get_id(), Some(base_snapshot.clone()))?; loop { - let mut snapshot_tobe_committed = TableSnapshot::from_previous( - latest_snapshot.as_ref(), + let mut snapshot_tobe_committed = TableSnapshot::try_from_previous( + latest_snapshot.clone(), Some(latest_table_info.ident.seq), - ); + table_meta_timestamps, + )?; let schema = self.schema(); let (segments_tobe_committed, statistics_tobe_committed) = Self::merge_with_base( @@ -356,6 +370,13 @@ impl FuseTable { snapshot_tobe_committed.segments = segments_tobe_committed; snapshot_tobe_committed.summary = statistics_tobe_committed; + decorate_snapshot( + &mut snapshot_tobe_committed, + ctx.txn_mgr(), + Some(base_snapshot.clone()), + self.get_id(), + )?; + match Self::commit_to_meta_server( ctx.as_ref(), latest_table_info, diff --git a/src/query/storages/fuse/src/operations/common/generators/append_generator.rs b/src/query/storages/fuse/src/operations/common/generators/append_generator.rs index 015621e92c65..52f43ec70f6d 100644 --- a/src/query/storages/fuse/src/operations/common/generators/append_generator.rs +++ b/src/query/storages/fuse/src/operations/common/generators/append_generator.rs @@ -28,10 +28,10 @@ use databend_common_sql::field_default_value; use databend_storages_common_table_meta::meta::ClusterKey; use databend_storages_common_table_meta::meta::ColumnStatistics; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use log::info; use log::warn; -use uuid::Uuid; use crate::operations::common::ConflictResolveContext; use crate::operations::common::SnapshotGenerator; @@ -119,6 +119,7 @@ impl SnapshotGenerator for AppendGenerator { cluster_key_meta: Option, previous: &Option>, prev_table_seq: Option, + table_meta_timestamps: TableMetaTimestamps, table_name: &str, ) -> Result { let (snapshot_merged, expected_schema) = self.conflict_resolve_ctx()?; @@ -128,15 +129,11 @@ impl SnapshotGenerator for AppendGenerator { expected_schema, schema ))); } - let mut prev_timestamp = None; - let mut prev_snapshot_id = None; let mut table_statistics_location = None; let mut new_segments = snapshot_merged.merged_segments.clone(); let mut new_summary = snapshot_merged.merged_statistics.clone(); if let Some(snapshot) = previous { - prev_timestamp = snapshot.timestamp; - prev_snapshot_id = Some((snapshot.snapshot_id, snapshot.format_version)); table_statistics_location = snapshot.table_statistics_location.clone(); if !self.overwrite { @@ -227,17 +224,16 @@ impl SnapshotGenerator for AppendGenerator { .set_compaction_num_block_hint(table_name, compact_num_block_hint); } - Ok(TableSnapshot::new( - Uuid::new_v4(), + TableSnapshot::try_new( prev_table_seq, - &prev_timestamp, - prev_snapshot_id, + previous.clone(), schema, new_summary, new_segments, cluster_key_meta, table_statistics_location, - )) + table_meta_timestamps, + ) } } diff --git a/src/query/storages/fuse/src/operations/common/generators/mod.rs b/src/query/storages/fuse/src/operations/common/generators/mod.rs index 4d0308222521..c02421090c52 100644 --- a/src/query/storages/fuse/src/operations/common/generators/mod.rs +++ b/src/query/storages/fuse/src/operations/common/generators/mod.rs @@ -23,6 +23,7 @@ pub use conflict_resolve_context::ConflictResolveContext; pub use conflict_resolve_context::SnapshotChanges; pub use conflict_resolve_context::SnapshotMerged; pub use mutation_generator::MutationGenerator; +pub use snapshot_generator::decorate_snapshot; pub use snapshot_generator::SnapshotGenerator; pub use truncate_generator::TruncateGenerator; pub use truncate_generator::TruncateMode; diff --git a/src/query/storages/fuse/src/operations/common/generators/mutation_generator.rs b/src/query/storages/fuse/src/operations/common/generators/mutation_generator.rs index ecc768366d3b..4185846f8284 100644 --- a/src/query/storages/fuse/src/operations/common/generators/mutation_generator.rs +++ b/src/query/storages/fuse/src/operations/common/generators/mutation_generator.rs @@ -21,10 +21,10 @@ use databend_common_expression::TableSchema; use databend_common_metrics::storage::*; use databend_common_sql::executor::physical_plans::MutationKind; use databend_storages_common_table_meta::meta::ClusterKey; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::readers::snapshot_reader::TableSnapshotAccessor; use log::info; -use uuid::Uuid; use crate::operations::common::ConflictResolveContext; use crate::operations::common::SnapshotGenerator; @@ -63,6 +63,7 @@ impl SnapshotGenerator for MutationGenerator { cluster_key_meta: Option, previous: &Option>, prev_table_seq: Option, + table_meta_timestamps: TableMetaTimestamps, _table_name: &str, ) -> Result { let default_cluster_key_id = cluster_key_meta.clone().map(|v| v.0); @@ -90,17 +91,16 @@ impl SnapshotGenerator for MutationGenerator { default_cluster_key_id, ); deduct_statistics_mut(&mut new_summary, &ctx.removed_statistics); - let new_snapshot = TableSnapshot::new( - Uuid::new_v4(), + let new_snapshot = TableSnapshot::try_new( prev_table_seq, - &previous.timestamp(), - previous.snapshot_id(), + previous.clone(), schema, new_summary, new_segments, cluster_key_meta, previous.table_statistics_location(), - ); + table_meta_timestamps, + )?; if matches!( self.mutation_kind, diff --git a/src/query/storages/fuse/src/operations/common/generators/snapshot_generator.rs b/src/query/storages/fuse/src/operations/common/generators/snapshot_generator.rs index 9310931dc6c6..facc73ffc8f6 100644 --- a/src/query/storages/fuse/src/operations/common/generators/snapshot_generator.rs +++ b/src/query/storages/fuse/src/operations/common/generators/snapshot_generator.rs @@ -19,6 +19,7 @@ use databend_common_exception::Result; use databend_common_expression::TableSchema; use databend_storages_common_session::TxnManagerRef; use databend_storages_common_table_meta::meta::ClusterKey; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use crate::operations::common::ConflictResolveContext; @@ -46,6 +47,7 @@ pub trait SnapshotGenerator { prev_table_seq: Option, txn_mgr: TxnManagerRef, table_id: u64, + table_meta_timestamps: TableMetaTimestamps, table_name: &str, ) -> Result { let mut snapshot = self.do_generate_new_snapshot( @@ -53,23 +55,10 @@ pub trait SnapshotGenerator { cluster_key_meta, &previous, prev_table_seq, + table_meta_timestamps, table_name, )?; - - let has_pending_transactional_mutations = { - let guard = txn_mgr.lock(); - // NOTE: - // When generating a new snapshot for a mutation of table for the first time, - // there is no buffered table ID inside txn_mgr for this table. - guard.is_active() && guard.get_table_from_buffer_by_id(table_id).is_some() - }; - - if has_pending_transactional_mutations { - // Adjust the `prev_snapshot_id` of the newly created snapshot to match the - // `prev_snapshot_id` of the table when it first appeared in the transaction. - let previous_of_previous = previous.as_ref().and_then(|prev| prev.prev_snapshot_id); - snapshot.prev_snapshot_id = previous_of_previous; - } + decorate_snapshot(&mut snapshot, txn_mgr, previous, table_id)?; Ok(snapshot) } @@ -79,6 +68,30 @@ pub trait SnapshotGenerator { cluster_key_meta: Option, previous: &Option>, prev_table_seq: Option, + table_meta_timestamps: TableMetaTimestamps, table_name: &str, ) -> Result; } + +pub fn decorate_snapshot( + snapshot: &mut TableSnapshot, + txn_mgr: TxnManagerRef, + previous: Option>, + table_id: u64, +) -> Result<()> { + let has_pending_transactional_mutations = { + let guard = txn_mgr.lock(); + // NOTE: + // When generating a new snapshot for a mutation of table for the first time, + // there is no buffered table ID inside txn_mgr for this table. + guard.is_active() && guard.get_table_from_buffer_by_id(table_id).is_some() + }; + + if has_pending_transactional_mutations { + // Adjust the `prev_snapshot_id` of the newly created snapshot to match the + // `prev_snapshot_id` of the table when it first appeared in the transaction. + let previous_of_previous = previous.as_ref().and_then(|prev| prev.prev_snapshot_id); + snapshot.prev_snapshot_id = previous_of_previous; + } + Ok(()) +} diff --git a/src/query/storages/fuse/src/operations/common/generators/truncate_generator.rs b/src/query/storages/fuse/src/operations/common/generators/truncate_generator.rs index c6c7834e2943..2ed1d518d6c3 100644 --- a/src/query/storages/fuse/src/operations/common/generators/truncate_generator.rs +++ b/src/query/storages/fuse/src/operations/common/generators/truncate_generator.rs @@ -18,8 +18,8 @@ use std::sync::Arc; use databend_common_exception::Result; use databend_common_expression::TableSchema; use databend_storages_common_table_meta::meta::ClusterKey; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; -use uuid::Uuid; use crate::operations::common::SnapshotGenerator; @@ -60,28 +60,18 @@ impl SnapshotGenerator for TruncateGenerator { cluster_key_meta: Option, previous: &Option>, prev_table_seq: Option, + table_meta_timestamps: TableMetaTimestamps, _table_name: &str, ) -> Result { - let (prev_timestamp, prev_snapshot_id) = if let Some(prev_snapshot) = previous { - ( - prev_snapshot.timestamp, - Some((prev_snapshot.snapshot_id, prev_snapshot.format_version)), - ) - } else { - (None, None) - }; - - let new_snapshot = TableSnapshot::new( - Uuid::new_v4(), + TableSnapshot::try_new( prev_table_seq, - &prev_timestamp, - prev_snapshot_id, + previous.clone(), schema, Default::default(), vec![], cluster_key_meta, None, - ); - Ok(new_snapshot) + table_meta_timestamps, + ) } } 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 49e76d5332dd..40dcdaa7eceb 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 @@ -33,6 +33,7 @@ use databend_common_meta_app::schema::UpdateTempTableReq; use databend_common_meta_types::MatchSeq; use databend_common_pipeline_sinks::AsyncSink; use databend_storages_common_session::TxnManagerRef; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use log::debug; @@ -54,6 +55,7 @@ pub struct CommitMultiTableInsert { update_stream_meta: Vec, deduplicated_label: Option, catalog: Arc, + table_meta_timestampss: HashMap, } impl CommitMultiTableInsert { @@ -64,6 +66,7 @@ impl CommitMultiTableInsert { update_stream_meta: Vec, deduplicated_label: Option, catalog: Arc, + table_meta_timestampss: HashMap, ) -> Self { Self { commit_metas: Default::default(), @@ -73,6 +76,7 @@ impl CommitMultiTableInsert { update_stream_meta, deduplicated_label, catalog, + table_meta_timestampss, } } } @@ -104,6 +108,7 @@ impl AsyncSink for CommitMultiTableInsert { table.as_ref(), &snapshot_generator, self.ctx.txn_mgr(), + *self.table_meta_timestampss.get(&table.get_id()).unwrap(), ) .await?, table.get_table_info().clone(), @@ -191,6 +196,7 @@ impl AsyncSink for CommitMultiTableInsert { table.as_ref(), snapshot_generators.get(&tid).unwrap(), self.ctx.txn_mgr(), + *self.table_meta_timestampss.get(&tid).unwrap(), ) .await?; break; @@ -245,6 +251,7 @@ async fn build_update_table_meta_req( table: &dyn Table, snapshot_generator: &AppendGenerator, txn_mgr: TxnManagerRef, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { let fuse_table = FuseTable::try_from_table(table)?; let previous = fuse_table.read_table_snapshot().await?; @@ -255,6 +262,7 @@ async fn build_update_table_meta_req( Some(fuse_table.table_info.ident.seq), txn_mgr, table.get_id(), + table_meta_timestamps, table.name(), )?; 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 551d0aa2c1c3..3c38dc737bf7 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 @@ -37,6 +37,7 @@ use databend_common_pipeline_core::processors::ProcessorPtr; use databend_storages_common_table_meta::meta::ClusterKey; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SnapshotId; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::TableSnapshot; use databend_storages_common_table_meta::meta::Versioned; use log::debug; @@ -53,7 +54,6 @@ use crate::operations::SnapshotGenerator; use crate::operations::TruncateGenerator; use crate::operations::TruncateMode; use crate::FuseTable; - enum State { None, FillDefault, @@ -97,6 +97,7 @@ pub struct CommitSink { change_tracking: bool, update_stream_meta: Vec, deduplicated_label: Option, + table_meta_timestamps: TableMetaTimestamps, } impl CommitSink @@ -113,6 +114,7 @@ where F: SnapshotGenerator + Send + 'static max_retry_elapsed: Option, prev_snapshot_id: Option, deduplicated_label: Option, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { let purge = Self::do_purge(table, &snapshot_gen); Ok(ProcessorPtr::create(Box::new(CommitSink { @@ -134,6 +136,7 @@ where F: SnapshotGenerator + Send + 'static change_tracking: table.change_tracking_enabled(), update_stream_meta, deduplicated_label, + table_meta_timestamps, }))) } @@ -297,6 +300,7 @@ where F: SnapshotGenerator + Send + 'static Some(table_info.ident.seq), self.ctx.txn_mgr(), table_info.ident.table_id, + self.table_meta_timestamps, table_info.name.as_str(), ) { Ok(snapshot) => { diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs index 3a42880d3b31..ed3c6ebe2355 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_mutation_aggregator.rs @@ -33,6 +33,7 @@ use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::Location; use databend_storages_common_table_meta::meta::SegmentInfo; use databend_storages_common_table_meta::meta::Statistics; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::Versioned; use itertools::Itertools; use log::debug; @@ -82,6 +83,7 @@ pub struct TableMutationAggregator { start_time: Instant, finished_tasks: usize, table_id: u64, + table_meta_timestamps: TableMetaTimestamps, } // takes in table mutation logs and aggregates them (former mutation_transform) @@ -133,6 +135,7 @@ impl TableMutationAggregator { removed_segment_indexes: Vec, removed_statistics: Statistics, kind: MutationKind, + table_meta_timestamps: TableMetaTimestamps, ) -> Self { TableMutationAggregator { ctx, @@ -154,6 +157,7 @@ impl TableMutationAggregator { finished_tasks: 0, start_time: Instant::now(), table_id: table.get_id(), + table_meta_timestamps, } } @@ -264,6 +268,7 @@ impl TableMutationAggregator { let location_gen = self.location_gen.clone(); let op = self.dal.clone(); + let table_meta_timestamps = self.table_meta_timestamps; tasks.push(async move { write_segment( op, @@ -273,6 +278,7 @@ impl TableMutationAggregator { default_cluster_key, all_perfect, MutationKind::Recluster, + table_meta_timestamps, ) .await }); @@ -415,6 +421,7 @@ impl TableMutationAggregator { let location_gen = self.location_gen.clone(); let kind = self.kind; + let table_meta_timestamps = self.table_meta_timestamps; let mut all_perfect = false; tasks.push(async move { let (new_blocks, origin_summary) = if let Some(loc) = location { @@ -470,6 +477,7 @@ impl TableMutationAggregator { default_cluster_key_id, all_perfect, kind, + table_meta_timestamps, ) .await?; @@ -542,8 +550,9 @@ async fn write_segment( default_cluster_key: Option, all_perfect: bool, kind: MutationKind, + table_meta_timestamps: TableMetaTimestamps, ) -> Result<(String, Statistics)> { - let location = location_gen.gen_segment_info_location(); + let location = location_gen.gen_segment_info_location(table_meta_timestamps); let mut new_summary = reduce_block_metas(&blocks, thresholds, default_cluster_key); if all_perfect { // To fix issue #13217. diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs index 5064f96e729c..219afe8aaf5f 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_block.rs @@ -33,6 +33,7 @@ use databend_common_pipeline_core::PipeItem; use databend_common_sql::executor::physical_plans::MutationKind; use databend_common_storage::MutationStatus; use databend_storages_common_index::BloomIndex; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use opendal::Operator; use crate::io::create_inverted_index_builders; @@ -81,8 +82,18 @@ impl TransformSerializeBlock { table: &FuseTable, cluster_stats_gen: ClusterStatsGenerator, kind: MutationKind, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { - Self::do_create(ctx, input, output, table, cluster_stats_gen, kind, false) + Self::do_create( + ctx, + input, + output, + table, + cluster_stats_gen, + kind, + false, + table_meta_timestamps, + ) } pub fn try_create_with_tid( @@ -92,8 +103,18 @@ impl TransformSerializeBlock { table: &FuseTable, cluster_stats_gen: ClusterStatsGenerator, kind: MutationKind, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { - Self::do_create(ctx, input, output, table, cluster_stats_gen, kind, true) + Self::do_create( + ctx, + input, + output, + table, + cluster_stats_gen, + kind, + true, + table_meta_timestamps, + ) } fn do_create( @@ -104,6 +125,7 @@ impl TransformSerializeBlock { cluster_stats_gen: ClusterStatsGenerator, kind: MutationKind, with_tid: bool, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { // remove virtual computed fields. let mut fields = table @@ -138,6 +160,7 @@ impl TransformSerializeBlock { cluster_stats_gen, bloom_columns_map, inverted_index_builders, + table_meta_timestamps, }; Ok(TransformSerializeBlock { state: State::Consume, diff --git a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs index 9c392eccc360..a7685dfb1055 100644 --- a/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs +++ b/src/query/storages/fuse/src/operations/common/processors/transform_serialize_segment.rs @@ -31,6 +31,7 @@ use databend_storages_common_cache::CacheAccessor; use databend_storages_common_cache::CachedObject; use databend_storages_common_table_meta::meta::BlockMeta; use databend_storages_common_table_meta::meta::SegmentInfo; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::meta::Versioned; use log::info; use opendal::Operator; @@ -70,6 +71,7 @@ pub struct TransformSerializeSegment { thresholds: BlockThresholds, default_cluster_key_id: Option, + table_meta_timestamps: TableMetaTimestamps, } impl TransformSerializeSegment { @@ -78,6 +80,7 @@ impl TransformSerializeSegment { output: Arc, table: &FuseTable, thresholds: BlockThresholds, + table_meta_timestamps: TableMetaTimestamps, ) -> Self { let default_cluster_key_id = table.cluster_key_id(); TransformSerializeSegment { @@ -93,6 +96,7 @@ impl TransformSerializeSegment { as u64, thresholds, default_cluster_key_id, + table_meta_timestamps, } } @@ -186,7 +190,9 @@ impl Processor for TransformSerializeSegment { self.state = State::SerializedSegment { data: segment_info.to_bytes()?, - location: self.meta_locations.gen_segment_info_location(), + location: self + .meta_locations + .gen_segment_info_location(self.table_meta_timestamps), segment: Arc::new(segment_info), } } diff --git a/src/query/storages/fuse/src/operations/gc.rs b/src/query/storages/fuse/src/operations/gc.rs index 94b265843437..8b5a33ec8e42 100644 --- a/src/query/storages/fuse/src/operations/gc.rs +++ b/src/query/storages/fuse/src/operations/gc.rs @@ -676,7 +676,9 @@ impl FuseTable { locations_to_be_purged: HashSet, ) -> Result<()> { let fuse_file = Files::create(ctx.clone(), self.operator.clone()); - fuse_file.remove_file_in_batch(locations_to_be_purged).await + fuse_file + .remove_file_in_batch(locations_to_be_purged, false) + .await } // Purge file by location chunks. diff --git a/src/query/storages/fuse/src/operations/merge.rs b/src/query/storages/fuse/src/operations/merge.rs index 62c00e26dd56..309cee80a714 100644 --- a/src/query/storages/fuse/src/operations/merge.rs +++ b/src/query/storages/fuse/src/operations/merge.rs @@ -22,6 +22,7 @@ use databend_common_expression::TableSchemaRef; use databend_common_pipeline_core::PipeItem; use databend_storages_common_index::BloomIndex; use databend_storages_common_table_meta::meta::Location; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use super::merge_into::MatchedAggregator; use super::mutation::SegmentIndex; @@ -82,6 +83,7 @@ impl FuseTable { io_request_semaphore: Arc, segment_locations: Vec<(SegmentIndex, Location)>, target_build_optimization: bool, + table_meta_timestamps: TableMetaTimestamps, ) -> Result { let new_schema: TableSchemaRef = self .schema_with_stream() @@ -100,6 +102,7 @@ impl FuseTable { cluster_stats_gen, bloom_columns_map, inverted_index_builders, + table_meta_timestamps, }; let aggregator = MatchedAggregator::create( ctx, diff --git a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs index 3dbcf5856760..394f92d009a4 100644 --- a/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs +++ b/src/query/storages/fuse/src/operations/mutation/mutator/segment_compact_mutator.rs @@ -355,7 +355,9 @@ impl<'a> SegmentCompactor<'a> { // 2.2 write down new segment let new_segment = SegmentInfo::new(blocks, new_statistics); - let location = self.location_generator.gen_segment_info_location(); + let location = self + .location_generator + .gen_segment_info_location(Default::default()); new_segment .write_meta_through_cache(self.operator, &location) .await?; diff --git a/src/query/storages/fuse/src/operations/navigate.rs b/src/query/storages/fuse/src/operations/navigate.rs index 2f2ad06df661..84648a960325 100644 --- a/src/query/storages/fuse/src/operations/navigate.rs +++ b/src/query/storages/fuse/src/operations/navigate.rs @@ -25,6 +25,7 @@ use databend_common_exception::ResultExt; use databend_common_meta_app::schema::TableInfo; use databend_common_meta_app::schema::TableStatistics; use databend_storages_common_table_meta::meta::TableSnapshot; +use databend_storages_common_table_meta::meta::VACUUM2_OBJECT_KEY_PREFIX; use databend_storages_common_table_meta::table::OPT_KEY_SNAPSHOT_LOCATION; use databend_storages_common_table_meta::table::OPT_KEY_SOURCE_TABLE_ID; use futures::TryStreamExt; @@ -286,6 +287,7 @@ impl FuseTable { snapshot_id: &str, retention_point: DateTime, ) -> Result<(String, Vec)> { + // TODO(Sky): unify location related logic into a single place let mut location = None; let prefix = format!( "{}/{}/", @@ -293,10 +295,11 @@ impl FuseTable { FUSE_TBL_SNAPSHOT_PREFIX, ); let prefix_loc = format!("{}{}", prefix, snapshot_id); + let prefix_loc_v5 = format!("{}{}{}", prefix, VACUUM2_OBJECT_KEY_PREFIX, snapshot_id); let files = self .list_files(prefix, |loc, modified| { - if loc.starts_with(&prefix_loc) { + if loc.starts_with(&prefix_loc) || loc.starts_with(&prefix_loc_v5) { location = Some(loc); } modified <= retention_point diff --git a/src/query/storages/fuse/src/operations/truncate.rs b/src/query/storages/fuse/src/operations/truncate.rs index 6da256d0f730..4f481d35ba3b 100644 --- a/src/query/storages/fuse/src/operations/truncate.rs +++ b/src/query/storages/fuse/src/operations/truncate.rs @@ -73,6 +73,8 @@ impl FuseTable { )?; let snapshot_gen = TruncateGenerator::new(mode); + let table_meta_timestamps = + ctx.get_table_meta_timestamps(self.get_id(), Some(prev_snapshot))?; pipeline.add_sink(|input| { CommitSink::try_create( self, @@ -84,6 +86,7 @@ impl FuseTable { None, prev_snapshot_id, None, + table_meta_timestamps, ) }) } diff --git a/src/query/storages/fuse/src/table_functions/table_args.rs b/src/query/storages/fuse/src/table_functions/table_args.rs index e84ddaf7c52e..df8d2fbbc3e1 100644 --- a/src/query/storages/fuse/src/table_functions/table_args.rs +++ b/src/query/storages/fuse/src/table_functions/table_args.rs @@ -28,10 +28,21 @@ pub fn string_value(value: &Scalar) -> Result { } } +pub fn bool_value(value: &Scalar) -> Result { + match value { + Scalar::Boolean(val) => Ok(*val), + _ => Err(ErrorCode::BadArguments("invalid boolean.")), + } +} + pub fn string_literal(val: &str) -> Scalar { Scalar::String(val.to_string()) } +pub fn bool_literal(val: bool) -> Scalar { + Scalar::Boolean(val) +} + pub fn u64_literal(val: u64) -> Scalar { Scalar::Number(NumberScalar::UInt64(val)) } diff --git a/src/query/storages/hive/hive/src/hive_table.rs b/src/query/storages/hive/hive/src/hive_table.rs index 3d505a6520e3..26cc30a8a8ba 100644 --- a/src/query/storages/hive/hive/src/hive_table.rs +++ b/src/query/storages/hive/hive/src/hive_table.rs @@ -54,6 +54,7 @@ use databend_common_storages_parquet::ParquetRSPruner; use databend_common_storages_parquet::ParquetRSReaderBuilder; use databend_storages_common_pruner::partition_prunner::PartitionPruner; use databend_storages_common_table_meta::meta::SnapshotId; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::ChangeType; use futures::TryStreamExt; use log::info; @@ -450,6 +451,7 @@ impl Table for HiveTable { _overwrite: bool, _prev_snapshot_id: Option, _deduplicated_label: Option, + _table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { Err(ErrorCode::Unimplemented(format!( "commit_insertion operation for table {} is not implemented, table engine is {}", diff --git a/src/query/storages/memory/src/memory_table.rs b/src/query/storages/memory/src/memory_table.rs index 2ec25250b6c7..1d21f4e0e60d 100644 --- a/src/query/storages/memory/src/memory_table.rs +++ b/src/query/storages/memory/src/memory_table.rs @@ -51,6 +51,7 @@ use databend_common_storage::StorageMetrics; use databend_storages_common_blocks::memory::InMemoryDataKey; use databend_storages_common_blocks::memory::IN_MEMORY_DATA; use databend_storages_common_table_meta::meta::SnapshotId; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use databend_storages_common_table_meta::table::ChangeType; use databend_storages_common_table_meta::table::OPT_KEY_TEMP_PREFIX; use parking_lot::Mutex; @@ -235,7 +236,12 @@ impl Table for MemoryTable { ) } - fn append_data(&self, _ctx: Arc, _pipeline: &mut Pipeline) -> Result<()> { + fn append_data( + &self, + _ctx: Arc, + _pipeline: &mut Pipeline, + _table_meta_timestamps: TableMetaTimestamps, + ) -> Result<()> { Ok(()) } @@ -248,6 +254,7 @@ impl Table for MemoryTable { overwrite: bool, _prev_snapshot_id: Option, _deduplicated_label: Option, + _table_meta_timestamps: TableMetaTimestamps, ) -> Result<()> { pipeline.try_resize(1)?; diff --git a/src/query/storages/null/Cargo.toml b/src/query/storages/null/Cargo.toml index 40d952258c81..3aa6194f44d3 100644 --- a/src/query/storages/null/Cargo.toml +++ b/src/query/storages/null/Cargo.toml @@ -20,6 +20,7 @@ databend-common-meta-app = { workspace = true } databend-common-pipeline-core = { workspace = true } databend-common-pipeline-sinks = { workspace = true } databend-common-pipeline-sources = { workspace = true } +databend-storages-common-table-meta = { workspace = true } [lints] workspace = true diff --git a/src/query/storages/null/src/null_table.rs b/src/query/storages/null/src/null_table.rs index 5bb625429694..0821bbc10607 100644 --- a/src/query/storages/null/src/null_table.rs +++ b/src/query/storages/null/src/null_table.rs @@ -32,6 +32,7 @@ use databend_common_pipeline_core::Pipeline; use databend_common_pipeline_sinks::EmptySink; use databend_common_pipeline_sources::SyncSource; use databend_common_pipeline_sources::SyncSourcer; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; pub struct NullTable { table_info: TableInfo, @@ -92,7 +93,12 @@ impl Table for NullTable { Ok(()) } - fn append_data(&self, _: Arc, pipeline: &mut Pipeline) -> Result<()> { + fn append_data( + &self, + _: Arc, + pipeline: &mut Pipeline, + _table_meta_timestamps: TableMetaTimestamps, + ) -> Result<()> { pipeline.add_sink(|input| Ok(ProcessorPtr::create(EmptySink::create(input))))?; Ok(()) } diff --git a/src/query/storages/stage/src/stage_table.rs b/src/query/storages/stage/src/stage_table.rs index 2675f4e0b1f8..ca7a25875aa3 100644 --- a/src/query/storages/stage/src/stage_table.rs +++ b/src/query/storages/stage/src/stage_table.rs @@ -36,6 +36,7 @@ use databend_common_storage::StageFileInfo; use databend_common_storages_orc::OrcTableForCopy; use databend_common_storages_parquet::ParquetTableForCopy; use databend_storages_common_stage::SingleFilePartition; +use databend_storages_common_table_meta::meta::TableMetaTimestamps; use opendal::Operator; use crate::read::row_based::RowBasedReadPipelineBuilder; @@ -202,7 +203,12 @@ impl Table for StageTable { } } - fn append_data(&self, ctx: Arc, pipeline: &mut Pipeline) -> Result<()> { + fn append_data( + &self, + ctx: Arc, + pipeline: &mut Pipeline, + _table_meta_timestamps: TableMetaTimestamps, + ) -> Result<()> { self.do_append_data(ctx, pipeline) } diff --git a/tests/sqllogictests/suites/base/06_show/06_0014_show_table_functions.test b/tests/sqllogictests/suites/base/06_show/06_0014_show_table_functions.test index d55bb6c5744c..134f178de932 100644 --- a/tests/sqllogictests/suites/base/06_show/06_0014_show_table_functions.test +++ b/tests/sqllogictests/suites/base/06_show/06_0014_show_table_functions.test @@ -16,6 +16,7 @@ fuse_segment fuse_snapshot fuse_statistic fuse_time_travel_size +fuse_vacuum2 fuse_vacuum_temporary_table query T diff --git a/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test b/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test index 0d9d0ea33133..5c2cd2d0710f 100644 --- a/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test +++ b/tests/sqllogictests/suites/base/09_fuse_engine/09_0008_fuse_optimize_table.test @@ -840,5 +840,4 @@ select cluster_key, type, average_overlaps, average_depth from clustering_inform statement ok -DROP DATABASE db_09_0008 - +DROP DATABASE db_09_0008 \ No newline at end of file diff --git a/tests/suites/1_stateful/02_query/02_0001_create_table_with_external_location.sh b/tests/suites/1_stateful/02_query/02_0001_create_table_with_external_location.sh index 1d52ed0dbb0b..0a0a3293933a 100755 --- a/tests/suites/1_stateful/02_query/02_0001_create_table_with_external_location.sh +++ b/tests/suites/1_stateful/02_query/02_0001_create_table_with_external_location.sh @@ -4,7 +4,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../../../shell_env.sh echo "drop table if exists table_external_location;" | $BENDSQL_CLIENT_CONNECT -echo "drop table if exists table_external_location_with_location_prefix;" | $BENDSQL_CLIENT_CONNECT ## Create table echo "create table table_external_location(a int) 's3://testbucket/admin/data/' connection=(access_key_id ='minioadmin' secret_access_key ='minioadmin' endpoint_url='${STORAGE_S3_ENDPOINT_URL}');" | $BENDSQL_CLIENT_CONNECT diff --git a/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.result b/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.result new file mode 100644 index 000000000000..c142d1c73b83 --- /dev/null +++ b/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.result @@ -0,0 +1,2 @@ +>>>> create or replace table test_vacuum2(a int); +>>>> insert into test_vacuum2 values(1); diff --git a/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.sh b/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.sh new file mode 100755 index 000000000000..1a59b81f6729 --- /dev/null +++ b/tests/suites/8_faked_time_prepare/00_prepare/00_0001_vacuum2_prepare.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +stmt "create or replace table test_vacuum2(a int);" + +stmt "insert into test_vacuum2 values(1);" \ No newline at end of file diff --git a/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.result b/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.result new file mode 100644 index 000000000000..25838f156f1c --- /dev/null +++ b/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.result @@ -0,0 +1,2 @@ +>>>> create or replace table test_vacuum2_respect_time_travel(a int); +>>>> insert into test_vacuum2_respect_time_travel values(1); diff --git a/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.sh b/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.sh new file mode 100755 index 000000000000..b2dd08fdf333 --- /dev/null +++ b/tests/suites/8_faked_time_prepare/00_prepare/00_0002_vacuum2_respect_time_travel.sh @@ -0,0 +1,8 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +stmt "create or replace table test_vacuum2_respect_time_travel(a int);" + +stmt "insert into test_vacuum2_respect_time_travel values(1);" \ No newline at end of file diff --git a/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.result b/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.result new file mode 100644 index 000000000000..5f5a2cf6a2fd --- /dev/null +++ b/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.result @@ -0,0 +1,13 @@ +>>>> insert into test_vacuum2 values(2); +>>>> set data_retention_time_in_days = 2;truncate table test_vacuum2; +>>>> insert into test_vacuum2 values(3); +>>>> select count(*) from fuse_snapshot('default','test_vacuum2') +4 +<<<< +>>>> select count(*) from fuse_snapshot('default','test_vacuum2') +1 +<<<< +>>>> select * from test_vacuum2; +3 +<<<< +>>>> set data_retention_time_in_days = 1; diff --git a/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.sh b/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.sh new file mode 100755 index 000000000000..5e5ec20e830f --- /dev/null +++ b/tests/suites/9_faked_time/00_dummy_cases/00_0002_vacuum2.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +SEGMENTS=$(echo "select file_location from fuse_segment('default','test_vacuum2');" | $BENDSQL_CLIENT_CONNECT) +BLOCKS=$(echo "select block_location from fuse_block('default','test_vacuum2');" | $BENDSQL_CLIENT_CONNECT) + +stmt "insert into test_vacuum2 values(2);" + +IFS=$'\n' read -d '' -r -a segments <<< "$SEGMENTS" +IFS=$'\n' read -d '' -r -a blocks <<< "$BLOCKS" +blooms=() +for block in "${blocks[@]}"; do + bloom=$(echo "$block" | sed -E 's|(1/[0-9]+)/_b/g([0-9a-f]{32})_v2\.parquet|\1/_i_b_v2/\2_v4.parquet|') + blooms+=("$bloom") +done + +stmt "set data_retention_time_in_days = 2;truncate table test_vacuum2;" + +SNAPSHOTS=$(echo "select snapshot_location from fuse_snapshot('default','test_vacuum2');" | $BENDSQL_CLIENT_CONNECT) +IFS=$'\n' read -d '' -r -a snapshots <<< "$SNAPSHOTS" +to_be_vacuumed=("${snapshots[@]}" "${segments[@]}" "${blocks[@]}" "${blooms[@]}") + +# gc root +stmt "insert into test_vacuum2 values(3);" + + +# should have 4 snapshots +query "select count(*) from fuse_snapshot('default','test_vacuum2')" + +RESULTS=$(echo "set data_retention_time_in_days = 0;select * from fuse_vacuum2('default','test_vacuum2');" | $BENDSQL_CLIENT_CONNECT) +IFS=$'\n' read -d '' -r -a results <<< "$RESULTS" + +# verify the vacuum result +sorted_results=($(printf "%s\n" "${results[@]}" | sort)) +sorted_to_be_vacuumed=($(printf "%s\n" "${to_be_vacuumed[@]}" | sort)) + +if [ "$(printf "%s" "${sorted_results[@]}")" != "$(printf "%s" "${sorted_to_be_vacuumed[@]}")" ]; then + echo "Vacuum failed" + echo "Results array: ${sorted_results[@]}" + echo "To be vacuumed array: ${sorted_to_be_vacuumed[@]}" + exit 1 +fi + +# remain two snapshots +query "select count(*) from fuse_snapshot('default','test_vacuum2')" + +# verify the data +query "select * from test_vacuum2;" + +# restore default value +stmt "set data_retention_time_in_days = 1;" \ No newline at end of file diff --git a/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.result b/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.result new file mode 100644 index 000000000000..0163305ee37c --- /dev/null +++ b/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.result @@ -0,0 +1,13 @@ +>>>> insert into test_vacuum2_respect_time_travel values(2); +>>>> set data_retention_time_in_days = 2;truncate table test_vacuum2_respect_time_travel; +>>>> insert into test_vacuum2_respect_time_travel values(3); +>>>> select count(*) from fuse_snapshot('default','test_vacuum2_respect_time_travel') +4 +<<<< +>>>> select count(*) from fuse_snapshot('default','test_vacuum2_respect_time_travel') +1 +<<<< +>>>> select * from test_vacuum2_respect_time_travel; +3 +<<<< +>>>> set data_retention_time_in_days = 1; \ No newline at end of file diff --git a/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.sh b/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.sh new file mode 100755 index 000000000000..bc18b7c5cc6b --- /dev/null +++ b/tests/suites/9_faked_time/00_dummy_cases/00_0003_vacuum2_respect_time_travel.sh @@ -0,0 +1,53 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. "$CURDIR"/../../../shell_env.sh + +SEGMENTS=$(echo "select file_location from fuse_segment('default','test_vacuum2_respect_time_travel');" | $BENDSQL_CLIENT_CONNECT) +BLOCKS=$(echo "select block_location from fuse_block('default','test_vacuum2_respect_time_travel');" | $BENDSQL_CLIENT_CONNECT) + +stmt "insert into test_vacuum2_respect_time_travel values(2);" + +IFS=$'\n' read -d '' -r -a segments <<< "$SEGMENTS" +IFS=$'\n' read -d '' -r -a blocks <<< "$BLOCKS" +blooms=() +for block in "${blocks[@]}"; do + bloom=$(echo "$block" | sed -E 's|(1/[0-9]+)/_b/g([0-9a-f]{32})_v2\.parquet|\1/_i_b_v2/\2_v4.parquet|') + blooms+=("$bloom") +done + +stmt "set data_retention_time_in_days = 2;truncate table test_vacuum2_respect_time_travel;" + +SNAPSHOTS=$(echo "select snapshot_location from fuse_snapshot('default','test_vacuum2_respect_time_travel');" | $BENDSQL_CLIENT_CONNECT) +IFS=$'\n' read -d '' -r -a snapshots <<< "$SNAPSHOTS" +to_be_vacuumed=("${snapshots[@]}" "${segments[@]}" "${blocks[@]}" "${blooms[@]}") + +# gc root +stmt "insert into test_vacuum2_respect_time_travel values(3);" + + +# should have 4 snapshots +query "select count(*) from fuse_snapshot('default','test_vacuum2_respect_time_travel')" + +RESULTS=$(echo "set data_retention_time_in_days = 0;select * from fuse_vacuum2('default','test_vacuum2_respect_time_travel',true);" | $BENDSQL_CLIENT_CONNECT) +IFS=$'\n' read -d '' -r -a results <<< "$RESULTS" + +# verify the vacuum result +sorted_results=($(printf "%s\n" "${results[@]}" | sort)) +sorted_to_be_vacuumed=($(printf "%s\n" "${to_be_vacuumed[@]}" | sort)) + +if [ "$(printf "%s" "${sorted_results[@]}")" != "$(printf "%s" "${sorted_to_be_vacuumed[@]}")" ]; then + echo "Vacuum failed" + echo "Results array: ${sorted_results[@]}" + echo "To be vacuumed array: ${sorted_to_be_vacuumed[@]}" + exit 1 +fi + +# remain two snapshots +query "select count(*) from fuse_snapshot('default','test_vacuum2_respect_time_travel')" + +# verify the data +query "select * from test_vacuum2_respect_time_travel;" + +# restore default value +stmt "set data_retention_time_in_days = 1;" \ No newline at end of file