From af708798588b9b320515c857d649390f88c4ced3 Mon Sep 17 00:00:00 2001 From: Tao Wu Date: Mon, 17 Jun 2024 15:10:25 +0800 Subject: [PATCH 01/17] feat(catalog): support attoptions in pg_attribute (#17271) --- e2e_test/batch/catalog/pg_attribute.slt.part | 5 +++++ .../src/catalog/system_catalog/pg_catalog/pg_attribute.rs | 2 ++ 2 files changed, 7 insertions(+) diff --git a/e2e_test/batch/catalog/pg_attribute.slt.part b/e2e_test/batch/catalog/pg_attribute.slt.part index 4c8570ce2f47..2615466685a3 100644 --- a/e2e_test/batch/catalog/pg_attribute.slt.part +++ b/e2e_test/batch/catalog/pg_attribute.slt.part @@ -42,5 +42,10 @@ tmp_idx id1 {2,1,3,5} tmp_idx id2 {2,1,3,5} tmp_idx id3 {2,1,3,5} +query T +select attoptions from pg_catalog.pg_attribute LIMIT 1; +---- +NULL + statement ok drop table tmp; diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs index a0bf9190df2c..89197e8f95da 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_attribute.rs @@ -39,6 +39,7 @@ use risingwave_frontend_macro::system_catalog; ELSE ''::varchar END AS attgenerated, -1 AS atttypmod, + NULL::text[] AS attoptions, 0 AS attcollation FROM rw_catalog.rw_columns c WHERE c.is_hidden = false" @@ -56,5 +57,6 @@ struct PgAttribute { attidentity: String, attgenerated: String, atttypmod: i32, + attoptions: Vec, attcollation: i32, } From dfbce1bb04301b82e64c39c49cd9457b1e17b6a1 Mon Sep 17 00:00:00 2001 From: August Date: Mon, 17 Jun 2024 15:36:06 +0800 Subject: [PATCH 02/17] feat: expose PK info in index system catalog (#17262) --- e2e_test/batch/catalog/pg_indexes.slt.part | 3 +- .../system_catalog/pg_catalog/pg_index.rs | 29 ++++++++++++++++++- .../system_catalog/pg_catalog/pg_indexes.rs | 13 +++++++++ 3 files changed, 43 insertions(+), 2 deletions(-) diff --git a/e2e_test/batch/catalog/pg_indexes.slt.part b/e2e_test/batch/catalog/pg_indexes.slt.part index 634739406e91..c8e8ef626703 100644 --- a/e2e_test/batch/catalog/pg_indexes.slt.part +++ b/e2e_test/batch/catalog/pg_indexes.slt.part @@ -1,5 +1,5 @@ statement ok -create table t(a int, b int); +create table t(a int primary key, b int); statement ok create index idx1 on t(a); @@ -12,6 +12,7 @@ select schemaname, tablename, indexname, tablespace, indexdef from pg_catalog.pg ---- public t idx1 NULL CREATE INDEX idx1 ON t(a) public t idx2 NULL CREATE INDEX idx2 ON t(b) +public t t_pkey NULL (empty) statement ok drop table t; diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs index b91bd9b698cb..f370b8615ad4 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_index.rs @@ -39,7 +39,34 @@ use risingwave_frontend_macro::system_catalog; true AS indisready, true AS indislive, false AS indisreplident - FROM rw_catalog.rw_indexes" + FROM rw_catalog.rw_indexes + UNION ALL + SELECT c.relation_id AS indexrelid, + c.relation_id AS indrelid, + COUNT(*)::smallint AS indnatts, + COUNT(*)::smallint AS indnkeyatts, + true AS indisunique, + ARRAY_AGG(c.position)::smallint[] AS indkey, + ARRAY[]::smallint[] as indoption, + NULL AS indexprs, + NULL AS indpred, + TRUE AS indisprimary, + ARRAY[]::int[] AS indclass, + false AS indisexclusion, + true AS indimmediate, + false AS indisclustered, + true AS indisvalid, + false AS indcheckxmin, + true AS indisready, + true AS indislive, + false AS indisreplident + FROM rw_catalog.rw_columns c + WHERE c.is_primary_key = true AND c.is_hidden = false + AND c.relation_id IN ( + SELECT id + FROM rw_catalog.rw_tables + ) + GROUP BY c.relation_id" )] #[derive(Fields)] struct PgIndex { diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs index e35aba9567ae..a602e7180477 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_indexes.rs @@ -28,6 +28,19 @@ use risingwave_frontend_macro::system_catalog; FROM rw_catalog.rw_indexes i JOIN rw_catalog.rw_tables t ON i.primary_table_id = t.id JOIN rw_catalog.rw_schemas s ON i.schema_id = s.id + UNION ALL + SELECT s.name AS schemaname, + t.name AS tablename, + concat(t.name, '_pkey') AS indexname, + NULL AS tablespace, + '' AS indexdef + FROM rw_catalog.rw_tables t + JOIN rw_catalog.rw_schemas s ON t.schema_id = s.id + WHERE t.id IN ( + SELECT DISTINCT relation_id + FROM rw_catalog.rw_columns + WHERE is_primary_key = true AND is_hidden = false + ) " )] #[derive(Fields)] From adf465ab5e62201be5ca50fa17d8b22db3fb139a Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 17 Jun 2024 15:39:38 +0800 Subject: [PATCH 03/17] fix(meta): purge hummock state table after pre apply drop command (#17274) --- src/meta/src/barrier/recovery.rs | 52 ++++++++++-------------- src/meta/src/controller/streaming_job.rs | 19 ++------- src/meta/src/rpc/ddl_controller_v2.rs | 2 +- 3 files changed, 27 insertions(+), 46 deletions(-) diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index f17b4e163901..7200426d7830 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -313,30 +313,22 @@ impl GlobalBarrierManagerContext { let (dropped_actors, cancelled) = scheduled_barriers.pre_apply_drop_cancel_scheduled(); let applied = !dropped_actors.is_empty() || !cancelled.is_empty(); if !cancelled.is_empty() { - let unregister_table_ids = match &self.metadata_manager { + match &self.metadata_manager { MetadataManager::V1(mgr) => { mgr.fragment_manager .drop_table_fragments_vec(&cancelled) - .await? + .await?; } MetadataManager::V2(mgr) => { - let mut unregister_table_ids = Vec::new(); for job_id in cancelled { - let (_, table_ids_to_unregister) = mgr - .catalog_controller + mgr.catalog_controller .try_abort_creating_streaming_job(job_id.table_id as _, true) .await?; - unregister_table_ids.extend(table_ids_to_unregister); } - unregister_table_ids - .into_iter() - .map(|table_id| table_id as u32) - .collect() } }; - self.hummock_manager - .unregister_table_ids(&unregister_table_ids) - .await?; + // no need to unregister state table id from hummock manager here, because it's expected that + // we call `purge_state_table_from_hummock` anyway after the current method returns. } Ok(applied) } @@ -377,11 +369,6 @@ impl GlobalBarrierManager { .await .context("clean dirty streaming jobs")?; - self.context - .purge_state_table_from_hummock() - .await - .context("purge state table from hummock")?; - // Mview progress needs to be recovered. tracing::info!("recovering mview progress"); self.context @@ -436,18 +423,6 @@ impl GlobalBarrierManager { })? }; - let mut control_stream_manager = - ControlStreamManager::new(self.context.clone()); - - control_stream_manager - .reset(prev_epoch.value().0, active_streaming_nodes.current()) - .await - .inspect_err(|err| { - warn!(error = %err.as_report(), "reset compute nodes failed"); - })?; - - self.context.sink_manager.reset().await; - if self .context .pre_apply_drop_cancel(&self.scheduled_barriers) @@ -462,6 +437,23 @@ impl GlobalBarrierManager { })? } + self.context + .purge_state_table_from_hummock() + .await + .context("purge state table from hummock")?; + + let mut control_stream_manager = + ControlStreamManager::new(self.context.clone()); + + control_stream_manager + .reset(prev_epoch.value().0, active_streaming_nodes.current()) + .await + .inspect_err(|err| { + warn!(error = %err.as_report(), "reset compute nodes failed"); + })?; + + self.context.sink_manager.reset().await; + // update and build all actors. self.context.update_actors(&info).await.inspect_err(|err| { warn!(error = %err.as_report(), "update actors failed"); diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 3066ce223785..ad852cf8ac51 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -411,7 +411,7 @@ impl CatalogController { &self, job_id: ObjectId, is_cancelled: bool, - ) -> MetaResult<(bool, Vec)> { + ) -> MetaResult { let inner = self.inner.write().await; let txn = inner.db.begin().await?; @@ -421,7 +421,7 @@ impl CatalogController { id = job_id, "streaming job not found when aborting creating, might be cleaned by recovery" ); - return Ok((true, Vec::new())); + return Ok(true); } if !is_cancelled { @@ -436,7 +436,7 @@ impl CatalogController { id = job_id, "streaming job is created in background and still in creating status" ); - return Ok((false, Vec::new())); + return Ok(false); } } } @@ -449,13 +449,6 @@ impl CatalogController { .all(&txn) .await?; - let mv_table_id: Option = Table::find_by_id(job_id) - .select_only() - .column(table::Column::TableId) - .into_tuple() - .one(&txn) - .await?; - let associated_source_id: Option = Table::find_by_id(job_id) .select_only() .column(table::Column::OptionalAssociatedSourceId) @@ -476,11 +469,7 @@ impl CatalogController { } txn.commit().await?; - let mut state_table_ids = internal_table_ids; - - state_table_ids.extend(mv_table_id.into_iter()); - - Ok((true, state_table_ids)) + Ok(true) } pub async fn post_collect_table_fragments( diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs index 13ecfca13c78..37448e50a07d 100644 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ b/src/meta/src/rpc/ddl_controller_v2.rs @@ -104,7 +104,7 @@ impl DdlController { self.env.event_log_manager_ref().add_event_logs(vec![ risingwave_pb::meta::event_log::Event::CreateStreamJobFail(event), ]); - let (aborted, _) = mgr + let aborted = mgr .catalog_controller .try_abort_creating_streaming_job(job_id as _, false) .await?; From 9274ebc630d00d7c5909a58a12221d19647dd2b2 Mon Sep 17 00:00:00 2001 From: August Date: Mon, 17 Jun 2024 15:45:45 +0800 Subject: [PATCH 04/17] chore: update migration doc, add DATABASE_URL env example (#17276) --- src/meta/model_v2/migration/README.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/meta/model_v2/migration/README.md b/src/meta/model_v2/migration/README.md index 527d4ba53807..d68a624ced8a 100644 --- a/src/meta/model_v2/migration/README.md +++ b/src/meta/model_v2/migration/README.md @@ -9,11 +9,11 @@ > **DO NOT** modify already published migration files. ## How to run the migrator CLI -- Generate a new migration file +- Generate a new migration file, a database endpoint is required but not used. ```sh - cargo run -- generate MIGRATION_NAME + export DATABASE_URL=sqlite::memory:; cargo run -- generate MIGRATION_NAME ``` -- Apply all pending migrations for test purposes, `DATABASE_URL` required. +- Apply all pending migrations for test purposes, change `DATABASE_URL` to the actual database endpoint. ```sh cargo run ``` From e34c83b4c63ebefd7a4b292fb02e952c2db9909f Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Mon, 17 Jun 2024 16:40:05 +0800 Subject: [PATCH 05/17] feat(expr): add function `pg_index_column_has_property` (#17275) Signed-off-by: Runji Wang Co-authored-by: August --- proto/catalog.proto | 9 ++ proto/expr.proto | 1 + src/frontend/src/binder/expr/function.rs | 1 + src/frontend/src/catalog/index_catalog.rs | 13 +- src/frontend/src/expr/function_impl/mod.rs | 1 + .../pg_index_column_has_property.rs | 128 ++++++++++++++++++ src/frontend/src/expr/pure.rs | 1 + src/frontend/src/handler/create_index.rs | 10 +- .../src/optimizer/plan_expr_visitor/strong.rs | 1 + src/meta/model_v2/migration/src/lib.rs | 2 + ...20240617_070131_index_column_properties.rs | 35 +++++ src/meta/model_v2/src/index.rs | 4 +- src/meta/model_v2/src/lib.rs | 5 + src/meta/src/controller/mod.rs | 1 + src/prost/build.rs | 1 + 15 files changed, 210 insertions(+), 3 deletions(-) create mode 100644 src/frontend/src/expr/function_impl/pg_index_column_has_property.rs create mode 100644 src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs diff --git a/proto/catalog.proto b/proto/catalog.proto index 7dfefa003217..5b4f5ae40ff4 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -241,6 +241,7 @@ message Index { // Only `InputRef` type index is supported Now. // The index of `InputRef` is the column index of the primary table. repeated expr.ExprNode index_item = 8; + repeated IndexColumnProperties index_column_properties = 16; reserved 9; // Deprecated repeated int32 original_columns = 9; optional uint64 initialized_at_epoch = 10; @@ -255,6 +256,14 @@ message Index { optional string created_at_cluster_version = 15; } +// https://www.postgresql.org/docs/current/functions-info.html#FUNCTIONS-INFO-INDEX-COLUMN-PROPS +message IndexColumnProperties { + // Whether the column sort in ascending(false) or descending(true) order on a forward scan. + bool is_desc = 1; + // Does the column sort with nulls first on a forward scan? + bool nulls_first = 2; +} + message Function { uint32 id = 1; uint32 schema_id = 2; diff --git a/proto/expr.proto b/proto/expr.proto index 602c712975ec..ada2159bb80a 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -302,6 +302,7 @@ message ExprNode { PG_INDEXES_SIZE = 2404; PG_RELATION_SIZE = 2405; PG_GET_SERIAL_SEQUENCE = 2406; + PG_INDEX_COLUMN_HAS_PROPERTY = 2410; HAS_TABLE_PRIVILEGE = 2407; HAS_ANY_COLUMN_PRIVILEGE = 2408; HAS_SCHEMA_PRIVILEGE = 2409; diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 2a4812c2e1d1..09c05a29695d 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1256,6 +1256,7 @@ impl Binder { ("pg_get_userbyid", raw_call(ExprType::PgGetUserbyid)), ("pg_get_indexdef", raw_call(ExprType::PgGetIndexdef)), ("pg_get_viewdef", raw_call(ExprType::PgGetViewdef)), + ("pg_index_column_has_property", raw_call(ExprType::PgIndexColumnHasProperty)), ("pg_relation_size", raw(|_binder, mut inputs|{ if inputs.is_empty() { return Err(ErrorCode::ExprError( diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index 7e13c365407e..098eca22af9e 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -21,7 +21,7 @@ use itertools::Itertools; use risingwave_common::catalog::{Field, IndexId, Schema}; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; -use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus}; +use risingwave_pb::catalog::{PbIndex, PbIndexColumnProperties, PbStreamJobStatus}; use crate::catalog::{DatabaseId, OwnedByUserCatalog, SchemaId, TableCatalog}; use crate::expr::{Expr, ExprDisplay, ExprImpl, FunctionCall}; @@ -40,6 +40,10 @@ pub struct IndexCatalog { /// The input args of `FuncCall` is also the column index of the primary table. pub index_item: Vec, + /// The properties of the index columns. + /// + pub index_column_properties: Vec, + pub index_table: Arc, pub primary_table: Arc, @@ -112,6 +116,7 @@ impl IndexCatalog { id: index_prost.id.into(), name: index_prost.name.clone(), index_item, + index_column_properties: index_prost.index_column_properties.clone(), index_table: Arc::new(index_table.clone()), primary_table: Arc::new(primary_table.clone()), primary_to_secondary_mapping, @@ -179,6 +184,7 @@ impl IndexCatalog { .iter() .map(|expr| expr.to_expr_proto()) .collect_vec(), + index_column_properties: self.index_column_properties.clone(), index_columns_len: self.index_columns_len, initialized_at_epoch: self.initialized_at_epoch.map(|e| e.0), created_at_epoch: self.created_at_epoch.map(|e| e.0), @@ -188,6 +194,11 @@ impl IndexCatalog { } } + /// Get the column properties of the index column. + pub fn get_column_properties(&self, column_idx: usize) -> Option { + self.index_column_properties.get(column_idx).cloned() + } + pub fn get_column_def(&self, column_idx: usize) -> Option { if let Some(col) = self.index_table.columns.get(column_idx) { if col.is_hidden { diff --git a/src/frontend/src/expr/function_impl/mod.rs b/src/frontend/src/expr/function_impl/mod.rs index 3a70ebf5db47..a0cff36840b4 100644 --- a/src/frontend/src/expr/function_impl/mod.rs +++ b/src/frontend/src/expr/function_impl/mod.rs @@ -19,5 +19,6 @@ mod has_privilege; mod pg_get_indexdef; mod pg_get_userbyid; mod pg_get_viewdef; +mod pg_index_column_has_property; mod pg_indexes_size; mod pg_relation_size; diff --git a/src/frontend/src/expr/function_impl/pg_index_column_has_property.rs b/src/frontend/src/expr/function_impl/pg_index_column_has_property.rs new file mode 100644 index 000000000000..fdabf5aefc77 --- /dev/null +++ b/src/frontend/src/expr/function_impl/pg_index_column_has_property.rs @@ -0,0 +1,128 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use risingwave_expr::{capture_context, function, Result}; + +use super::context::{CATALOG_READER, DB_NAME}; +use crate::catalog::CatalogReader; + +/// Tests whether an index column has the named property. +/// +/// `index` is the OID of the index. +/// `column` is the column number (1-based) within the index. +/// +/// NULL is returned if the property name is not known or does not apply to the particular object, +/// or if the OID or column number does not identify a valid object. +/// +/// # Supported Properties +/// +/// - `asc`: Does the column sort in ascending order on a forward scan? +/// - `desc`: Does the column sort in descending order on a forward scan? +/// - `nulls_first`: Does the column sort with nulls first on a forward scan? +/// - `nulls_last`: Does the column sort with nulls last on a forward scan? +/// +/// # Examples +/// +/// ```slt +/// statement ok +/// create table t(a int, b int); +/// +/// statement ok +/// create index i on t (a asc, b desc); +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 1, 'asc'); +/// ---- +/// t +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 1, 'DESC'); +/// ---- +/// f +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 1, 'nulls_FIRST'); +/// ---- +/// f +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 1, 'nulls_last'); +/// ---- +/// t +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 2, 'asc'); +/// ---- +/// f +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 2, 'desc'); +/// ---- +/// t +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 2, 'nulls_first'); +/// ---- +/// t +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 2, 'nulls_last'); +/// ---- +/// f +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 1, 'gg'); -- invalid property +/// ---- +/// NULL +/// +/// query B +/// select pg_index_column_has_property('i'::regclass, 0, 'asc'); -- column 0 does not exist +/// ---- +/// NULL +/// +/// statement ok +/// drop index i; +/// +/// statement ok +/// drop table t; +/// ``` +#[function("pg_index_column_has_property(int4, int4, varchar) -> boolean")] +fn pg_index_column_has_property(index: i32, column: i32, property: &str) -> Result> { + pg_index_column_has_property_impl_captured(index, column, property) +} + +#[capture_context(CATALOG_READER, DB_NAME)] +fn pg_index_column_has_property_impl( + catalog: &CatalogReader, + db_name: &str, + index_id: i32, + column_idx: i32, + property: &str, + // `Result` is not necessary for this function, but it's required by `capture_context`. +) -> Result> { + let catalog_reader = catalog.read_guard(); + let Ok(index) = catalog_reader.get_index_by_id(db_name, index_id as u32) else { + return Ok(None); + }; + let Some(properties) = index.get_column_properties((column_idx - 1) as usize) else { + return Ok(None); + }; + Ok(match property.to_lowercase().as_str() { + "asc" => Some(!properties.is_desc), + "desc" => Some(properties.is_desc), + "nulls_first" => Some(properties.nulls_first), + "nulls_last" => Some(!properties.nulls_first), + _ => None, + }) +} diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index d03b7507fdfb..b404fb3408df 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -270,6 +270,7 @@ impl ExprVisitor for ImpureAnalyzer { | Type::PgIndexesSize | Type::PgRelationSize | Type::PgGetSerialSequence + | Type::PgIndexColumnHasProperty | Type::HasTablePrivilege | Type::HasAnyColumnPrivilege | Type::HasSchemaPrivilege diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 6e9793646c7e..a0586ab20e4d 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -24,7 +24,7 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::acl::AclMode; use risingwave_common::catalog::{IndexId, TableDesc, TableId}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; -use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus, PbTable}; +use risingwave_pb::catalog::{PbIndex, PbIndexColumnProperties, PbStreamJobStatus, PbTable}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_pb::user::grant_privilege::Object; use risingwave_sqlparser::ast; @@ -225,6 +225,13 @@ pub(crate) fn gen_create_index_plan( index_table_prost.dependent_relations = vec![table.id.table_id]; let index_columns_len = index_columns_ordered_expr.len() as u32; + let index_column_properties = index_columns_ordered_expr + .iter() + .map(|(_, order)| PbIndexColumnProperties { + is_desc: order.is_descending(), + nulls_first: order.nulls_are_first(), + }) + .collect(); let index_item = build_index_item( index_table.table_desc().into(), table.name(), @@ -241,6 +248,7 @@ pub(crate) fn gen_create_index_plan( index_table_id: TableId::placeholder().table_id, primary_table_id: table.id.table_id, index_item, + index_column_properties, index_columns_len, initialized_at_epoch: None, created_at_epoch: None, diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index fefdd1e4547f..55d8e3a18adf 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -303,6 +303,7 @@ impl Strong { | ExprType::PgIndexesSize | ExprType::PgRelationSize | ExprType::PgGetSerialSequence + | ExprType::PgIndexColumnHasProperty | ExprType::IcebergTransform | ExprType::HasTablePrivilege | ExprType::HasAnyColumnPrivilege diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 66f136b6159d..627e565a47f3 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -11,6 +11,7 @@ mod m20240417_062305_subscription_internal_table_name; mod m20240418_142249_function_runtime; mod m20240506_112555_subscription_partial_ckpt; mod m20240525_090457_secret; +mod m20240617_070131_index_column_properties; pub struct Migrator; @@ -27,6 +28,7 @@ impl MigratorTrait for Migrator { Box::new(m20240418_142249_function_runtime::Migration), Box::new(m20240506_112555_subscription_partial_ckpt::Migration), Box::new(m20240525_090457_secret::Migration), + Box::new(m20240617_070131_index_column_properties::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs b/src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs new file mode 100644 index 000000000000..daff755da999 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240617_070131_index_column_properties.rs @@ -0,0 +1,35 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(Index::Table) + .add_column(ColumnDef::new(Index::IndexColumnProperties).binary()) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(Index::Table) + .drop_column(Index::IndexColumnProperties) + .to_owned(), + ) + .await + } +} + +#[derive(DeriveIden)] +enum Index { + Table, + IndexColumnProperties, +} diff --git a/src/meta/model_v2/src/index.rs b/src/meta/model_v2/src/index.rs index ca2f39c0f179..a291e325b24e 100644 --- a/src/meta/model_v2/src/index.rs +++ b/src/meta/model_v2/src/index.rs @@ -17,7 +17,7 @@ use sea_orm::entity::prelude::*; use sea_orm::ActiveValue::Set; use serde::{Deserialize, Serialize}; -use crate::{ExprNodeArray, IndexId, TableId}; +use crate::{ExprNodeArray, IndexColumnPropertiesArray, IndexId, TableId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] #[sea_orm(table_name = "index")] @@ -28,6 +28,7 @@ pub struct Model { pub index_table_id: TableId, pub primary_table_id: TableId, pub index_items: ExprNodeArray, + pub index_column_properties: IndexColumnPropertiesArray, pub index_columns_len: i32, } @@ -76,6 +77,7 @@ impl From for ActiveModel { primary_table_id: Set(pb_index.primary_table_id as _), index_items: Set(pb_index.index_item.into()), index_columns_len: Set(pb_index.index_columns_len as _), + index_column_properties: Set(pb_index.index_column_properties.into()), } } } diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 647ce99ec1e7..11c5209bdc56 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -324,6 +324,11 @@ derive_array_from_blob!( risingwave_pb::common::PbColumnOrder, PbColumnOrderArray ); +derive_array_from_blob!( + IndexColumnPropertiesArray, + risingwave_pb::catalog::PbIndexColumnProperties, + PbIndexColumnPropertiesArray +); derive_from_blob!(SinkFormatDesc, risingwave_pb::catalog::PbSinkFormatDesc); derive_from_blob!(Cardinality, risingwave_pb::plan_common::PbCardinality); derive_from_blob!(TableVersion, risingwave_pb::catalog::table::PbTableVersion); diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index ac61ceb67b77..66491d515dd4 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -274,6 +274,7 @@ impl From> for PbIndex { index_table_id: value.0.index_table_id as _, primary_table_id: value.0.primary_table_id as _, index_item: value.0.index_items.to_protobuf(), + index_column_properties: value.0.index_column_properties.to_protobuf(), index_columns_len: value.0.index_columns_len as _, initialized_at_epoch: Some( Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, diff --git a/src/prost/build.rs b/src/prost/build.rs index 6cbfa82225e6..961dbe196944 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -111,6 +111,7 @@ fn main() -> Result<(), Box> { // The requirement is from Source node -> SourceCatalog -> WatermarkDesc -> expr .type_attribute("catalog.WatermarkDesc", "#[derive(Eq, Hash)]") .type_attribute("catalog.StreamSourceInfo", "#[derive(Eq, Hash)]") + .type_attribute("catalog.IndexColumnProperties", "#[derive(Eq, Hash)]") .type_attribute("expr.ExprNode", "#[derive(Eq, Hash)]") .type_attribute("data.DataType", "#[derive(Eq, Hash)]") .type_attribute("expr.ExprNode.rex_node", "#[derive(Eq, Hash)]") From 1b8a56373495b9602f2d892ef85e3d2825929f34 Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Mon, 17 Jun 2024 17:31:23 +0800 Subject: [PATCH 06/17] feat(object store): introduce object prefix for opendal object store (#16542) --- e2e_test/batch/catalog/pg_settings.slt.part | 1 + proto/java_binding.proto | 1 + proto/meta.proto | 1 + src/common/src/system_param/mod.rs | 3 ++ src/common/src/system_param/reader.rs | 8 +++ src/compute/src/server.rs | 2 +- src/config/docs.md | 1 + src/ctl/src/cmd_impl/bench.rs | 8 ++- src/ctl/src/cmd_impl/hummock/list_kv.rs | 6 ++- src/ctl/src/cmd_impl/hummock/sst_dump.rs | 16 ++++-- .../src/cmd_impl/hummock/validate_version.rs | 8 ++- src/ctl/src/cmd_impl/table/scan.rs | 24 +++++++-- src/ctl/src/common/hummock_service.rs | 16 +++++- src/ctl/src/lib.rs | 53 ++++++++++++++++--- src/java_binding/src/hummock_iterator.rs | 1 + src/meta/model_v2/migration/src/lib.rs | 2 +- src/meta/node/src/server.rs | 8 --- src/meta/src/backup_restore/restore.rs | 1 + src/meta/src/manager/env.rs | 44 +++++++++++++-- src/object_store/src/object/mem.rs | 2 +- src/object_store/src/object/mod.rs | 10 ++-- .../opendal_engine/opendal_object_store.rs | 18 ++++--- src/object_store/src/object/prefix.rs | 20 +++++++ src/object_store/src/object/s3.rs | 3 +- src/object_store/src/object/sim/mod.rs | 2 +- src/storage/benches/bench_compactor.rs | 1 + src/storage/benches/bench_multi_builder.rs | 1 + src/storage/compactor/src/server.rs | 1 + .../hummock_test/src/bin/replay/main.rs | 4 ++ .../src/hummock/iterator/test_utils.rs | 1 + src/storage/src/hummock/sstable_store.rs | 17 +++++- src/storage/src/store_impl.rs | 2 + .../src/compaction_test_runner.rs | 1 + .../src/delete_range_runner.rs | 1 + 34 files changed, 239 insertions(+), 49 deletions(-) diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 2a130de04c19..a3fd0edc6522 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -7,6 +7,7 @@ internal data_directory internal parallel_compact_size_mb internal sstable_size_mb internal state_store +internal use_new_object_prefix_strategy postmaster backup_storage_directory postmaster backup_storage_url postmaster barrier_interval_ms diff --git a/proto/java_binding.proto b/proto/java_binding.proto index 32ed2f5df199..72558438d176 100644 --- a/proto/java_binding.proto +++ b/proto/java_binding.proto @@ -34,4 +34,5 @@ message ReadPlan { catalog.Table table_catalog = 7; repeated uint32 vnode_ids = 8; + bool use_new_object_prefix_strategy = 9; } diff --git a/proto/meta.proto b/proto/meta.proto index 9ad18cb3df7d..4a67dd0455e5 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -602,6 +602,7 @@ message SystemParams { optional bool pause_on_next_bootstrap = 13; optional string wasm_storage_url = 14 [deprecated = true]; optional bool enable_tracing = 15; + optional bool use_new_object_prefix_strategy = 16; } message GetSystemParamsRequest {} diff --git a/src/common/src/system_param/mod.rs b/src/common/src/system_param/mod.rs index f71324cb4e55..c8382d35fee8 100644 --- a/src/common/src/system_param/mod.rs +++ b/src/common/src/system_param/mod.rs @@ -87,6 +87,7 @@ macro_rules! for_all_params { { max_concurrent_creating_streaming_jobs, u32, Some(1_u32), true, "Max number of concurrent creating streaming jobs.", }, { pause_on_next_bootstrap, bool, Some(false), true, "Whether to pause all data sources on next bootstrap.", }, { enable_tracing, bool, Some(false), true, "Whether to enable distributed tracing.", }, + { use_new_object_prefix_strategy, bool, None, false, "Whether to split object prefix.", }, } }; } @@ -376,6 +377,7 @@ macro_rules! impl_system_params_for_test { ret.state_store = Some("hummock+memory".to_string()); ret.backup_storage_url = Some("memory".into()); ret.backup_storage_directory = Some("backup".into()); + ret.use_new_object_prefix_strategy = Some(false); ret } }; @@ -441,6 +443,7 @@ mod tests { (MAX_CONCURRENT_CREATING_STREAMING_JOBS_KEY, "1"), (PAUSE_ON_NEXT_BOOTSTRAP_KEY, "false"), (ENABLE_TRACING_KEY, "true"), + (USE_NEW_OBJECT_PREFIX_STRATEGY_KEY, "false"), ("a_deprecated_param", "foo"), ]; diff --git a/src/common/src/system_param/reader.rs b/src/common/src/system_param/reader.rs index 3374e7212023..9a2c6e49534a 100644 --- a/src/common/src/system_param/reader.rs +++ b/src/common/src/system_param/reader.rs @@ -137,6 +137,14 @@ where self.inner().data_directory.as_ref().unwrap() } + fn use_new_object_prefix_strategy(&self) -> bool { + *self + .inner() + .use_new_object_prefix_strategy + .as_ref() + .unwrap() + } + fn backup_storage_url(&self) -> &str { self.inner().backup_storage_url.as_ref().unwrap() } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 61f62fbc7e00..a28631a29bea 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -91,7 +91,6 @@ pub async fn compute_node_serve( ) -> (Vec>, Sender<()>) { // Load the configuration. let config = load_config(&opts.config_path, &opts); - info!("Starting compute node",); info!("> config: {:?}", config); info!( @@ -211,6 +210,7 @@ pub async fn compute_node_serve( storage_metrics.clone(), compactor_metrics.clone(), await_tree_config.clone(), + system_params.use_new_object_prefix_strategy(), ) .await .unwrap(); diff --git a/src/config/docs.md b/src/config/docs.md index ab3355926016..59c8961a15be 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -167,3 +167,4 @@ This page is automatically generated by `./risedev generate-example-config` | pause_on_next_bootstrap | Whether to pause all data sources on next bootstrap. | false | | sstable_size_mb | Target size of the Sstable. | 256 | | state_store | URL for the state store | | +| use_new_object_prefix_strategy | Whether to split object prefix. | | diff --git a/src/ctl/src/cmd_impl/bench.rs b/src/ctl/src/cmd_impl/bench.rs index d3c0cde6d20e..dce4a21115d6 100644 --- a/src/ctl/src/cmd_impl/bench.rs +++ b/src/ctl/src/cmd_impl/bench.rs @@ -42,6 +42,8 @@ pub enum BenchCommands { #[clap(long, default_value_t = 1)] threads: usize, data_dir: Option, + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, } @@ -86,9 +88,13 @@ pub async fn do_bench(context: &CtlContext, cmd: BenchCommands) -> Result<()> { mv_name, threads, data_dir, + use_new_object_prefix_strategy, } => { let (hummock, metrics) = context - .hummock_store_with_metrics(HummockServiceOpts::from_env(data_dir)?) + .hummock_store_with_metrics(HummockServiceOpts::from_env( + data_dir, + use_new_object_prefix_strategy, + )?) .await?; let table = get_table_catalog(meta.clone(), mv_name).await?; let mut handlers = vec![]; diff --git a/src/ctl/src/cmd_impl/hummock/list_kv.rs b/src/ctl/src/cmd_impl/hummock/list_kv.rs index 2eb54362b413..f90712a02505 100644 --- a/src/ctl/src/cmd_impl/hummock/list_kv.rs +++ b/src/ctl/src/cmd_impl/hummock/list_kv.rs @@ -27,9 +27,13 @@ pub async fn list_kv( epoch: u64, table_id: u32, data_dir: Option, + use_new_object_prefix_strategy: bool, ) -> anyhow::Result<()> { let hummock = context - .hummock_store(HummockServiceOpts::from_env(data_dir)?) + .hummock_store(HummockServiceOpts::from_env( + data_dir, + use_new_object_prefix_strategy, + )?) .await?; if is_max_epoch(epoch) { tracing::info!("using MAX EPOCH as epoch"); diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs index 0fc65054b51e..ce69ab87f1b7 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -59,6 +59,8 @@ pub struct SstDumpArgs { print_table: bool, #[clap(short = 'd')] data_dir: Option, + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, } pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result<()> { @@ -72,7 +74,10 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result if args.print_level { // Level information is retrieved from meta service let hummock = context - .hummock_store(HummockServiceOpts::from_env(args.data_dir.clone())?) + .hummock_store(HummockServiceOpts::from_env( + args.data_dir.clone(), + args.use_new_object_prefix_strategy, + )?) .await?; let version = hummock.inner().get_pinned_version().version().clone(); let sstable_store = hummock.sstable_store(); @@ -108,8 +113,13 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result } } else { // Object information is retrieved from object store. Meta service is not required. - let hummock_service_opts = HummockServiceOpts::from_env(args.data_dir.clone())?; - let sstable_store = hummock_service_opts.create_sstable_store().await?; + let hummock_service_opts = HummockServiceOpts::from_env( + args.data_dir.clone(), + args.use_new_object_prefix_strategy, + )?; + let sstable_store = hummock_service_opts + .create_sstable_store(args.use_new_object_prefix_strategy) + .await?; if let Some(obj_id) = &args.object_id { let obj_store = sstable_store.store(); let obj_path = sstable_store.get_sst_data_path(*obj_id); diff --git a/src/ctl/src/cmd_impl/hummock/validate_version.rs b/src/ctl/src/cmd_impl/hummock/validate_version.rs index b2ae1c22f66c..e8f61a8c9835 100644 --- a/src/ctl/src/cmd_impl/hummock/validate_version.rs +++ b/src/ctl/src/cmd_impl/hummock/validate_version.rs @@ -65,6 +65,7 @@ pub async fn print_user_key_in_archive( archive_ids: Vec, data_dir: String, user_key: String, + use_new_object_prefix_strategy: bool, ) -> anyhow::Result<()> { let user_key_bytes = hex::decode(user_key.clone()).unwrap_or_else(|_| { panic!("cannot decode user key {} into raw bytes", user_key); @@ -72,7 +73,8 @@ pub async fn print_user_key_in_archive( let user_key = UserKey::decode(&user_key_bytes); println!("user key: {user_key:?}"); - let hummock_opts = HummockServiceOpts::from_env(Some(data_dir.clone()))?; + let hummock_opts = + HummockServiceOpts::from_env(Some(data_dir.clone()), use_new_object_prefix_strategy)?; let hummock = context.hummock_store(hummock_opts).await?; let sstable_store = hummock.sstable_store(); let archive_object_store = sstable_store.store(); @@ -178,8 +180,10 @@ pub async fn print_version_delta_in_archive( archive_ids: Vec, data_dir: String, sst_id: HummockSstableObjectId, + use_new_object_prefix_strategy: bool, ) -> anyhow::Result<()> { - let hummock_opts = HummockServiceOpts::from_env(Some(data_dir.clone()))?; + let hummock_opts = + HummockServiceOpts::from_env(Some(data_dir.clone()), use_new_object_prefix_strategy)?; let hummock = context.hummock_store(hummock_opts).await?; let sstable_store = hummock.sstable_store(); let archive_object_store = sstable_store.store(); diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index 8c21d975009f..0689e315f74c 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -86,19 +86,35 @@ pub fn make_storage_table( )) } -pub async fn scan(context: &CtlContext, mv_name: String, data_dir: Option) -> Result<()> { +pub async fn scan( + context: &CtlContext, + mv_name: String, + data_dir: Option, + use_new_object_prefix_strategy: bool, +) -> Result<()> { let meta_client = context.meta_client().await?; let hummock = context - .hummock_store(HummockServiceOpts::from_env(data_dir)?) + .hummock_store(HummockServiceOpts::from_env( + data_dir, + use_new_object_prefix_strategy, + )?) .await?; let table = get_table_catalog(meta_client, mv_name).await?; do_scan(table, hummock).await } -pub async fn scan_id(context: &CtlContext, table_id: u32, data_dir: Option) -> Result<()> { +pub async fn scan_id( + context: &CtlContext, + table_id: u32, + data_dir: Option, + use_new_object_prefix_strategy: bool, +) -> Result<()> { let meta_client = context.meta_client().await?; let hummock = context - .hummock_store(HummockServiceOpts::from_env(data_dir)?) + .hummock_store(HummockServiceOpts::from_env( + data_dir, + use_new_object_prefix_strategy, + )?) .await?; let table = get_table_catalog_by_id(meta_client, table_id).await?; do_scan(table, hummock).await diff --git a/src/ctl/src/common/hummock_service.rs b/src/ctl/src/common/hummock_service.rs index 59d272c3a27f..e885548d5a1e 100644 --- a/src/ctl/src/common/hummock_service.rs +++ b/src/ctl/src/common/hummock_service.rs @@ -36,6 +36,8 @@ pub struct HummockServiceOpts { pub hummock_url: String, pub data_dir: Option, + use_new_object_prefix_strategy: bool, + heartbeat_handle: Option>, heartbeat_shutdown_sender: Option>, } @@ -55,7 +57,10 @@ impl HummockServiceOpts { /// Currently, we will read these variables for meta: /// /// * `RW_HUMMOCK_URL`: hummock store address - pub fn from_env(data_dir: Option) -> Result { + pub fn from_env( + data_dir: Option, + use_new_object_prefix_strategy: bool, + ) -> Result { let hummock_url = match env::var("RW_HUMMOCK_URL") { Ok(url) => { if !url.starts_with("hummock+") { @@ -80,11 +85,13 @@ impl HummockServiceOpts { bail!(MESSAGE); } }; + Ok(Self { hummock_url, data_dir, heartbeat_handle: None, heartbeat_shutdown_sender: None, + use_new_object_prefix_strategy, }) } @@ -142,6 +149,7 @@ impl HummockServiceOpts { metrics.storage_metrics.clone(), metrics.compactor_metrics.clone(), None, + self.use_new_object_prefix_strategy, ) .await?; @@ -157,7 +165,10 @@ impl HummockServiceOpts { } } - pub async fn create_sstable_store(&self) -> Result> { + pub async fn create_sstable_store( + &self, + use_new_object_prefix_strategy: bool, + ) -> Result> { let object_store = build_remote_object_store( self.hummock_url.strip_prefix("hummock+").unwrap(), Arc::new(ObjectStoreMetrics::unused()), @@ -190,6 +201,7 @@ impl HummockServiceOpts { state_store_metrics: Arc::new(global_hummock_state_store_metrics( MetricLevel::Disabled, )), + use_new_object_prefix_strategy, meta_cache, block_cache, }))) diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 1f50250276d6..002817993d85 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -193,6 +193,9 @@ enum HummockCommands { // data directory for hummock state store. None: use default data_dir: Option, + + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, SstDump(SstDumpArgs), /// trigger a targeted compaction through `compaction_group_id` @@ -302,6 +305,8 @@ enum HummockCommands { /// KVs that are matched with the user key are printed. #[clap(long)] user_key: String, + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, PrintVersionDeltaInArchive { /// The ident of the archive file in object store. It's also the first Hummock version id of this archive. @@ -313,6 +318,8 @@ enum HummockCommands { /// Version deltas that are related to the SST id are printed. #[clap(long)] sst_id: u64, + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, } @@ -324,6 +331,9 @@ enum TableCommands { mv_name: String, // data directory for hummock state store. None: use default data_dir: Option, + + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, /// scan a state table using Id ScanById { @@ -331,6 +341,8 @@ enum TableCommands { table_id: u32, // data directory for hummock state store. None: use default data_dir: Option, + #[clap(short, long = "use-new-object-prefix-strategy", default_value = "true")] + use_new_object_prefix_strategy: bool, }, /// list all state tables List, @@ -640,8 +652,16 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { epoch, table_id, data_dir, + use_new_object_prefix_strategy, }) => { - cmd_impl::hummock::list_kv(context, epoch, table_id, data_dir).await?; + cmd_impl::hummock::list_kv( + context, + epoch, + table_id, + data_dir, + use_new_object_prefix_strategy, + ) + .await?; } Commands::Hummock(HummockCommands::SstDump(args)) => { cmd_impl::hummock::sst_dump(context, args).await.unwrap() @@ -764,12 +784,14 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { archive_ids, data_dir, sst_id, + use_new_object_prefix_strategy, }) => { cmd_impl::hummock::print_version_delta_in_archive( context, archive_ids, data_dir, sst_id, + use_new_object_prefix_strategy, ) .await?; } @@ -777,15 +799,32 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { archive_ids, data_dir, user_key, + use_new_object_prefix_strategy, }) => { - cmd_impl::hummock::print_user_key_in_archive(context, archive_ids, data_dir, user_key) - .await?; + cmd_impl::hummock::print_user_key_in_archive( + context, + archive_ids, + data_dir, + user_key, + use_new_object_prefix_strategy, + ) + .await?; } - Commands::Table(TableCommands::Scan { mv_name, data_dir }) => { - cmd_impl::table::scan(context, mv_name, data_dir).await? + Commands::Table(TableCommands::Scan { + mv_name, + data_dir, + use_new_object_prefix_strategy, + }) => { + cmd_impl::table::scan(context, mv_name, data_dir, use_new_object_prefix_strategy) + .await? } - Commands::Table(TableCommands::ScanById { table_id, data_dir }) => { - cmd_impl::table::scan_id(context, table_id, data_dir).await? + Commands::Table(TableCommands::ScanById { + table_id, + data_dir, + use_new_object_prefix_strategy, + }) => { + cmd_impl::table::scan_id(context, table_id, data_dir, use_new_object_prefix_strategy) + .await? } Commands::Table(TableCommands::List) => cmd_impl::table::list(context).await?, Commands::Bench(cmd) => cmd_impl::bench::do_bench(context, cmd).await?, diff --git a/src/java_binding/src/hummock_iterator.rs b/src/java_binding/src/hummock_iterator.rs index 4b6fc5b01742..5cfeb0ecf7e0 100644 --- a/src/java_binding/src/hummock_iterator.rs +++ b/src/java_binding/src/hummock_iterator.rs @@ -106,6 +106,7 @@ pub(crate) async fn new_hummock_java_binding_iter( state_store_metrics: Arc::new(global_hummock_state_store_metrics( MetricLevel::Disabled, )), + use_new_object_prefix_strategy: read_plan.use_new_object_prefix_strategy, meta_cache, block_cache, })); diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 627e565a47f3..fa886d0e7df9 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -1,7 +1,7 @@ #![allow(clippy::enum_variant_names)] pub use sea_orm_migration::prelude::*; - +pub use sea_orm_migration::MigrationStatus; mod m20230908_072257_init; mod m20231008_020431_hummock; mod m20240304_074901_subscription; diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index e8b738305dce..43b6342e22ed 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -36,7 +36,6 @@ use risingwave_meta::rpc::intercept::MetricsMiddlewareLayer; use risingwave_meta::rpc::ElectionClientRef; use risingwave_meta::stream::ScaleController; use risingwave_meta::MetaStoreBackend; -use risingwave_meta_model_migration::{Migrator, MigratorTrait}; use risingwave_meta_service::backup_service::BackupServiceImpl; use risingwave_meta_service::cloud_service::CloudServiceImpl; use risingwave_meta_service::cluster_service::ClusterServiceImpl; @@ -408,13 +407,6 @@ pub async fn start_service_as_election_leader( mut svc_shutdown_rx: WatchReceiver<()>, ) -> MetaResult<()> { tracing::info!("Defining leader services"); - if let MetaStoreImpl::Sql(sql_store) = &meta_store_impl { - // Try to upgrade if any new model changes are added. - Migrator::up(&sql_store.conn, None) - .await - .expect("Failed to upgrade models in meta store"); - } - let env = MetaSrvEnv::new( opts.clone(), init_system_params, diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index 1ba13c8cbf8b..e24abb68def1 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -246,6 +246,7 @@ mod tests { SystemParams { state_store: Some("state_store".into()), data_directory: Some("data_directory".into()), + use_new_object_prefix_strategy: Some(true), backup_storage_url: Some("backup_storage_url".into()), backup_storage_directory: Some("backup_storage_directory".into()), ..SystemConfig::default().into_init_system_params() diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index b623e441c0c2..7a284ca2fccd 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -20,6 +20,7 @@ use risingwave_common::config::{ }; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; +use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{StreamClientPool, StreamClientPoolRef}; @@ -348,15 +349,32 @@ impl MetaOpts { } } +/// This function `is_first_launch_for_sql_backend_cluster` is used to check whether the cluster, which uses SQL as the backend, is a new cluster. +/// It determines this by inspecting the applied migrations. If the migration `m20230908_072257_init` has been applied, +/// then it is considered an old cluster. +/// +/// Note: this check should be performed before `Migrator::up()`. +pub async fn is_first_launch_for_sql_backend_cluster( + sql_meta_store: &SqlMetaStore, +) -> MetaResult { + let migrations = Migrator::get_applied_migrations(&sql_meta_store.conn).await?; + for migration in migrations { + if migration.name() == "m20230908_072257_init" + && migration.status() == MigrationStatus::Applied + { + return Ok(false); + } + } + Ok(true) +} + impl MetaSrvEnv { pub async fn new( opts: MetaOpts, - init_system_params: SystemParams, + mut init_system_params: SystemParams, init_session_config: SessionConfig, meta_store_impl: MetaStoreImpl, ) -> MetaResult { - let notification_manager = - Arc::new(NotificationManager::new(meta_store_impl.clone()).await); let idle_manager = Arc::new(IdleManager::new(opts.max_idle_ms)); let stream_client_pool = Arc::new(StreamClientPool::default()); let event_log_manager = Arc::new(start_event_log_manager( @@ -366,6 +384,8 @@ impl MetaSrvEnv { let env = match &meta_store_impl { MetaStoreImpl::Kv(meta_store) => { + let notification_manager = + Arc::new(NotificationManager::new(meta_store_impl.clone()).await); let id_gen_manager = Arc::new(IdGeneratorManager::new(meta_store.clone()).await); let (cluster_id, cluster_first_launch) = if let Some(id) = ClusterId::from_meta_store(meta_store).await? { @@ -373,6 +393,11 @@ impl MetaSrvEnv { } else { (ClusterId::new(), true) }; + + // For new clusters, the name of the object store needs to be prefixed according to the object id. + // For old clusters, the prefix is ​​not divided for the sake of compatibility. + + init_system_params.use_new_object_prefix_strategy = Some(cluster_first_launch); let system_params_manager = Arc::new( SystemParamsManager::new( meta_store.clone(), @@ -415,11 +440,24 @@ impl MetaSrvEnv { } } MetaStoreImpl::Sql(sql_meta_store) => { + let is_sql_backend_cluster_first_launch = + is_first_launch_for_sql_backend_cluster(sql_meta_store).await?; + // Try to upgrade if any new model changes are added. + Migrator::up(&sql_meta_store.conn, None) + .await + .expect("Failed to upgrade models in meta store"); + + let notification_manager = + Arc::new(NotificationManager::new(meta_store_impl.clone()).await); let cluster_id = Cluster::find() .one(&sql_meta_store.conn) .await? .map(|c| c.cluster_id.to_string().into()) .unwrap(); + init_system_params.use_new_object_prefix_strategy = + Some(is_sql_backend_cluster_first_launch); + // For new clusters, the name of the object store needs to be prefixed according to the object id. + // For old clusters, the prefix is ​​not divided for the sake of compatibility. let system_param_controller = Arc::new( SystemParamsController::new( sql_meta_store.clone(), diff --git a/src/object_store/src/object/mem.rs b/src/object_store/src/object/mem.rs index 6bc78ae8b198..270cac3719d6 100644 --- a/src/object_store/src/object/mem.rs +++ b/src/object_store/src/object/mem.rs @@ -101,7 +101,7 @@ pub struct InMemObjectStore { impl ObjectStore for InMemObjectStore { type StreamingUploader = InMemStreamingUploader; - fn get_object_prefix(&self, _obj_id: u64) -> String { + fn get_object_prefix(&self, _obj_id: u64, _use_new_object_prefix_strategy: bool) -> String { String::default() } diff --git a/src/object_store/src/object/mod.rs b/src/object_store/src/object/mod.rs index 5369d914a751..e5e9cb966126 100644 --- a/src/object_store/src/object/mod.rs +++ b/src/object_store/src/object/mod.rs @@ -73,8 +73,8 @@ pub trait StreamingUploader: Send { #[async_trait::async_trait] pub trait ObjectStore: Send + Sync { type StreamingUploader: StreamingUploader; - /// Get the key prefix for object - fn get_object_prefix(&self, obj_id: u64) -> String; + /// Get the key prefix for object, the prefix is determined by the type of object store and `devise_object_prefix`. + fn get_object_prefix(&self, obj_id: u64, use_new_object_prefix_strategy: bool) -> String; /// Uploads the object to `ObjectStore`. async fn upload(&self, path: &str, obj: Bytes) -> ObjectResult<()>; @@ -325,8 +325,10 @@ impl ObjectStoreImpl { object_store_impl_method_body!(self, list(prefix).await) } - pub fn get_object_prefix(&self, obj_id: u64) -> String { - dispatch_object_store_enum!(self, |store| store.inner.get_object_prefix(obj_id)) + pub fn get_object_prefix(&self, obj_id: u64, use_new_object_prefix_strategy: bool) -> String { + dispatch_object_store_enum!(self, |store| store + .inner + .get_object_prefix(obj_id, use_new_object_prefix_strategy)) } pub fn support_streaming_upload(&self) -> bool { diff --git a/src/object_store/src/object/opendal_engine/opendal_object_store.rs b/src/object_store/src/object/opendal_engine/opendal_object_store.rs index 47ca4f362702..0d946e95d43f 100644 --- a/src/object_store/src/object/opendal_engine/opendal_object_store.rs +++ b/src/object_store/src/object/opendal_engine/opendal_object_store.rs @@ -72,18 +72,20 @@ impl OpendalObjectStore { impl ObjectStore for OpendalObjectStore { type StreamingUploader = OpendalStreamingUploader; - fn get_object_prefix(&self, obj_id: u64) -> String { + fn get_object_prefix(&self, obj_id: u64, use_new_object_prefix_strategy: bool) -> String { match self.engine_type { EngineType::S3 => prefix::s3::get_object_prefix(obj_id), EngineType::Minio => prefix::s3::get_object_prefix(obj_id), EngineType::Memory => String::default(), - EngineType::Hdfs => String::default(), - EngineType::Gcs => String::default(), - EngineType::Obs => String::default(), - EngineType::Oss => String::default(), - EngineType::Webhdfs => String::default(), - EngineType::Azblob => String::default(), - EngineType::Fs => String::default(), + EngineType::Hdfs + | EngineType::Gcs + | EngineType::Obs + | EngineType::Oss + | EngineType::Webhdfs + | EngineType::Azblob + | EngineType::Fs => { + prefix::opendal_engine::get_object_prefix(obj_id, use_new_object_prefix_strategy) + } } } diff --git a/src/object_store/src/object/prefix.rs b/src/object_store/src/object/prefix.rs index e29729bf752f..5229d900a8b6 100644 --- a/src/object_store/src/object/prefix.rs +++ b/src/object_store/src/object/prefix.rs @@ -23,3 +23,23 @@ pub(crate) mod s3 { obj_prefix } } + +pub(crate) mod opendal_engine { + /// The number of Azblob bucket prefixes + pub(crate) const NUM_BUCKET_PREFIXES_AZBLOB: u32 = 256; + + pub(crate) fn get_object_prefix(obj_id: u64, use_new_object_prefix_strategy: bool) -> String { + // For OpenDAL object storage, whether objects are divided by prefixes depends on whether it is a new cluster: + // If it is a new cluster, objects will be divided into `NUM_BUCKET_PREFIXES_AZBLOB` prefixes. + // If it is an old cluster, prefixes are not used due to the need to read and write old data. + match use_new_object_prefix_strategy { + true => { + let prefix = crc32fast::hash(&obj_id.to_be_bytes()) % NUM_BUCKET_PREFIXES_AZBLOB; + let mut obj_prefix = prefix.to_string(); + obj_prefix.push('/'); + obj_prefix + } + false => String::default(), + } + } +} diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index 6c72ced36563..f1f569cb7d36 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -402,8 +402,9 @@ pub struct S3ObjectStore { impl ObjectStore for S3ObjectStore { type StreamingUploader = S3StreamingUploader; - fn get_object_prefix(&self, obj_id: u64) -> String { + fn get_object_prefix(&self, obj_id: u64, _use_new_object_prefix_strategy: bool) -> String { // Delegate to static method to avoid creating an `S3ObjectStore` in unit test. + // Using aws s3 sdk as object storage, the object prefix will be divided by default. prefix::s3::get_object_prefix(obj_id) } diff --git a/src/object_store/src/object/sim/mod.rs b/src/object_store/src/object/sim/mod.rs index 4c91531020ea..2f06b9183909 100644 --- a/src/object_store/src/object/sim/mod.rs +++ b/src/object_store/src/object/sim/mod.rs @@ -118,7 +118,7 @@ pub struct SimObjectStore { impl ObjectStore for SimObjectStore { type StreamingUploader = SimStreamingUploader; - fn get_object_prefix(&self, _obj_id: u64) -> String { + fn get_object_prefix(&self, _obj_id: u64, _use_new_object_prefix_strategy: bool) -> String { String::default() } diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index 1ccb3a4eccaa..08da1438709c 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -80,6 +80,7 @@ pub async fn mock_sstable_store() -> SstableStoreRef { max_prefetch_block_number: 16, recent_filter: None, state_store_metrics: Arc::new(global_hummock_state_store_metrics(MetricLevel::Disabled)), + use_new_object_prefix_strategy: true, meta_cache, block_cache, diff --git a/src/storage/benches/bench_multi_builder.rs b/src/storage/benches/bench_multi_builder.rs index b67be3467c48..08d744189a1d 100644 --- a/src/storage/benches/bench_multi_builder.rs +++ b/src/storage/benches/bench_multi_builder.rs @@ -148,6 +148,7 @@ async fn generate_sstable_store(object_store: Arc) -> Arc Result Ssta recent_filter: None, state_store_metrics: Arc::new(global_hummock_state_store_metrics(MetricLevel::Disabled)), + use_new_object_prefix_strategy: true, meta_cache, block_cache, diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index a6ab2a6162e0..3240a4243a58 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -125,6 +125,7 @@ pub struct SstableStoreConfig { pub max_prefetch_block_number: usize, pub recent_filter: Option>>, pub state_store_metrics: Arc, + pub use_new_object_prefix_strategy: bool, pub meta_cache: HybridCache>, pub block_cache: HybridCache>, @@ -144,6 +145,15 @@ pub struct SstableStore { prefetch_buffer_usage: Arc, prefetch_buffer_capacity: usize, max_prefetch_block_number: usize, + /// Whether the object store is divided into prefixes depends on two factors: + /// 1. The specific object store type. + /// 2. Whether the existing cluster is a new cluster. + /// + /// The value of `use_new_object_prefix_strategy` is determined by the `use_new_object_prefix_strategy` field in the system parameters. + /// For a new cluster, `use_new_object_prefix_strategy` is set to True. + /// For an old cluster, `use_new_object_prefix_strategy` is set to False. + /// The final decision of whether to divide prefixes is based on this field and the specific object store type, this approach is implemented to ensure backward compatibility. + use_new_object_prefix_strategy: bool, } impl SstableStore { @@ -162,6 +172,7 @@ impl SstableStore { prefetch_buffer_usage: Arc::new(AtomicUsize::new(0)), prefetch_buffer_capacity: config.prefetch_buffer_capacity, max_prefetch_block_number: config.max_prefetch_block_number, + use_new_object_prefix_strategy: config.use_new_object_prefix_strategy, } } @@ -173,6 +184,7 @@ impl SstableStore { path: String, block_cache_capacity: usize, meta_cache_capacity: usize, + use_new_object_prefix_strategy: bool, ) -> HummockResult { let meta_cache = HybridCacheBuilder::new() .memory(meta_cache_capacity) @@ -205,6 +217,7 @@ impl SstableStore { prefetch_buffer_capacity: block_cache_capacity, max_prefetch_block_number: 16, /* compactor won't use this parameter, so just assign a default value. */ recent_filter: None, + use_new_object_prefix_strategy, meta_cache, block_cache, @@ -508,7 +521,9 @@ impl SstableStore { } pub fn get_sst_data_path(&self, object_id: HummockSstableObjectId) -> String { - let obj_prefix = self.store.get_object_prefix(object_id); + let obj_prefix = self + .store + .get_object_prefix(object_id, self.use_new_object_prefix_strategy); format!( "{}/{}{}.{}", self.path, obj_prefix, object_id, OBJECT_SUFFIX diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index fdecccc85893..2512f680b536 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -617,6 +617,7 @@ impl StateStoreImpl { storage_metrics: Arc, compactor_metrics: Arc, await_tree_config: Option, + use_new_object_prefix_strategy: bool, ) -> StorageResult { const MB: usize = 1 << 20; @@ -752,6 +753,7 @@ impl StateStoreImpl { max_prefetch_block_number: opts.max_prefetch_block_number, recent_filter, state_store_metrics: state_store_metrics.clone(), + use_new_object_prefix_strategy, meta_cache, block_cache, diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index e492b6497907..f294d8fb6fe5 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -711,6 +711,7 @@ pub async fn create_hummock_store_with_metrics( metrics.storage_metrics.clone(), metrics.compactor_metrics.clone(), None, + true, ) .await?; diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index 7183ee7e2e85..5d6c6ff7e70d 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -227,6 +227,7 @@ async fn compaction_test( max_prefetch_block_number: storage_opts.max_prefetch_block_number, recent_filter: None, state_store_metrics: state_store_metrics.clone(), + use_new_object_prefix_strategy: system_params.use_new_object_prefix_strategy(), meta_cache, block_cache, })); From e4dea80cb82fa7c4277f9209cfb462f226acabd5 Mon Sep 17 00:00:00 2001 From: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Date: Mon, 17 Jun 2024 18:15:36 +0800 Subject: [PATCH 07/17] fix(test): fix cassandra sink error (#17148) Co-authored-by: William Wen <44139337+wenym1@users.noreply.github.com> --- ci/scripts/e2e-cassandra-sink-test.sh | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/ci/scripts/e2e-cassandra-sink-test.sh b/ci/scripts/e2e-cassandra-sink-test.sh index cae03843c470..41ff0e97190d 100755 --- a/ci/scripts/e2e-cassandra-sink-test.sh +++ b/ci/scripts/e2e-cassandra-sink-test.sh @@ -33,7 +33,8 @@ tar xf ./risingwave-connector.tar.gz -C ./connector-node echo "--- starting risingwave cluster" risedev ci-start ci-sink-test -sleep 1 +# Wait cassandra server to start +sleep 40 echo "--- create cassandra table" curl https://downloads.apache.org/cassandra/4.1.3/apache-cassandra-4.1.3-bin.tar.gz --output apache-cassandra-4.1.3-bin.tar.gz @@ -47,7 +48,7 @@ pip3 install --break-system-packages cassandra-driver cd apache-cassandra-4.1.3/bin export CQLSH_HOST=cassandra-server export CQLSH_PORT=9042 -./cqlsh -e "CREATE KEYSPACE demo WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};use demo; +./cqlsh --request-timeout=20 -e "CREATE KEYSPACE demo WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1};use demo; CREATE table demo_bhv_table(v1 int primary key,v2 smallint,v3 bigint,v4 float,v5 double,v6 text,v7 date,v8 timestamp,v9 boolean);" echo "--- testing sinks" @@ -55,7 +56,7 @@ cd ../../ sqllogictest -p 4566 -d dev './e2e_test/sink/cassandra_sink.slt' sleep 1 cd apache-cassandra-4.1.3/bin -./cqlsh -e "COPY demo.demo_bhv_table TO './query_result.csv' WITH HEADER = false AND ENCODING = 'UTF-8';" +./cqlsh --request-timeout=20 -e "COPY demo.demo_bhv_table TO './query_result.csv' WITH HEADER = false AND ENCODING = 'UTF-8';" if cat ./query_result.csv | awk -F "," '{ exit !($1 == 1 && $2 == 1 && $3 == 1 && $4 == 1.1 && $5 == 1.2 && $6 == "test" && $7 == "2013-01-01" && $8 == "2013-01-01 01:01:01.000+0000" && $9 == "False\r"); }'; then From 32ecc0cd044a1837bded29e9600f3cd8ae21d241 Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Mon, 17 Jun 2024 21:16:14 +0800 Subject: [PATCH 08/17] test(connector): add integration test for sql server sink (#17280) --- integration_tests/sqlserver-sink/README.md | 86 +++++++++++++++++++ .../sqlserver-sink/docker-compose.yml | 46 ++++++++++ 2 files changed, 132 insertions(+) create mode 100644 integration_tests/sqlserver-sink/README.md create mode 100644 integration_tests/sqlserver-sink/docker-compose.yml diff --git a/integration_tests/sqlserver-sink/README.md b/integration_tests/sqlserver-sink/README.md new file mode 100644 index 000000000000..7006b55ff861 --- /dev/null +++ b/integration_tests/sqlserver-sink/README.md @@ -0,0 +1,86 @@ +# Demo: Sinking to Microsoft SQL Server + +In this demo, we want to showcase how RisingWave is able to sink data to Microsoft SQL Server. + + +1. Launch the cluster: + +```sh +docker-compose up -d +``` + +The cluster contains a RisingWave cluster and its necessary dependencies, a datagen that generates the data, a SQL Server instance for sink. + +2. Create the SQL Server table: + +```sh +docker exec -it sqlserver-server /opt/mssql-tools/bin/sqlcmd -S localhost -U SA -P SomeTestOnly@SA -Q " +CREATE DATABASE SinkTest; +GO +USE SinkTest; +GO +CREATE TABLE t_many_data_type ( + k1 int, k2 int, + c_boolean bit, + c_int16 smallint, + c_int32 int, + c_int64 bigint, + c_float32 float, + c_float64 float, + c_decimal decimal, + c_date date, + c_time time, + c_timestamp datetime2, + c_nvarchar nvarchar(1024), + c_varbinary varbinary(1024), + PRIMARY KEY (k1,k2) +); +GO" +``` + +3. Create the RisingWave table and sink: + +```sh +docker exec -it postgres-0 psql -h 127.0.0.1 -p 4566 -d dev -U root -c " +CREATE TABLE t_many_data_type_rw ( + k1 int, k2 int, + c_int16 smallint, + c_int32 int, + c_int64 bigint, + c_float32 float, + c_float64 double, + c_timestamp timestamp, + c_nvarchar string +) WITH ( + connector = 'datagen', + datagen.split.num = '1', + datagen.rows.per.second = '100', + fields.k1.kind = 'random', + fields.k1.min = '0', + fields.k1.max = '10000', + fields.k2.kind = 'random', + fields.k2.min = '0', + fields.k2.max = '10000' +); + +CREATE SINK s_many_data_type FROM t_many_data_type_rw WITH ( + connector = 'sqlserver', + type = 'upsert', + sqlserver.host = 'localhost', + sqlserver.port = 1433, + sqlserver.user = 'SA', + sqlserver.password = 'SomeTestOnly@SA', + sqlserver.database = 'SinkTest', + sqlserver.table = 't_many_data_type', + primary_key = 'k1,k2', +); +" +``` + +4. Verify the result in SQL Server, for example: + +```sh +docker exec -it sqlserver-server /opt/mssql-tools/bin/sqlcmd -S localhost -U SA -P SomeTestOnly@SA -Q " +SELECT count(*) FROM SinkTest.dbo.t_many_data_type; +" +``` diff --git a/integration_tests/sqlserver-sink/docker-compose.yml b/integration_tests/sqlserver-sink/docker-compose.yml new file mode 100644 index 000000000000..ec31f7d47e12 --- /dev/null +++ b/integration_tests/sqlserver-sink/docker-compose.yml @@ -0,0 +1,46 @@ +--- +services: + sqlserver-server: + container_name: sqlserver-server + image: mcr.microsoft.com/mssql/server:2022-latest + hostname: sqlserver-server + ports: + - 1433:1433 + environment: + ACCEPT_EULA: 'Y' + SA_PASSWORD: 'SomeTestOnly@SA' + risingwave-standalone: + extends: + file: ../../docker/docker-compose.yml + service: risingwave-standalone + postgres-0: + container_name: postgres-0 + extends: + file: ../../docker/docker-compose.yml + service: postgres-0 + grafana-0: + extends: + file: ../../docker/docker-compose.yml + service: grafana-0 + minio-0: + extends: + file: ../../docker/docker-compose.yml + service: minio-0 + prometheus-0: + extends: + file: ../../docker/docker-compose.yml + service: prometheus-0 +volumes: + risingwave-standalone: + external: false + postgres-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false + message_queue: + external: false +name: risingwave-compose From 8bea264b3f63d22a8134e7a8355c8b9db3d9b976 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 17 Jun 2024 21:53:05 +0800 Subject: [PATCH 09/17] refactor(meta): maintain fragment actor and table info in barrier manager (#17192) --- src/meta/src/barrier/command.rs | 194 +++++++++++------- src/meta/src/barrier/info.rs | 182 ++++++++++++---- src/meta/src/barrier/mod.rs | 9 +- src/meta/src/barrier/recovery.rs | 27 +-- src/meta/src/controller/catalog.rs | 8 +- src/meta/src/controller/fragment.rs | 67 +++--- src/meta/src/controller/utils.rs | 26 ++- src/meta/src/hummock/manager/commit_epoch.rs | 55 ++++- .../manager/compaction_group_manager.rs | 54 ++--- src/meta/src/hummock/manager/tests.rs | 99 ++++----- .../src/hummock/mock_hummock_meta_client.rs | 29 +-- src/meta/src/hummock/mod.rs | 1 - src/meta/src/hummock/test_utils.rs | 22 +- src/meta/src/manager/catalog/fragment.rs | 78 ++++--- src/meta/src/model/stream.rs | 19 -- src/meta/src/rpc/ddl_controller_v2.rs | 2 + src/meta/src/stream/stream_manager.rs | 14 +- src/meta/src/stream/test_fragmenter.rs | 8 +- .../compaction_group/hummock_version_ext.rs | 10 - 19 files changed, 539 insertions(+), 365 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 865835c49ede..785abbbc8907 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -41,7 +41,7 @@ use risingwave_pb::stream_service::WaitEpochCommitRequest; use thiserror_ext::AsReport; use tracing::warn; -use super::info::{ActorDesc, CommandActorChanges, InflightActorInfo}; +use super::info::{CommandActorChanges, CommandFragmentChanges, InflightActorInfo}; use super::trace::TracedEpoch; use crate::barrier::GlobalBarrierManagerContext; use crate::manager::{DdlType, MetadataManager, WorkerId}; @@ -101,26 +101,42 @@ pub struct ReplaceTablePlan { impl ReplaceTablePlan { fn actor_changes(&self) -> CommandActorChanges { - let worker_actors = self.new_table_fragments.worker_actor_ids(); - let barrier_inject_actors: &HashSet<_> = &self - .new_table_fragments - .barrier_inject_actor_ids() - .into_iter() - .collect(); - let to_add = worker_actors - .into_iter() - .flat_map(|(node_id, actors)| { - actors.into_iter().map(move |actor_id| ActorDesc { - id: actor_id, - node_id, - is_injectable: barrier_inject_actors.contains(&actor_id), - }) - }) - .collect_vec(); - CommandActorChanges { - to_add, - to_remove: self.old_table_fragments.actor_ids().into_iter().collect(), + let mut fragment_changes = HashMap::new(); + for fragment in self.new_table_fragments.fragments.values() { + let fragment_change = CommandFragmentChanges::NewFragment { + new_actors: fragment + .actors + .iter() + .map(|actor| { + ( + actor.actor_id, + self.new_table_fragments + .actor_status + .get(&actor.actor_id) + .expect("should exist") + .get_parallel_unit() + .expect("should set") + .worker_node_id, + ) + }) + .collect(), + table_ids: fragment + .state_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + is_injectable: TableFragments::is_injectable(fragment.fragment_type_mask), + }; + assert!(fragment_changes + .insert(fragment.fragment_id, fragment_change) + .is_none()); } + for fragment in self.old_table_fragments.fragments.values() { + assert!(fragment_changes + .insert(fragment.fragment_id, CommandFragmentChanges::RemoveFragment) + .is_none()); + } + CommandActorChanges { fragment_changes } } } @@ -153,8 +169,8 @@ pub enum Command { /// drop actors, and then delete the table fragments info from meta store. DropStreamingJobs { actors: Vec, - unregistered_table_fragment_ids: HashSet, unregistered_state_table_ids: HashSet, + unregistered_fragment_ids: HashSet, }, /// `CreateStreamingJob` command generates a `Add` barrier by given info. @@ -247,67 +263,91 @@ impl Command { Command::Plain(_) => None, Command::Pause(_) => None, Command::Resume(_) => None, - Command::DropStreamingJobs { actors, .. } => Some(CommandActorChanges { - to_add: Default::default(), - to_remove: actors.iter().cloned().collect(), + Command::DropStreamingJobs { + unregistered_fragment_ids, + .. + } => Some(CommandActorChanges { + fragment_changes: unregistered_fragment_ids + .iter() + .map(|fragment_id| (*fragment_id, CommandFragmentChanges::RemoveFragment)) + .collect(), }), Command::CreateStreamingJob { table_fragments, replace_table, .. } => { - let worker_actors = table_fragments.worker_actor_ids(); - let barrier_inject_actors: &HashSet<_> = &table_fragments - .barrier_inject_actor_ids() - .into_iter() - .collect(); - let mut to_add = worker_actors - .into_iter() - .flat_map(|(node_id, actors)| { - actors.into_iter().map(move |actor_id| ActorDesc { - id: actor_id, - node_id, - is_injectable: barrier_inject_actors.contains(&actor_id), - }) + let fragment_changes = table_fragments + .fragments + .values() + .map(|fragment| { + ( + fragment.fragment_id, + CommandFragmentChanges::NewFragment { + new_actors: fragment + .actors + .iter() + .map(|actor| { + ( + actor.actor_id, + table_fragments + .actor_status + .get(&actor.actor_id) + .expect("should exist") + .get_parallel_unit() + .expect("should set") + .worker_node_id, + ) + }) + .collect(), + table_ids: fragment + .state_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + is_injectable: TableFragments::is_injectable( + fragment.fragment_type_mask, + ), + }, + ) }) - .collect_vec(); + .collect(); + let mut changes = CommandActorChanges { fragment_changes }; if let Some(plan) = replace_table { - let CommandActorChanges { - to_add: to_add_plan, - to_remove, - } = plan.actor_changes(); - to_add.extend(to_add_plan); - Some(CommandActorChanges { to_add, to_remove }) - } else { - Some(CommandActorChanges { - to_add, - to_remove: Default::default(), - }) + let extra_change = plan.actor_changes(); + changes.extend(extra_change); } + + Some(changes) } Command::CancelStreamingJob(table_fragments) => Some(CommandActorChanges { - to_add: Default::default(), - to_remove: table_fragments.actor_ids().into_iter().collect(), + fragment_changes: table_fragments + .fragments + .values() + .map(|fragment| (fragment.fragment_id, CommandFragmentChanges::RemoveFragment)) + .collect(), + }), + Command::RescheduleFragment { reschedules, .. } => Some(CommandActorChanges { + fragment_changes: reschedules + .iter() + .map(|(fragment_id, reschedule)| { + ( + *fragment_id, + CommandFragmentChanges::Reschedule { + new_actors: reschedule + .added_actors + .iter() + .flat_map(|(node_id, actors)| { + actors.iter().map(|actor_id| (*actor_id, *node_id)) + }) + .collect(), + to_remove: reschedule.removed_actors.iter().cloned().collect(), + }, + ) + }) + .collect(), }), - Command::RescheduleFragment { reschedules, .. } => { - let mut to_add = vec![]; - let mut to_remove = HashSet::new(); - for reschedule in reschedules.values() { - for (node_id, added_actors) in &reschedule.added_actors { - for actor_id in added_actors { - to_add.push(ActorDesc { - id: *actor_id, - node_id: *node_id, - is_injectable: reschedule.injectable, - }); - } - } - to_remove.extend(reschedule.removed_actors.iter().copied()); - } - - Some(CommandActorChanges { to_add, to_remove }) - } Command::ReplaceTable(plan) => Some(plan.actor_changes()), Command::SourceSplitAssignment(_) => None, Command::Throttle(_) => None, @@ -873,13 +913,9 @@ impl CommandContext { // Tell compute nodes to drop actors. self.clean_up(actors.clone()).await?; - let unregistered_state_table_ids = unregistered_state_table_ids - .iter() - .map(|table_id| table_id.table_id) - .collect_vec(); self.barrier_manager_context .hummock_manager - .unregister_table_ids(&unregistered_state_table_ids) + .unregister_table_ids(unregistered_state_table_ids.iter().cloned()) .await?; } @@ -896,12 +932,9 @@ impl CommandContext { // It won't clean the tables on failure, // since the failure could be recoverable. // As such it needs to be handled here. - let table_id = table_fragments.table_id().table_id; - let mut table_ids = table_fragments.internal_table_ids(); - table_ids.push(table_id); self.barrier_manager_context .hummock_manager - .unregister_table_ids(&table_ids) + .unregister_table_ids(table_fragments.all_table_ids().map(TableId::new)) .await?; match &self.barrier_manager_context.metadata_manager { @@ -942,7 +975,10 @@ impl CommandContext { } MetadataManager::V2(mgr) => { mgr.catalog_controller - .try_abort_creating_streaming_job(table_id as _, true) + .try_abort_creating_streaming_job( + table_fragments.table_id().table_id as _, + true, + ) .await?; } } diff --git a/src/meta/src/barrier/info.rs b/src/meta/src/barrier/info.rs index aa8882d438dc..f6617b9ceef4 100644 --- a/src/meta/src/barrier/info.rs +++ b/src/meta/src/barrier/info.rs @@ -19,20 +19,37 @@ use risingwave_pb::common::PbWorkerNode; use tracing::warn; use crate::barrier::Command; -use crate::manager::{ActiveStreamingWorkerNodes, ActorInfos, WorkerId}; -use crate::model::ActorId; +use crate::manager::{ActiveStreamingWorkerNodes, ActorInfos, InflightFragmentInfo, WorkerId}; +use crate::model::{ActorId, FragmentId}; #[derive(Debug, Clone)] -pub struct ActorDesc { - pub id: ActorId, - pub node_id: WorkerId, - pub is_injectable: bool, +pub(crate) enum CommandFragmentChanges { + NewFragment { + new_actors: HashMap, + table_ids: HashSet, + is_injectable: bool, + }, + Reschedule { + new_actors: HashMap, + to_remove: HashSet, + }, + RemoveFragment, } #[derive(Debug, Clone)] pub struct CommandActorChanges { - pub(crate) to_add: Vec, - pub(crate) to_remove: HashSet, + pub(crate) fragment_changes: HashMap, +} + +impl CommandActorChanges { + pub fn extend(&mut self, other: CommandActorChanges) { + for (fragment_id, fragment) in other.fragment_changes { + assert!(self + .fragment_changes + .insert(fragment_id, fragment) + .is_none()); + } + } } /// [`InflightActorInfo`] resolves the actor info read from meta store for @@ -53,6 +70,8 @@ pub struct InflightActorInfo { /// `mv_table_id` => `subscription_id` => retention seconds pub mv_depended_subscriptions: HashMap>, + + pub fragment_infos: HashMap, } impl InflightActorInfo { @@ -64,22 +83,40 @@ impl InflightActorInfo { ) -> Self { let node_map = active_nodes.current().clone(); - let actor_map = actor_infos - .actor_maps - .into_iter() - .map(|(node_id, actor_ids)| (node_id, actor_ids.into_iter().collect::>())) - .collect::>(); + let actor_map = { + let mut map: HashMap<_, HashSet<_>> = HashMap::new(); + for info in actor_infos.fragment_infos.values() { + for (actor_id, worker_id) in &info.actors { + map.entry(*worker_id).or_default().insert(*actor_id); + } + } + map + }; - let actor_map_to_send = actor_infos - .barrier_inject_actor_maps - .into_iter() - .map(|(node_id, actor_ids)| (node_id, actor_ids.into_iter().collect::>())) - .collect::>(); + let actor_map_to_send = { + let mut map: HashMap<_, HashSet<_>> = HashMap::new(); + for info in actor_infos + .fragment_infos + .values() + .filter(|info| info.is_injectable) + { + for (actor_id, worker_id) in &info.actors { + map.entry(*worker_id).or_default().insert(*actor_id); + } + } + map + }; - let actor_location_map = actor_map - .iter() - .flat_map(|(node_id, actor_ids)| actor_ids.iter().map(|actor_id| (*actor_id, *node_id))) - .collect::>(); + let actor_location_map = actor_infos + .fragment_infos + .values() + .flat_map(|fragment| { + fragment + .actors + .iter() + .map(|(actor_id, workder_id)| (*actor_id, *workder_id)) + }) + .collect(); Self { node_map, @@ -87,6 +124,7 @@ impl InflightActorInfo { actor_map_to_send, actor_location_map, mv_depended_subscriptions, + fragment_infos: actor_infos.fragment_infos, } } @@ -107,33 +145,69 @@ impl InflightActorInfo { /// Apply some actor changes before issuing a barrier command, if the command contains any new added actors, we should update /// the info correspondingly. pub fn pre_apply(&mut self, command: &Command) { - if let Some(CommandActorChanges { to_add, .. }) = command.actor_changes() { - for actor_desc in to_add { - assert!(self.node_map.contains_key(&actor_desc.node_id)); + if let Some(CommandActorChanges { fragment_changes }) = command.actor_changes() { + let mut to_add = HashMap::new(); + for (fragment_id, change) in fragment_changes { + match change { + CommandFragmentChanges::NewFragment { + new_actors, + table_ids, + is_injectable, + } => { + for (actor_id, node_id) in &new_actors { + assert!(to_add + .insert(*actor_id, (*node_id, is_injectable)) + .is_none()); + } + assert!(self + .fragment_infos + .insert( + fragment_id, + InflightFragmentInfo { + actors: new_actors, + state_table_ids: table_ids, + is_injectable, + } + ) + .is_none()); + } + CommandFragmentChanges::Reschedule { new_actors, .. } => { + let info = self + .fragment_infos + .get_mut(&fragment_id) + .expect("should exist"); + let actors = &mut info.actors; + for (actor_id, node_id) in new_actors { + assert!(to_add + .insert(actor_id, (node_id, info.is_injectable)) + .is_none()); + assert!(actors.insert(actor_id, node_id).is_none()); + } + } + CommandFragmentChanges::RemoveFragment => {} + } + } + for (actor_id, (node_id, is_injectable)) in to_add { + assert!(self.node_map.contains_key(&node_id)); assert!( - self.actor_map - .entry(actor_desc.node_id) - .or_default() - .insert(actor_desc.id), + self.actor_map.entry(node_id).or_default().insert(actor_id), "duplicate actor in command changes" ); - if actor_desc.is_injectable { + if is_injectable { assert!( self.actor_map_to_send - .entry(actor_desc.node_id) + .entry(node_id) .or_default() - .insert(actor_desc.id), + .insert(actor_id), "duplicate actor in command changes" ); } assert!( - self.actor_location_map - .insert(actor_desc.id, actor_desc.node_id) - .is_none(), + self.actor_location_map.insert(actor_id, node_id).is_none(), "duplicate actor in command changes" ); } - }; + } if let Command::CreateSubscription { subscription_id, upstream_mv_table_id, @@ -154,8 +228,33 @@ impl InflightActorInfo { /// Apply some actor changes after the barrier command is collected, if the command contains any actors that are dropped, we should /// remove that from the snapshot correspondingly. pub fn post_apply(&mut self, command: &Command) { - if let Some(CommandActorChanges { to_remove, .. }) = command.actor_changes() { - for actor_id in to_remove { + if let Some(fragment_changes) = command.actor_changes() { + let mut all_to_remove = HashSet::new(); + for (fragment_id, changes) in fragment_changes.fragment_changes { + match changes { + CommandFragmentChanges::NewFragment { .. } => {} + CommandFragmentChanges::Reschedule { to_remove, .. } => { + let info = self + .fragment_infos + .get_mut(&fragment_id) + .expect("should exist"); + for actor_id in to_remove { + assert!(all_to_remove.insert(actor_id)); + assert!(info.actors.remove(&actor_id).is_some()); + } + } + CommandFragmentChanges::RemoveFragment => { + let info = self + .fragment_infos + .remove(&fragment_id) + .expect("should exist"); + for (actor_id, _) in info.actors { + assert!(all_to_remove.insert(actor_id)); + } + } + } + } + for actor_id in all_to_remove { let node_id = self .actor_location_map .remove(&actor_id) @@ -208,4 +307,11 @@ impl InflightActorInfo { .unwrap_or_default() .into_iter() } + + pub fn existing_table_ids(&self) -> HashSet { + self.fragment_infos + .values() + .flat_map(|info| info.state_table_ids.iter().cloned()) + .collect() + } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 6d5657a158e4..bb6737735dd4 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -900,10 +900,7 @@ impl GlobalBarrierManagerContext { BarrierKind::Initial => {} BarrierKind::Checkpoint(epochs) => { let commit_info = collect_commit_epoch_info(resps, command_ctx, epochs); - new_snapshot = self - .hummock_manager - .commit_epoch(command_ctx.prev_epoch.value().0, commit_info) - .await?; + new_snapshot = self.hummock_manager.commit_epoch(commit_info).await?; } BarrierKind::Barrier => { new_snapshot = Some(self.hummock_manager.update_current_epoch(prev_epoch)); @@ -1205,6 +1202,8 @@ fn collect_commit_epoch_info( }), ); + let epoch = command_ctx.prev_epoch.value().0; + CommitEpochInfo::new( synced_ssts, merge_multiple_new_table_watermarks( @@ -1226,5 +1225,7 @@ fn collect_commit_epoch_info( sst_to_worker, new_table_fragment_info, table_new_change_log, + BTreeMap::from_iter([(epoch, command_ctx.info.existing_table_ids())]), + epoch, ) } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 7200426d7830..8f4f54692e02 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -112,24 +112,11 @@ impl GlobalBarrierManagerContext { Ok(()) } - async fn purge_state_table_from_hummock(&self) -> MetaResult<()> { - let all_state_table_ids = match &self.metadata_manager { - MetadataManager::V1(mgr) => mgr - .catalog_manager - .list_tables() - .await - .into_iter() - .map(|t| t.id) - .collect_vec(), - MetadataManager::V2(mgr) => mgr - .catalog_controller - .list_all_state_table_ids() - .await? - .into_iter() - .map(|id| id as u32) - .collect_vec(), - }; - self.hummock_manager.purge(&all_state_table_ids).await?; + async fn purge_state_table_from_hummock( + &self, + all_state_table_ids: &HashSet, + ) -> MetaResult<()> { + self.hummock_manager.purge(all_state_table_ids).await?; Ok(()) } @@ -437,8 +424,10 @@ impl GlobalBarrierManager { })? } + let info = info; + self.context - .purge_state_table_from_hummock() + .purge_state_table_from_hummock(&info.existing_table_ids()) .await .context("purge state table from hummock")?; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 200736725cd0..050539b12c06 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -100,6 +100,8 @@ pub struct ReleaseContext { pub(crate) source_fragments: HashMap>, /// Dropped actors. pub(crate) removed_actors: HashSet, + + pub(crate) removed_fragments: HashSet, } impl CatalogController { @@ -228,7 +230,7 @@ impl CatalogController { .all(&txn) .await?; - let (source_fragments, removed_actors) = + let (source_fragments, removed_actors, removed_fragments) = resolve_source_register_info_for_jobs(&txn, streaming_jobs.clone()).await?; let state_table_ids: Vec = Table::find() @@ -332,6 +334,7 @@ impl CatalogController { connections, source_fragments, removed_actors, + removed_fragments, }, version, )) @@ -2079,7 +2082,7 @@ impl CatalogController { to_drop_objects.extend(to_drop_internal_table_objs); } - let (source_fragments, removed_actors) = + let (source_fragments, removed_actors, removed_fragments) = resolve_source_register_info_for_jobs(&txn, to_drop_streaming_jobs.clone()).await?; let fragment_mappings = @@ -2215,6 +2218,7 @@ impl CatalogController { connections: vec![], source_fragments, removed_actors, + removed_fragments, }, version, )) diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index fa5377374148..5320d555b94f 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.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, BTreeSet, HashMap, HashSet}; use std::mem::swap; @@ -54,8 +55,8 @@ use crate::controller::utils::{ get_actor_dispatchers, get_parallel_unit_to_worker_map, FragmentDesc, PartialActorLocation, PartialFragmentStateTables, }; -use crate::manager::{ActorInfos, LocalNotification}; -use crate::model::TableParallelism; +use crate::manager::{ActorInfos, InflightFragmentInfo, LocalNotification}; +use crate::model::{TableFragments, TableParallelism}; use crate::stream::SplitAssignment; use crate::{MetaError, MetaResult}; @@ -779,16 +780,6 @@ impl CatalogController { Ok(table_fragments) } - /// Check if the fragment type mask is injectable. - fn is_injectable(fragment_type_mask: u32) -> bool { - (fragment_type_mask - & (PbFragmentTypeFlag::Source as u32 - | PbFragmentTypeFlag::Now as u32 - | PbFragmentTypeFlag::Values as u32 - | PbFragmentTypeFlag::BarrierRecv as u32)) - != 0 - } - pub async fn list_actor_locations(&self) -> MetaResult> { let inner = self.inner.read().await; let actor_locations: Vec = @@ -873,37 +864,53 @@ impl CatalogController { /// collected pub async fn load_all_actors(&self) -> MetaResult { let inner = self.inner.read().await; - let actor_info: Vec<(ActorId, WorkerId, i32)> = Actor::find() + let actor_info: Vec<(ActorId, WorkerId, FragmentId, i32, Vec)> = Actor::find() .select_only() .column(actor::Column::ActorId) .column(actor::Column::WorkerId) + .column(fragment::Column::FragmentId) .column(fragment::Column::FragmentTypeMask) + .column(fragment::Column::StateTableIds) .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) .filter(actor::Column::Status.eq(ActorStatus::Running)) .into_tuple() .all(&inner.db) .await?; - let mut actor_maps = HashMap::new(); - let mut barrier_inject_actor_maps = HashMap::new(); - - for (actor_id, worker_id, type_mask) in actor_info { - actor_maps - .entry(worker_id as _) - .or_insert_with(Vec::new) - .push(actor_id as _); - if Self::is_injectable(type_mask as _) { - barrier_inject_actor_maps - .entry(worker_id as _) - .or_insert_with(Vec::new) - .push(actor_id as _); + let mut fragment_infos = HashMap::new(); + + for (actor_id, worker_id, fragment_id, type_mask, state_table_ids) in actor_info { + match fragment_infos.entry(fragment_id as crate::model::FragmentId) { + Entry::Occupied(mut entry) => { + let info: &mut InflightFragmentInfo = entry.get_mut(); + debug_assert_eq!( + info.state_table_ids, + state_table_ids + .into_iter() + .map(|table_id| risingwave_common::catalog::TableId::new(table_id as _)) + .collect() + ); + assert!(info.actors.insert(actor_id as _, worker_id as _).is_none()); + assert_eq!( + info.is_injectable, + TableFragments::is_injectable(type_mask as _) + ); + } + Entry::Vacant(entry) => { + let state_table_ids = state_table_ids + .into_iter() + .map(|table_id| risingwave_common::catalog::TableId::new(table_id as _)) + .collect(); + entry.insert(InflightFragmentInfo { + actors: HashMap::from_iter([(actor_id as _, worker_id as _)]), + state_table_ids, + is_injectable: TableFragments::is_injectable(type_mask as _), + }); + } } } - Ok(ActorInfos { - actor_maps, - barrier_inject_actor_maps, - }) + Ok(ActorInfos::new(fragment_infos)) } pub async fn migrate_actors(&self, plan: HashMap) -> MetaResult<()> { diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index b98788248a11..4597170ec364 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -934,15 +934,19 @@ pub fn find_stream_source(stream_node: &PbStreamNode) -> Option<&StreamSource> { pub async fn resolve_source_register_info_for_jobs( db: &C, streaming_jobs: Vec, -) -> MetaResult<(HashMap>, HashSet)> +) -> MetaResult<( + HashMap>, + HashSet, + HashSet, +)> where C: ConnectionTrait, { if streaming_jobs.is_empty() { - return Ok((HashMap::default(), HashSet::default())); + return Ok((HashMap::default(), HashSet::default(), HashSet::default())); } - let mut fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find() + let fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find() .select_only() .columns([ fragment::Column::FragmentId, @@ -963,10 +967,16 @@ where .all(db) .await?; - fragments.retain(|(_, mask, _)| *mask & PbFragmentTypeFlag::Source as i32 != 0); + let removed_fragments = fragments + .iter() + .map(|(fragment_id, _, _)| *fragment_id) + .collect(); let mut source_fragment_ids = HashMap::new(); - for (fragment_id, _, stream_node) in fragments { + for (fragment_id, mask, stream_node) in fragments { + if mask & PbFragmentTypeFlag::Source as i32 == 0 { + continue; + } if let Some(source) = find_stream_source(&stream_node.to_protobuf()) { source_fragment_ids .entry(source.source_id as SourceId) @@ -975,7 +985,11 @@ where } } - Ok((source_fragment_ids, actors.into_iter().collect())) + Ok(( + source_fragment_ids, + actors.into_iter().collect(), + removed_fragments, + )) } pub(crate) async fn get_parallel_unit_to_worker_map(db: &C) -> MetaResult> diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 44117139dc19..1437a6eb3bfd 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap}; +use std::collections::{BTreeMap, HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::TableId; @@ -54,6 +54,8 @@ pub struct CommitEpochInfo { pub sst_to_context: HashMap, pub new_table_fragment_info: Option, pub change_log_delta: HashMap, + pub table_committed_epoch: BTreeMap>, + pub max_committed_epoch: HummockEpoch, } impl CommitEpochInfo { @@ -63,6 +65,8 @@ impl CommitEpochInfo { sst_to_context: HashMap, new_table_fragment_info: Option, change_log_delta: HashMap, + table_committed_epoch: BTreeMap>, + max_committed_epoch: HummockEpoch, ) -> Self { Self { sstables, @@ -70,29 +74,46 @@ impl CommitEpochInfo { sst_to_context, new_table_fragment_info, change_log_delta, + table_committed_epoch, + max_committed_epoch, } } +} +impl HummockManager { #[cfg(any(test, feature = "test"))] - pub(crate) fn for_test( + pub async fn commit_epoch_for_test( + &self, + epoch: HummockEpoch, sstables: Vec>, sst_to_context: HashMap, - ) -> Self { - Self::new( + ) -> Result<()> { + let tables = self + .versioning + .read() + .await + .current_version + .state_table_info + .info() + .keys() + .cloned() + .collect(); + let info = CommitEpochInfo::new( sstables.into_iter().map(Into::into).collect(), HashMap::new(), sst_to_context, None, HashMap::new(), - ) + BTreeMap::from_iter([(epoch, tables)]), + epoch, + ); + self.commit_epoch(info).await?; + Ok(()) } -} -impl HummockManager { /// Caller should ensure `epoch` > `max_committed_epoch` pub async fn commit_epoch( &self, - epoch: HummockEpoch, commit_info: CommitEpochInfo, ) -> Result> { let CommitEpochInfo { @@ -101,6 +122,8 @@ impl HummockManager { sst_to_context, new_table_fragment_info, change_log_delta, + table_committed_epoch, + max_committed_epoch: epoch, } = commit_info; let mut versioning_guard = self.versioning.write().await; let _timer = start_measure_real_process_timer!(self, "commit_epoch"); @@ -341,6 +364,22 @@ impl HummockManager { new_version_delta.pre_apply(); + // TODO: remove the sanity check when supporting partial checkpoint + assert_eq!(1, table_committed_epoch.len()); + assert_eq!( + table_committed_epoch.iter().next().expect("non-empty"), + ( + &epoch, + &version + .latest_version() + .state_table_info + .info() + .keys() + .cloned() + .collect() + ) + ); + // Apply stats changes. let mut version_stats = HummockVersionStatsTransaction::new( &mut versioning.version_stats, diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 79596c8a3e77..c0c00d18a01e 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - get_compaction_group_ids, get_member_table_ids, TableGroupInfo, + get_compaction_group_ids, TableGroupInfo, }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::CompactionGroupId; @@ -152,10 +152,9 @@ impl HummockManager { table_fragments: &[crate::model::TableFragments], ) { self.unregister_table_ids( - &table_fragments + table_fragments .iter() - .flat_map(|t| t.all_table_ids()) - .collect_vec(), + .flat_map(|t| t.all_table_ids().map(TableId::new)), ) .await .unwrap(); @@ -164,16 +163,21 @@ impl HummockManager { /// Unregisters stale members and groups /// The caller should ensure `table_fragments_list` remain unchanged during `purge`. /// Currently `purge` is only called during meta service start ups. - pub async fn purge(&self, valid_ids: &[u32]) -> Result<()> { - let registered_members = - get_member_table_ids(&self.versioning.read().await.current_version); - let to_unregister = registered_members - .into_iter() + pub async fn purge(&self, valid_ids: &HashSet) -> Result<()> { + let to_unregister = self + .versioning + .read() + .await + .current_version + .state_table_info + .info() + .keys() + .cloned() .filter(|table_id| !valid_ids.contains(table_id)) .collect_vec(); // As we have released versioning lock, the version that `to_unregister` is calculated from // may not be the same as the one used in unregister_table_ids. It is OK. - self.unregister_table_ids(&to_unregister).await + self.unregister_table_ids(to_unregister).await } /// The implementation acquires `versioning` lock. @@ -275,8 +279,12 @@ impl HummockManager { Ok(()) } - pub async fn unregister_table_ids(&self, table_ids: &[StateTableId]) -> Result<()> { - if table_ids.is_empty() { + pub async fn unregister_table_ids( + &self, + table_ids: impl IntoIterator + Send, + ) -> Result<()> { + let mut table_ids = table_ids.into_iter().peekable(); + if table_ids.peek().is_none() { return Ok(()); } let mut versioning_guard = self.versioning.write().await; @@ -291,13 +299,9 @@ impl HummockManager { let mut modified_groups: HashMap = HashMap::new(); // Remove member tables - for table_id in table_ids.iter().unique() { + for table_id in table_ids.unique() { let version = new_version_delta.latest_version(); - let Some(info) = version - .state_table_info - .info() - .get(&TableId::new(*table_id)) - else { + let Some(info) = version.state_table_info.info().get(&table_id) else { continue; }; @@ -313,9 +317,7 @@ impl HummockManager { .len() as u64 - 1, ); - new_version_delta - .removed_table_ids - .insert(TableId::new(*table_id)); + new_version_delta.removed_table_ids.insert(table_id); } let groups_to_remove = modified_groups @@ -844,9 +846,8 @@ impl<'a> CompactionGroupTransaction<'a> { #[cfg(test)] mod tests { - use std::collections::BTreeMap; + use std::collections::{BTreeMap, HashSet}; - use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_config::MutableConfig; use risingwave_pb::meta::table_fragments::Fragment; @@ -986,11 +987,14 @@ mod tests { // Test purge_stale_members: table fragments compaction_group_manager - .purge(&table_fragment_2.all_table_ids().collect_vec()) + .purge(&table_fragment_2.all_table_ids().map(TableId::new).collect()) .await .unwrap(); assert_eq!(registered_number().await, 4); - compaction_group_manager.purge(&[]).await.unwrap(); + compaction_group_manager + .purge(&HashSet::new()) + .await + .unwrap(); assert_eq!(registered_number().await, 0); assert_eq!(group_number().await, 2); diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 8d4fcf89b33b..c2abc138e494 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -21,6 +21,7 @@ use std::sync::Arc; use itertools::Itertools; use prometheus::Registry; +use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::{test_epoch, EpochExt, INVALID_EPOCH}; use risingwave_hummock_sdk::compact::compact_task_to_string; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::get_compaction_group_ssts; @@ -46,7 +47,7 @@ use crate::hummock::compaction::selector::{ }; use crate::hummock::error::Error; use crate::hummock::test_utils::*; -use crate::hummock::{CommitEpochInfo, HummockManager, HummockManagerRef}; +use crate::hummock::{HummockManager, HummockManagerRef}; use crate::manager::{MetaSrvEnv, MetaStoreImpl, WorkerId}; use crate::model::MetadataModel; use crate::rpc::metrics::MetaMetrics; @@ -801,10 +802,7 @@ async fn test_invalid_sst_id() { .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), WorkerId::MAX)) .collect(); let error = hummock_manager - .commit_epoch( - epoch, - CommitEpochInfo::for_test(ssts.clone(), sst_to_worker), - ) + .commit_epoch_for_test(epoch, ssts.clone(), sst_to_worker) .await .unwrap_err(); assert!(matches!(error, Error::InvalidSst(1))); @@ -814,7 +812,7 @@ async fn test_invalid_sst_id() { .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) .collect(); hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await .unwrap(); } @@ -1188,9 +1186,10 @@ async fn test_extend_objects_to_delete() { assert_eq!(objects_to_delete.len(), orphan_sst_num as usize); let new_epoch = pinned_version2.max_committed_epoch.next_epoch(); hummock_manager - .commit_epoch( + .commit_epoch_for_test( new_epoch, - CommitEpochInfo::for_test(Vec::::new(), Default::default()), + Vec::::new(), + Default::default(), ) .await .unwrap(); @@ -1261,7 +1260,7 @@ async fn test_version_stats() { .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), worker_node.id)) .collect(); hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await .unwrap(); @@ -1360,10 +1359,7 @@ async fn test_split_compaction_group_on_commit() { table_stats: Default::default(), }; hummock_manager - .commit_epoch( - 30, - CommitEpochInfo::for_test(vec![sst_1], HashMap::from([(10, context_id)])), - ) + .commit_epoch_for_test(30, vec![sst_1], HashMap::from([(10, context_id)])) .await .unwrap(); let current_version = hummock_manager.get_current_version().await; @@ -1473,12 +1469,10 @@ async fn test_split_compaction_group_on_demand_basic() { table_stats: Default::default(), }; hummock_manager - .commit_epoch( + .commit_epoch_for_test( 30, - CommitEpochInfo::for_test( - vec![sst_1, sst_2], - HashMap::from([(10, context_id), (11, context_id)]), - ), + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), ) .await .unwrap(); @@ -1562,10 +1556,7 @@ async fn test_split_compaction_group_on_demand_non_trivial() { .await .unwrap(); hummock_manager - .commit_epoch( - 30, - CommitEpochInfo::for_test(vec![sst_1], HashMap::from([(10, context_id)])), - ) + .commit_epoch_for_test(30, vec![sst_1], HashMap::from([(10, context_id)])) .await .unwrap(); @@ -1670,17 +1661,15 @@ async fn test_split_compaction_group_trivial_expired() { sst_4.sst_info.sst_id = 9; sst_4.sst_info.object_id = 9; hummock_manager - .commit_epoch( + .commit_epoch_for_test( 30, - CommitEpochInfo::for_test( - vec![sst_1, sst_2, sst_3, sst_4], - HashMap::from([ - (10, context_id), - (11, context_id), - (9, context_id), - (8, context_id), - ]), - ), + vec![sst_1, sst_2, sst_3, sst_4], + HashMap::from([ + (10, context_id), + (11, context_id), + (9, context_id), + (8, context_id), + ]), ) .await .unwrap(); @@ -1817,10 +1806,7 @@ async fn test_split_compaction_group_on_demand_bottom_levels() { table_stats: Default::default(), }; hummock_manager - .commit_epoch( - 30, - CommitEpochInfo::for_test(vec![sst_1.clone()], HashMap::from([(10, context_id)])), - ) + .commit_epoch_for_test(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) .await .unwrap(); // Construct data via manual compaction @@ -1973,12 +1959,10 @@ async fn test_compaction_task_expiration_due_to_split_group() { table_stats: Default::default(), }; hummock_manager - .commit_epoch( + .commit_epoch_for_test( 30, - CommitEpochInfo::for_test( - vec![sst_1, sst_2], - HashMap::from([(10, context_id), (11, context_id)]), - ), + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), ) .await .unwrap(); @@ -2035,10 +2019,7 @@ async fn test_move_tables_between_compaction_group() { .unwrap(); let sst_1 = gen_extend_sstable_info(10, 2, 1, vec![100, 101, 102]); hummock_manager - .commit_epoch( - 30, - CommitEpochInfo::for_test(vec![sst_1.clone()], HashMap::from([(10, context_id)])), - ) + .commit_epoch_for_test(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) .await .unwrap(); // Construct data via manual compaction @@ -2061,10 +2042,7 @@ async fn test_move_tables_between_compaction_group() { .unwrap()); let sst_2 = gen_extend_sstable_info(14, 2, 1, vec![101, 102]); hummock_manager - .commit_epoch( - 31, - CommitEpochInfo::for_test(vec![sst_2.clone()], HashMap::from([(14, context_id)])), - ) + .commit_epoch_for_test(31, vec![sst_2.clone()], HashMap::from([(14, context_id)])) .await .unwrap(); let current_version = hummock_manager.get_current_version().await; @@ -2213,10 +2191,7 @@ async fn test_partition_level() { .unwrap(); let sst_1 = gen_extend_sstable_info(10, 2, 1, vec![100, 101]); hummock_manager - .commit_epoch( - 30, - CommitEpochInfo::for_test(vec![sst_1.clone()], HashMap::from([(10, context_id)])), - ) + .commit_epoch_for_test(30, vec![sst_1.clone()], HashMap::from([(10, context_id)])) .await .unwrap(); // Construct data via manual compaction @@ -2259,9 +2234,10 @@ async fn test_partition_level() { sst.sst_info.file_size = 10 * MB; sst.sst_info.uncompressed_file_size = 10 * MB; hummock_manager - .commit_epoch( + .commit_epoch_for_test( epoch, - CommitEpochInfo::for_test(vec![sst], HashMap::from([(global_sst_id, context_id)])), + vec![sst], + HashMap::from([(global_sst_id, context_id)]), ) .await .unwrap(); @@ -2363,12 +2339,10 @@ async fn test_unregister_moved_table() { table_stats: Default::default(), }; hummock_manager - .commit_epoch( + .commit_epoch_for_test( 30, - CommitEpochInfo::for_test( - vec![sst_1, sst_2], - HashMap::from([(10, context_id), (11, context_id)]), - ), + vec![sst_1, sst_2], + HashMap::from([(10, context_id), (11, context_id)]), ) .await .unwrap(); @@ -2413,7 +2387,10 @@ async fn test_unregister_moved_table() { vec![100] ); - hummock_manager.unregister_table_ids(&[100]).await.unwrap(); + hummock_manager + .unregister_table_ids([TableId::new(100)]) + .await + .unwrap(); let current_version = hummock_manager.get_current_version().await; assert_eq!(current_version.levels.len(), 2); assert!(!current_version.levels.contains_key(&new_group_id)); diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 90f70a4f74f3..dea226e28b04 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::BTreeMap; use std::sync::atomic::{AtomicU32, Ordering}; use std::sync::Arc; use std::time::SystemTime; @@ -178,20 +179,22 @@ impl HummockMetaClient for MockHummockMetaClient { .map(|table_id| (table_id.table_id, 0)), ); self.hummock_manager - .commit_epoch( + .commit_epoch(CommitEpochInfo::new( + sync_result + .uncommitted_ssts + .into_iter() + .map(|sst| sst.into()) + .collect(), + new_table_watermark, + sst_to_worker, + None, + table_change_log, + BTreeMap::from_iter([( + epoch, + version.state_table_info.info().keys().cloned().collect(), + )]), epoch, - CommitEpochInfo::new( - sync_result - .uncommitted_ssts - .into_iter() - .map(|sst| sst.into()) - .collect(), - new_table_watermark, - sst_to_worker, - None, - table_change_log, - ), - ) + )) .await .map_err(mock_err)?; Ok(()) diff --git a/src/meta/src/hummock/mod.rs b/src/meta/src/hummock/mod.rs index 65017943cb90..4150f51280d3 100644 --- a/src/meta/src/hummock/mod.rs +++ b/src/meta/src/hummock/mod.rs @@ -23,7 +23,6 @@ mod metrics_utils; #[cfg(any(test, feature = "test"))] pub mod mock_hummock_meta_client; pub mod model; -#[cfg(any(test, feature = "test"))] pub mod test_utils; mod utils; mod vacuum; diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 0090cceeed1c..32cbd4932adf 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -12,10 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![cfg(any(test, feature = "test"))] + use std::sync::Arc; use std::time::Duration; use itertools::Itertools; +use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::key_with_epoch; @@ -24,15 +27,13 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, }; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; -#[cfg(test)] use risingwave_pb::hummock::compact_task::TaskStatus; -use risingwave_pb::hummock::{CompactionConfig, HummockSnapshot, KeyRange, SstableInfo}; +use risingwave_pb::hummock::{CompactionConfig, KeyRange, SstableInfo}; use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; -#[cfg(test)] use crate::hummock::compaction::selector::default_compaction_selector; -use crate::hummock::{CommitEpochInfo, CompactorManager, HummockManager, HummockManagerRef}; +use crate::hummock::{CompactorManager, HummockManager, HummockManagerRef}; use crate::manager::{ ClusterManager, ClusterManagerRef, FragmentManager, MetaSrvEnv, META_NODE_ID, }; @@ -49,7 +50,6 @@ pub fn to_local_sstable_info(ssts: &[SstableInfo]) -> Vec { .collect_vec() } -#[cfg(test)] pub async fn add_test_tables( hummock_manager: &HummockManager, context_id: HummockContextId, @@ -73,7 +73,7 @@ pub async fn add_test_tables( .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) .collect(); hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await .unwrap(); // Simulate a compaction and increase version by 1. @@ -148,7 +148,7 @@ pub async fn add_test_tables( .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) .collect(); hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await .unwrap(); vec![test_tables, test_tables_2, test_tables_3] @@ -251,7 +251,7 @@ pub async fn unregister_table_ids_from_compaction_group( table_ids: &[u32], ) { hummock_manager_ref - .unregister_table_ids(table_ids) + .unregister_table_ids(table_ids.iter().map(|table_id| TableId::new(*table_id))) .await .unwrap(); } @@ -379,13 +379,13 @@ pub async fn commit_from_meta_node( hummock_manager_ref: &HummockManager, epoch: HummockEpoch, ssts: Vec, -) -> crate::hummock::error::Result> { +) -> crate::hummock::error::Result<()> { let sst_to_worker = ssts .iter() .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), META_NODE_ID)) .collect(); hummock_manager_ref - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await } @@ -402,7 +402,7 @@ pub async fn add_ssts( .map(|LocalSstableInfo { sst_info, .. }| (sst_info.get_object_id(), context_id)) .collect(); hummock_manager - .commit_epoch(epoch, CommitEpochInfo::for_test(ssts, sst_to_worker)) + .commit_epoch_for_test(epoch, ssts, sst_to_worker) .await .unwrap(); test_tables diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 9cb0d25a09ae..6c8390ff9299 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -24,6 +24,7 @@ use risingwave_common::hash::{ActorMapping, ParallelUnitId, ParallelUnitMapping} use risingwave_common::util::stream_graph_visitor::{ visit_stream_node, visit_stream_node_cont, visit_stream_node_cont_mut, }; +use risingwave_common::util::worker_util::WorkerNodeId; use risingwave_connector::source::SplitImpl; use risingwave_meta_model_v2::SourceId; use risingwave_pb::common::{PbParallelUnitMapping, PbWorkerSlotMapping}; @@ -119,12 +120,21 @@ pub struct FragmentManager { core: RwLock, } +#[derive(Clone)] +pub struct InflightFragmentInfo { + pub actors: HashMap, + pub state_table_ids: HashSet, + pub is_injectable: bool, +} + pub struct ActorInfos { - /// `node_id` => `actor_ids` - pub actor_maps: HashMap>, + pub fragment_infos: HashMap, +} - /// all reachable barrier inject actors - pub barrier_inject_actor_maps: HashMap>, +impl ActorInfos { + pub fn new(fragment_infos: HashMap) -> Self { + Self { fragment_infos } + } } pub type FragmentManagerRef = Arc; @@ -726,39 +736,45 @@ impl FragmentManager { /// Used in [`crate::barrier::GlobalBarrierManager`], load all running actor that need to be sent or /// collected pub async fn load_all_actors(&self) -> ActorInfos { - let mut actor_maps = HashMap::new(); - let mut barrier_inject_actor_maps = HashMap::new(); + let mut fragment_infos = HashMap::new(); let map = &self.core.read().await.table_fragments; for fragments in map.values() { - for (worker_id, actor_states) in fragments.worker_actor_states() { - for (actor_id, actor_state) in actor_states { - if actor_state == ActorState::Running { - actor_maps - .entry(worker_id) - .or_insert_with(Vec::new) - .push(actor_id); - } - } - } - - let barrier_inject_actors = fragments.worker_barrier_inject_actor_states(); - for (worker_id, actor_states) in barrier_inject_actors { - for (actor_id, actor_state) in actor_states { - if actor_state == ActorState::Running { - barrier_inject_actor_maps - .entry(worker_id) - .or_insert_with(Vec::new) - .push(actor_id); - } - } + for fragment in fragments.fragments.values() { + let info = InflightFragmentInfo { + actors: fragment + .actors + .iter() + .filter_map(|actor| { + let status = fragments + .actor_status + .get(&actor.actor_id) + .expect("should exist"); + if status.state == ActorState::Running as i32 { + Some(( + actor.actor_id, + status + .get_parallel_unit() + .expect("should set") + .worker_node_id, + )) + } else { + None + } + }) + .collect(), + state_table_ids: fragment + .state_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + is_injectable: TableFragments::is_injectable(fragment.fragment_type_mask), + }; + assert!(fragment_infos.insert(fragment.fragment_id, info,).is_none()); } } - ActorInfos { - actor_maps, - barrier_inject_actor_maps, - } + ActorInfos::new(fragment_infos) } async fn migrate_fragment_actors_inner( diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 4e49a602ed8a..144a33ae7b24 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -345,11 +345,6 @@ impl TableFragments { .collect() } - /// Returns barrier inject actor ids. - pub fn barrier_inject_actor_ids(&self) -> Vec { - Self::filter_actor_ids(self, Self::is_injectable) - } - /// Check if the fragment type mask is injectable. pub fn is_injectable(fragment_type_mask: u32) -> bool { (fragment_type_mask @@ -561,20 +556,6 @@ impl TableFragments { actors } - pub fn worker_barrier_inject_actor_states( - &self, - ) -> BTreeMap> { - let mut map = BTreeMap::default(); - let barrier_inject_actor_ids = self.barrier_inject_actor_ids(); - for &actor_id in &barrier_inject_actor_ids { - let actor_status = &self.actor_status[&actor_id]; - map.entry(actor_status.get_parallel_unit().unwrap().worker_node_id as WorkerId) - .or_insert_with(Vec::new) - .push((actor_id, actor_status.state())); - } - map - } - /// Returns actor map: `actor_id` => `StreamActor`. pub fn actor_map(&self) -> HashMap { let mut actor_map = HashMap::default(); diff --git a/src/meta/src/rpc/ddl_controller_v2.rs b/src/meta/src/rpc/ddl_controller_v2.rs index 37448e50a07d..0dabc9b19022 100644 --- a/src/meta/src/rpc/ddl_controller_v2.rs +++ b/src/meta/src/rpc/ddl_controller_v2.rs @@ -384,6 +384,7 @@ impl DdlController { connections, source_fragments, removed_actors, + removed_fragments, } = release_ctx; // delete vpc endpoints. @@ -423,6 +424,7 @@ impl DdlController { removed_actors.into_iter().map(|id| id as _).collect(), streaming_job_ids, state_table_ids, + removed_fragments.iter().map(|id| *id as _).collect(), ) .await; diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index d582095367fd..d6a257575fbd 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -30,7 +30,7 @@ use tracing::Instrument; use super::{Locations, RescheduleOptions, ScaleControllerRef, TableResizePolicy}; use crate::barrier::{BarrierScheduler, Command, ReplaceTablePlan, StreamRpcManager}; use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, StreamingJob}; -use crate::model::{ActorId, MetadataModel, TableFragments, TableParallelism}; +use crate::model::{ActorId, FragmentId, MetadataModel, TableFragments, TableParallelism}; use crate::stream::{to_build_actor_info, SourceManagerRef}; use crate::{MetaError, MetaResult}; @@ -550,6 +550,7 @@ impl GlobalStreamManager { removed_actors: Vec, streaming_job_ids: Vec, state_table_ids: Vec, + fragment_ids: HashSet, ) { if !removed_actors.is_empty() || !streaming_job_ids.is_empty() @@ -559,14 +560,11 @@ impl GlobalStreamManager { .barrier_scheduler .run_command(Command::DropStreamingJobs { actors: removed_actors, - unregistered_table_fragment_ids: streaming_job_ids - .into_iter() - .map(|job_id| TableId::new(job_id as _)) - .collect(), unregistered_state_table_ids: state_table_ids .into_iter() .map(|table_id| TableId::new(table_id as _)) .collect(), + unregistered_fragment_ids: fragment_ids, }) .await .inspect_err(|err| { @@ -601,11 +599,14 @@ impl GlobalStreamManager { .barrier_scheduler .run_command(Command::DropStreamingJobs { actors: dropped_actors, - unregistered_table_fragment_ids: table_ids.into_iter().collect(), unregistered_state_table_ids: unregister_table_ids .into_iter() .map(TableId::new) .collect(), + unregistered_fragment_ids: table_fragments_vec + .iter() + .flat_map(|fragments| fragments.fragments.keys().cloned()) + .collect(), }) .await .inspect_err(|err| { @@ -815,7 +816,6 @@ mod tests { CatalogManager, CatalogManagerRef, ClusterManager, FragmentManager, FragmentManagerRef, RelationIdEnum, StreamingClusterInfo, }; - use crate::model::FragmentId; use crate::rpc::ddl_controller::DropMode; use crate::rpc::metrics::MetaMetrics; use crate::stream::{ScaleController, SourceManager}; diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index f2d9a4324172..510011e69eb8 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -474,7 +474,13 @@ async fn test_graph_builder() -> MetaResult<()> { let table_fragments = TableFragments::for_test(TableId::default(), graph); let actors = table_fragments.actors(); - let barrier_inject_actor_ids = table_fragments.barrier_inject_actor_ids(); + let barrier_inject_actor_ids = table_fragments + .fragments + .values() + .filter(|fragment| TableFragments::is_injectable(fragment.fragment_type_mask)) + .flat_map(|fragment| fragment.actors.iter().map(|actor| actor.actor_id)) + .sorted() + .collect_vec(); let mview_actor_ids = table_fragments.mview_actor_ids(); assert_eq!(actors.len(), 9); diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 102e364b8d00..1aba344e166a 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -987,16 +987,6 @@ pub fn get_compaction_group_ids( version.levels.keys().cloned() } -/// Gets all member table ids. -pub fn get_member_table_ids(version: &HummockVersion) -> HashSet { - version - .state_table_info - .info() - .keys() - .map(|table_id| table_id.table_id) - .collect() -} - pub fn get_table_compaction_group_id_mapping( version: &HummockVersion, ) -> HashMap { From b6f8f46b8f1901fb2b00da53a88a62d766057b0e Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Mon, 17 Jun 2024 22:53:58 +0800 Subject: [PATCH 10/17] chore: upgrade chrono to 0.4.38 (#17279) --- Cargo.lock | 6 +++--- src/common/src/array/arrow/arrow_impl.rs | 10 ++++------ src/common/src/hash/key.rs | 4 ++-- src/common/src/types/datetime.rs | 17 ++++++++++------- src/common/src/types/mod.rs | 4 ++-- src/common/src/util/value_encoding/mod.rs | 8 +++++--- src/connector/src/parser/avro/parser.rs | 6 ++++-- src/connector/src/parser/mysql.rs | 4 +++- src/connector/src/parser/unified/json.rs | 4 +++- src/connector/src/sink/encoder/json.rs | 2 +- src/connector/src/source/iceberg/mod.rs | 2 +- src/ctl/src/cmd_impl/meta/migration.rs | 20 +++++++++++++------- src/expr/impl/src/scalar/extract.rs | 7 ++++--- src/expr/impl/src/scalar/to_char.rs | 2 ++ src/expr/impl/src/scalar/tumble.rs | 4 ++-- src/meta/src/controller/mod.rs | 20 ++++++++++---------- 16 files changed, 69 insertions(+), 51 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index d713c3276b11..0ad318dc37db 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2386,9 +2386,9 @@ checksum = "613afe47fcd5fac7ccf1db93babcb082c5994d996f20b8b159f2ad1658eb5724" [[package]] name = "chrono" -version = "0.4.31" +version = "0.4.38" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7f2c685bad3eb3d45a01354cedb7d5faa66194d1d58ba6e267a8de788f79db38" +checksum = "a21f936df1771bf62b77f047b726c4625ff2e8aa607c01ec06e5a05bd8463401" dependencies = [ "android-tzdata", "iana-time-zone", @@ -2396,7 +2396,7 @@ dependencies = [ "num-traits", "serde", "wasm-bindgen", - "windows-targets 0.48.5", + "windows-targets 0.52.0", ] [[package]] diff --git a/src/common/src/array/arrow/arrow_impl.rs b/src/common/src/array/arrow/arrow_impl.rs index 773efdd088e3..10ea6b6e6ec4 100644 --- a/src/common/src/array/arrow/arrow_impl.rs +++ b/src/common/src/array/arrow/arrow_impl.rs @@ -43,7 +43,7 @@ use std::fmt::Write; use arrow_buffer::OffsetBuffer; -use chrono::{NaiveDateTime, NaiveTime}; +use chrono::{DateTime, NaiveDateTime, NaiveTime}; use itertools::Itertools; // This is important because we want to use the arrow version specified by the outer mod. @@ -848,11 +848,9 @@ impl FromIntoArrow for Timestamp { fn from_arrow(value: Self::ArrowType) -> Self { Timestamp( - NaiveDateTime::from_timestamp_opt( - (value / 1_000_000) as _, - (value % 1_000_000 * 1000) as _, - ) - .unwrap(), + DateTime::from_timestamp((value / 1_000_000) as _, (value % 1_000_000 * 1000) as _) + .unwrap() + .naive_utc(), ) } diff --git a/src/common/src/hash/key.rs b/src/common/src/hash/key.rs index c7e57173a3e7..e9f7e83ac914 100644 --- a/src/common/src/hash/key.rs +++ b/src/common/src/hash/key.rs @@ -567,8 +567,8 @@ impl HashKeyDe for Date { impl HashKeySer<'_> for Timestamp { fn serialize_into(self, mut buf: impl BufMut) { - buf.put_i64_ne(self.0.timestamp()); - buf.put_u32_ne(self.0.timestamp_subsec_nanos()); + buf.put_i64_ne(self.0.and_utc().timestamp()); + buf.put_u32_ne(self.0.and_utc().timestamp_subsec_nanos()); } fn exact_size() -> Option { diff --git a/src/common/src/types/datetime.rs b/src/common/src/types/datetime.rs index 7058d36ec6fd..bac96b6c1dea 100644 --- a/src/common/src/types/datetime.rs +++ b/src/common/src/types/datetime.rs @@ -21,7 +21,9 @@ use std::io::Write; use std::str::FromStr; use bytes::{Bytes, BytesMut}; -use chrono::{Datelike, Days, Duration, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Weekday}; +use chrono::{ + DateTime, Datelike, Days, Duration, NaiveDate, NaiveDateTime, NaiveTime, Timelike, Weekday, +}; use postgres_types::{accepts, to_sql_checked, FromSql, IsNull, ToSql, Type}; use risingwave_common_estimate_size::ZeroHeapSize; use thiserror::Error; @@ -570,7 +572,8 @@ impl Time { impl Timestamp { pub fn with_secs_nsecs(secs: i64, nsecs: u32) -> Result { Ok(Timestamp::new({ - NaiveDateTime::from_timestamp_opt(secs, nsecs) + DateTime::from_timestamp(secs, nsecs) + .map(|t| t.naive_utc()) .ok_or_else(|| InvalidParamsError::datetime(secs, nsecs))? })) } @@ -578,12 +581,12 @@ impl Timestamp { /// Although `Timestamp` takes 12 bytes, we drop 4 bytes in protobuf encoding. pub fn to_protobuf(self, output: &mut T) -> ArrayResult { output - .write(&(self.0.timestamp_micros()).to_be_bytes()) + .write(&(self.0.and_utc().timestamp_micros()).to_be_bytes()) .map_err(Into::into) } pub fn get_timestamp_nanos(&self) -> i64 { - self.0.timestamp_nanos_opt().unwrap() + self.0.and_utc().timestamp_nanos_opt().unwrap() } pub fn with_millis(timestamp_millis: i64) -> Result { @@ -599,7 +602,7 @@ impl Timestamp { } pub fn from_timestamp_uncheck(secs: i64, nsecs: u32) -> Self { - Self::new(NaiveDateTime::from_timestamp_opt(secs, nsecs).unwrap()) + Self::new(DateTime::from_timestamp(secs, nsecs).unwrap().naive_utc()) } /// Truncate the timestamp to the precision of microseconds. @@ -879,10 +882,10 @@ mod tests { Timestamp::from_str("2022-08-03 10:34:02").unwrap() ); let ts = Timestamp::from_str("0001-11-15 07:35:40.999999").unwrap(); - assert_eq!(ts.0.timestamp_micros(), -62108094259000001); + assert_eq!(ts.0.and_utc().timestamp_micros(), -62108094259000001); let ts = Timestamp::from_str("1969-12-31 23:59:59.999999").unwrap(); - assert_eq!(ts.0.timestamp_micros(), -1); + assert_eq!(ts.0.and_utc().timestamp_micros(), -1); // invalid datetime Date::from_str("1999-01-08AA").unwrap_err(); diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index d793561f5957..69e727cbde65 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -1001,8 +1001,8 @@ impl ScalarRefImpl<'_> { Self::Interval(v) => v.serialize(ser)?, Self::Date(v) => v.0.num_days_from_ce().serialize(ser)?, Self::Timestamp(v) => { - v.0.timestamp().serialize(&mut *ser)?; - v.0.timestamp_subsec_nanos().serialize(ser)?; + v.0.and_utc().timestamp().serialize(&mut *ser)?; + v.0.and_utc().timestamp_subsec_nanos().serialize(ser)?; } Self::Timestamptz(v) => v.serialize(ser)?, Self::Time(v) => { diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index aa80c1561704..a3da88911ad9 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -214,9 +214,11 @@ fn serialize_scalar(value: ScalarRefImpl<'_>, buf: &mut impl BufMut) { ScalarRefImpl::Decimal(v) => serialize_decimal(&v, buf), ScalarRefImpl::Interval(v) => serialize_interval(&v, buf), ScalarRefImpl::Date(v) => serialize_date(v.0.num_days_from_ce(), buf), - ScalarRefImpl::Timestamp(v) => { - serialize_timestamp(v.0.timestamp(), v.0.timestamp_subsec_nanos(), buf) - } + ScalarRefImpl::Timestamp(v) => serialize_timestamp( + v.0.and_utc().timestamp(), + v.0.and_utc().timestamp_subsec_nanos(), + buf, + ), ScalarRefImpl::Timestamptz(v) => buf.put_i64_le(v.timestamp_micros()), ScalarRefImpl::Time(v) => { serialize_time(v.0.num_seconds_from_midnight(), v.0.nanosecond(), buf) diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 4b73c6ac7c80..6d69460ee119 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -373,12 +373,14 @@ mod test { } Schema::TimestampMillis => { let datetime = Date::from_ymd_uncheck(1970, 1, 1).and_hms_uncheck(0, 0, 0); - let timestamp_mills = Value::TimestampMillis(datetime.0.timestamp() * 1_000); + let timestamp_mills = + Value::TimestampMillis(datetime.0.and_utc().timestamp() * 1_000); Some(timestamp_mills) } Schema::TimestampMicros => { let datetime = Date::from_ymd_uncheck(1970, 1, 1).and_hms_uncheck(0, 0, 0); - let timestamp_micros = Value::TimestampMicros(datetime.0.timestamp() * 1_000_000); + let timestamp_micros = + Value::TimestampMicros(datetime.0.and_utc().timestamp() * 1_000_000); Some(timestamp_micros) } Schema::Duration => { diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index cfc7f1fe76e6..d1df27263e80 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -98,7 +98,9 @@ pub fn mysql_row_to_owned_row(mysql_row: &mut MysqlRow, schema: &Schema) -> Owne .unwrap_or(Ok(None)); match res { Ok(val) => val.map(|v| { - ScalarImpl::from(Timestamptz::from_micros(v.timestamp_micros())) + ScalarImpl::from(Timestamptz::from_micros( + v.and_utc().timestamp_micros(), + )) }), Err(err) => { log_error!(name, err, "parse column failed"); diff --git a/src/connector/src/parser/unified/json.rs b/src/connector/src/parser/unified/json.rs index 09704d9192a4..e4a229bb61b9 100644 --- a/src/connector/src/parser/unified/json.rs +++ b/src/connector/src/parser/unified/json.rs @@ -496,7 +496,9 @@ impl JsonParseOptions { .as_str() .unwrap() .parse::() - .map(|naive_utc| Timestamptz::from_micros(naive_utc.0.timestamp_micros())) + .map(|naive_utc| { + Timestamptz::from_micros(naive_utc.0.and_utc().timestamp_micros()) + }) .map_err(|_| create_error())? .into(), // Unless explicitly requested `utc_without_utc`, we parse string with `YYYY-MM-DDTHH:MM:SSZ`. diff --git a/src/connector/src/sink/encoder/json.rs b/src/connector/src/sink/encoder/json.rs index e1ce9e61b6a1..474811660970 100644 --- a/src/connector/src/sink/encoder/json.rs +++ b/src/connector/src/sink/encoder/json.rs @@ -269,7 +269,7 @@ fn datum_to_json_object( } }, (DataType::Timestamp, ScalarRefImpl::Timestamp(v)) => match timestamp_handling_mode { - TimestampHandlingMode::Milli => json!(v.0.timestamp_millis()), + TimestampHandlingMode::Milli => json!(v.0.and_utc().timestamp_millis()), TimestampHandlingMode::String => json!(v.0.format("%Y-%m-%d %H:%M:%S%.6f").to_string()), }, (DataType::Bytea, ScalarRefImpl::Bytea(v)) => { diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index e7e4971dd42c..1090159fa9ca 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -190,7 +190,7 @@ impl IcebergSplitEnumerator { Some(snapshot) => snapshot.snapshot_id, None => { // convert unix time to human readable time - let time = chrono::NaiveDateTime::from_timestamp_millis(timestamp); + let time = chrono::DateTime::from_timestamp_millis(timestamp); if time.is_some() { bail!("Cannot find a snapshot older than {}", time.unwrap()); } else { diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index b9aa81b67751..a7f839527194 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -16,6 +16,7 @@ use std::collections::{BTreeSet, HashMap}; use std::time::Duration; use anyhow::Context; +use chrono::DateTime; use etcd_client::ConnectOptions; use itertools::Itertools; use risingwave_common::util::epoch::Epoch; @@ -68,7 +69,6 @@ use risingwave_pb::meta::table_fragments::State; use risingwave_pb::meta::PbSystemParams; use risingwave_pb::user::grant_privilege::PbObject as GrantObject; use risingwave_pb::user::PbUserInfo; -use sea_orm::prelude::DateTime; use sea_orm::ActiveValue::Set; use sea_orm::{ ColumnTrait, ConnectionTrait, DatabaseBackend, DbBackend, EntityTrait, IntoActiveModel, NotSet, @@ -360,11 +360,15 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an }; if let Some(epoch) = object.initialized_at_epoch.map(Epoch::from) { obj.initialized_at = - Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _).unwrap()); + Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _) + .unwrap() + .naive_utc()); } if let Some(epoch) = object.created_at_epoch.map(Epoch::from) { obj.created_at = - Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _).unwrap()); + Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _) + .unwrap() + .naive_utc()); } Object::insert(obj).exec(&meta_store_sql.conn).await?; } @@ -390,12 +394,14 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an ..Default::default() }; if let Some(epoch) = table.initialized_at_epoch.map(Epoch::from) { - obj.initialized_at = - Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _).unwrap()); + obj.initialized_at = Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _) + .unwrap() + .naive_utc()); } if let Some(epoch) = table.created_at_epoch.map(Epoch::from) { - obj.created_at = - Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _).unwrap()); + obj.created_at = Set(DateTime::from_timestamp_millis(epoch.as_unix_millis() as _) + .unwrap() + .naive_utc()); } Object::insert(obj).exec(&meta_store_sql.conn).await?; } diff --git a/src/expr/impl/src/scalar/extract.rs b/src/expr/impl/src/scalar/extract.rs index 080c41037736..74a50cf6869e 100644 --- a/src/expr/impl/src/scalar/extract.rs +++ b/src/expr/impl/src/scalar/extract.rs @@ -65,7 +65,7 @@ fn extract_from_timelike(time: impl Timelike, unit: Unit) -> Decimal { fn extract_from_date(date: Date, unit: &Unit) -> Decimal { match unit { Epoch => { - let epoch = date.0.and_time(NaiveTime::default()).timestamp(); + let epoch = date.0.and_time(NaiveTime::default()).and_utc().timestamp(); epoch.into() } Julian => { @@ -92,11 +92,12 @@ fn extract_from_time(time: Time, unit: &Unit) -> Decimal { fn extract_from_timestamp(timestamp: Timestamp, unit: &Unit) -> Decimal { match unit { Epoch => { - let epoch = timestamp.0.timestamp_micros(); + let epoch = timestamp.0.and_utc().timestamp_micros(); Decimal::from_i128_with_scale(epoch as i128, 6) } Julian => { - let epoch = Decimal::from_i128_with_scale(timestamp.0.timestamp_micros() as i128, 6); + let epoch = + Decimal::from_i128_with_scale(timestamp.0.and_utc().timestamp_micros() as i128, 6); epoch / (24 * 60 * 60).into() + 2_440_588.into() } _ if unit.is_date_unit() => extract_from_datelike(timestamp.0.date(), *unit), diff --git a/src/expr/impl/src/scalar/to_char.rs b/src/expr/impl/src/scalar/to_char.rs index eef011d29ef0..7c9e859675d0 100644 --- a/src/expr/impl/src/scalar/to_char.rs +++ b/src/expr/impl/src/scalar/to_char.rs @@ -351,6 +351,7 @@ fn format_inner(w: &mut impl Write, interval: Interval, item: &Item<'_>) -> Resu | WeekFromMon | IsoYearDiv100 | Timestamp | YearDiv100 | Internal(_) => { unreachable!() } + _ => unreachable!(), } Ok(()) } @@ -395,6 +396,7 @@ fn format_inner(w: &mut impl Write, interval: Interval, item: &Item<'_>) -> Resu | Nanosecond9 | RFC2822 | RFC3339 => unreachable!(), + _ => unreachable!(), } } Item::Error => Err(invalid_pattern_err()), diff --git a/src/expr/impl/src/scalar/tumble.rs b/src/expr/impl/src/scalar/tumble.rs index 3e383e59236e..bf6f32253207 100644 --- a/src/expr/impl/src/scalar/tumble.rs +++ b/src/expr/impl/src/scalar/tumble.rs @@ -38,7 +38,7 @@ pub fn tumble_start_date(timestamp: Date, window_size: Interval) -> Result timestamp")] pub fn tumble_start_date_time(timestamp: Timestamp, window_size: Interval) -> Result { - let timestamp_micro_second = timestamp.0.timestamp_micros(); + let timestamp_micro_second = timestamp.0.and_utc().timestamp_micros(); let window_start_micro_second = get_window_start(timestamp_micro_second, window_size)?; Ok(Timestamp::from_timestamp_uncheck( window_start_micro_second / 1_000_000, @@ -72,7 +72,7 @@ pub fn tumble_start_offset_date_time( window_size: Interval, offset: Interval, ) -> Result { - let timestamp_micro_second = time.0.timestamp_micros(); + let timestamp_micro_second = time.0.and_utc().timestamp_micros(); let window_start_micro_second = get_window_start_with_offset(timestamp_micro_second, window_size, offset)?; diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 66491d515dd4..4fb5d086a060 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -143,10 +143,10 @@ impl From> for PbTable { .cardinality .map(|cardinality| cardinality.to_protobuf()), initialized_at_epoch: Some( - Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.initialized_at.and_utc().timestamp_millis() as _).0, ), created_at_epoch: Some( - Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.created_at.and_utc().timestamp_millis() as _).0, ), cleaned_by_watermark: value.0.cleaned_by_watermark, stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. @@ -183,10 +183,10 @@ impl From> for PbSource { connection_id: value.0.connection_id.map(|id| id as _), // todo: using the timestamp from the database directly. initialized_at_epoch: Some( - Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.initialized_at.and_utc().timestamp_millis() as _).0, ), created_at_epoch: Some( - Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.created_at.and_utc().timestamp_millis() as _).0, ), version: value.0.version as _, optional_associated_table_id: value @@ -221,10 +221,10 @@ impl From> for PbSink { definition: value.0.definition, connection_id: value.0.connection_id.map(|id| id as _), initialized_at_epoch: Some( - Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.initialized_at.and_utc().timestamp_millis() as _).0, ), created_at_epoch: Some( - Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.created_at.and_utc().timestamp_millis() as _).0, ), db_name: value.0.db_name, sink_from_name: value.0.sink_from_name, @@ -250,10 +250,10 @@ impl From> for PbSubscription { retention_seconds: value.0.retention_seconds as _, definition: value.0.definition, initialized_at_epoch: Some( - Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.initialized_at.and_utc().timestamp_millis() as _).0, ), created_at_epoch: Some( - Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.created_at.and_utc().timestamp_millis() as _).0, ), initialized_at_cluster_version: value.1.initialized_at_cluster_version, created_at_cluster_version: value.1.created_at_cluster_version, @@ -277,10 +277,10 @@ impl From> for PbIndex { index_column_properties: value.0.index_column_properties.to_protobuf(), index_columns_len: value.0.index_columns_len as _, initialized_at_epoch: Some( - Epoch::from_unix_millis(value.1.initialized_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.initialized_at.and_utc().timestamp_millis() as _).0, ), created_at_epoch: Some( - Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, + Epoch::from_unix_millis(value.1.created_at.and_utc().timestamp_millis() as _).0, ), stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. initialized_at_cluster_version: value.1.initialized_at_cluster_version, From d555b5c11b6c540abe356454f3409b30a8fb43a6 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 17 Jun 2024 14:21:59 -0500 Subject: [PATCH 11/17] fix(migration): use correct type I32Array instead of Vec (#17282) --- src/meta/src/controller/fragment.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 5320d555b94f..fb8810071f16 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -864,7 +864,7 @@ impl CatalogController { /// collected pub async fn load_all_actors(&self) -> MetaResult { let inner = self.inner.read().await; - let actor_info: Vec<(ActorId, WorkerId, FragmentId, i32, Vec)> = Actor::find() + let actor_info: Vec<(ActorId, WorkerId, FragmentId, i32, I32Array)> = Actor::find() .select_only() .column(actor::Column::ActorId) .column(actor::Column::WorkerId) @@ -880,6 +880,7 @@ impl CatalogController { let mut fragment_infos = HashMap::new(); for (actor_id, worker_id, fragment_id, type_mask, state_table_ids) in actor_info { + let state_table_ids = state_table_ids.into_inner(); match fragment_infos.entry(fragment_id as crate::model::FragmentId) { Entry::Occupied(mut entry) => { let info: &mut InflightFragmentInfo = entry.get_mut(); From d5e10d830e0dd03cb51d8e46f4277b20ffb75c9e Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Mon, 17 Jun 2024 15:23:35 -0500 Subject: [PATCH 12/17] fix(secret): add a secret ref column to source (#17269) --- proto/catalog.proto | 19 +++++-- src/connector/src/sink/catalog/desc.rs | 3 +- src/connector/src/sink/catalog/mod.rs | 4 +- src/ctl/src/cmd_impl/meta/migration.rs | 4 +- src/frontend/src/utils/with_options.rs | 3 +- .../migration/src/m20240525_090457_secret.rs | 28 +++++++++- src/meta/model_v2/src/lib.rs | 53 ++++++++++++++++++- src/meta/model_v2/src/sink.rs | 2 +- src/meta/model_v2/src/source.rs | 7 ++- src/meta/src/controller/mod.rs | 4 +- src/prost/build.rs | 2 + 11 files changed, 110 insertions(+), 19 deletions(-) diff --git a/proto/catalog.proto b/proto/catalog.proto index 5b4f5ae40ff4..c9eedf93f41b 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -83,8 +83,8 @@ message StreamSourceInfo { // Options specified by user in the FORMAT ENCODE clause. map format_encode_options = 14; - // Handle the source relies on any sceret. The key is the propertity name and the value is the secret id. - map secret_ref = 16; + // Handle the source relies on any sceret. The key is the propertity name and the value is the secret id and type. + map secret_ref = 16; } message Source { @@ -180,8 +180,8 @@ message Sink { // Whether it should use background ddl or block until backfill finishes. CreateType create_type = 24; - // Handle the sink relies on any sceret. The key is the propertity name and the value is the secret id. - map secret_ref = 25; + // Handle the sink relies on any sceret. The key is the propertity name and the value is the secret id and type. + map secret_ref = 25; } message Subscription { @@ -450,3 +450,14 @@ message Secret { uint32 owner = 5; uint32 schema_id = 6; } + +message SecretRef { + enum RefAsType { + UNSPECIFIED = 0; + TEXT = 1; + // AS FILE + FILE = 2; + } + uint32 secret_id = 1; + RefAsType ref_as = 2; +} diff --git a/src/connector/src/sink/catalog/desc.rs b/src/connector/src/sink/catalog/desc.rs index 87ccc7b96caf..380cce6a8ebe 100644 --- a/src/connector/src/sink/catalog/desc.rs +++ b/src/connector/src/sink/catalog/desc.rs @@ -19,6 +19,7 @@ use risingwave_common::catalog::{ ColumnCatalog, ConnectionId, CreateType, DatabaseId, SchemaId, TableId, UserId, }; use risingwave_common::util::sort_util::ColumnOrder; +use risingwave_pb::catalog::PbSecretRef; use risingwave_pb::stream_plan::PbSinkDesc; use super::{SinkCatalog, SinkFormatDesc, SinkId, SinkType}; @@ -83,7 +84,7 @@ impl SinkDesc { owner: UserId, connection_id: Option, dependent_relations: Vec, - secret_ref: BTreeMap, + secret_ref: BTreeMap, ) -> SinkCatalog { SinkCatalog { id: self.id, diff --git a/src/connector/src/sink/catalog/mod.rs b/src/connector/src/sink/catalog/mod.rs index f02eb2cdcf9e..206236970d91 100644 --- a/src/connector/src/sink/catalog/mod.rs +++ b/src/connector/src/sink/catalog/mod.rs @@ -25,7 +25,7 @@ use risingwave_common::catalog::{ use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{ - PbCreateType, PbSink, PbSinkFormatDesc, PbSinkType, PbStreamJobStatus, + PbCreateType, PbSecretRef, PbSink, PbSinkFormatDesc, PbSinkType, PbStreamJobStatus, }; use super::{ @@ -339,7 +339,7 @@ pub struct SinkCatalog { pub create_type: CreateType, /// The secret reference for the sink, mapping from property name to secret id. - pub secret_ref: BTreeMap, + pub secret_ref: BTreeMap, } impl SinkCatalog { diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index a7f839527194..93be066d4e72 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -555,12 +555,12 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an *id = *connection_rewrite.get(id).unwrap(); } for secret_id in s.secret_ref.values_mut() { - *secret_id = *secret_rewrite.get(secret_id).unwrap(); + secret_id.secret_id = *secret_rewrite.get(&secret_id.secret_id).unwrap(); } object_dependencies.extend(s.secret_ref.values().map(|id| { object_dependency::ActiveModel { id: NotSet, - oid: Set(*id as _), + oid: Set(id.secret_id as _), used_by: Set(s.id as _), } })); diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index 01993fd24108..8f372b58b17e 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -19,6 +19,7 @@ use risingwave_connector::source::kafka::private_link::{ insert_privatelink_broker_rewrite_map, CONNECTION_NAME_KEY, PRIVATELINK_ENDPOINT_KEY, }; use risingwave_connector::WithPropertiesExt; +use risingwave_pb::catalog::PbSecretRef; use risingwave_sqlparser::ast::{ CreateConnectionStatement, CreateSinkStatement, CreateSourceStatement, CreateSubscriptionStatement, SqlOption, Statement, Value, @@ -119,7 +120,7 @@ impl WithOptions { pub(crate) fn resolve_secret_in_with_options( _with_options: &mut WithOptions, _session: &SessionImpl, -) -> RwResult> { +) -> RwResult> { // todo: implement the function and take `resolve_privatelink_in_with_option` as reference Ok(BTreeMap::new()) diff --git a/src/meta/model_v2/migration/src/m20240525_090457_secret.rs b/src/meta/model_v2/migration/src/m20240525_090457_secret.rs index f16bfca5ec03..ed23085c6657 100644 --- a/src/meta/model_v2/migration/src/m20240525_090457_secret.rs +++ b/src/meta/model_v2/migration/src/m20240525_090457_secret.rs @@ -37,12 +37,22 @@ impl MigrationTrait for Migration { ) .await?; - // Add a new column to the table + // Add a new column to the `sink` table manager .alter_table( MigrationTable::alter() .table(Sink::Table) - .add_column(ColumnDef::new(Sink::SecretRef).json_binary()) + .add_column(ColumnDef::new(Sink::SecretRef).binary()) + .to_owned(), + ) + .await?; + + // Add a new column to the `source` table + manager + .alter_table( + MigrationTable::alter() + .table(Source::Table) + .add_column(ColumnDef::new(Source::SecretRef).binary()) .to_owned(), ) .await?; @@ -60,6 +70,14 @@ impl MigrationTrait for Migration { .to_owned(), ) .await?; + manager + .alter_table( + MigrationTable::alter() + .table(Source::Table) + .drop_column(Source::SecretRef) + .to_owned(), + ) + .await?; Ok(()) } } @@ -77,3 +95,9 @@ enum Sink { Table, SecretRef, } + +#[derive(DeriveIden)] +enum Source { + Table, + SecretRef, +} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 11c5209bdc56..116cb66cab1d 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -14,7 +14,7 @@ use std::collections::BTreeMap; -use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus}; +use risingwave_pb::catalog::{PbCreateType, PbSecretRef, PbStreamJobStatus}; use risingwave_pb::meta::table_fragments::PbState as PbStreamJobState; use risingwave_pb::stream_plan::PbStreamNode; use sea_orm::entity::prelude::*; @@ -258,6 +258,55 @@ macro_rules! derive_array_from_blob { }; } +macro_rules! derive_btreemap_from_blob { + ($struct_name:ident, $key_type:ty, $value_type:ty, $field_type:ident) => { + #[derive(Clone, PartialEq, Eq, DeriveValueType, serde::Deserialize, serde::Serialize)] + pub struct $struct_name(#[sea_orm] Vec); + + #[derive(Clone, PartialEq, ::prost::Message)] + pub struct $field_type { + #[prost(btree_map = "string, message")] + inner: BTreeMap<$key_type, $value_type>, + } + impl Eq for $field_type {} + + impl $struct_name { + pub fn to_protobuf(&self) -> BTreeMap<$key_type, $value_type> { + let data: $field_type = prost::Message::decode(self.0.as_slice()).unwrap(); + data.inner + } + + fn from_protobuf(val: BTreeMap<$key_type, $value_type>) -> Self { + Self(prost::Message::encode_to_vec(&$field_type { inner: val })) + } + } + + impl From> for $struct_name { + fn from(value: BTreeMap<$key_type, $value_type>) -> Self { + Self::from_protobuf(value) + } + } + + impl std::fmt::Debug for $struct_name { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + self.to_protobuf().fmt(f) + } + } + + impl Default for $struct_name { + fn default() -> Self { + Self(vec![]) + } + } + + impl sea_orm::sea_query::Nullable for $struct_name { + fn null() -> Value { + Value::Bytes(None) + } + } + }; +} + pub(crate) use {derive_array_from_blob, derive_from_blob}; derive_from_json_struct!(I32Array, Vec); @@ -286,7 +335,7 @@ impl From>> for ActorUpstreamActors { } } -derive_from_json_struct!(SecretRef, BTreeMap); +derive_btreemap_from_blob!(SecretRef, String, PbSecretRef, PbSecretRefMap); derive_from_blob!(StreamNode, PbStreamNode); derive_from_blob!(DataType, risingwave_pb::data::PbDataType); diff --git a/src/meta/model_v2/src/sink.rs b/src/meta/model_v2/src/sink.rs index 78d0806f98a5..25d6293b0b12 100644 --- a/src/meta/model_v2/src/sink.rs +++ b/src/meta/model_v2/src/sink.rs @@ -72,7 +72,7 @@ pub struct Model { pub sink_from_name: String, pub sink_format_desc: Option, pub target_table: Option, - // `secret_ref` stores a json string, mapping from property name to secret id. + // `secret_ref` stores the mapping info mapping from property name to secret id and type. pub secret_ref: Option, } diff --git a/src/meta/model_v2/src/source.rs b/src/meta/model_v2/src/source.rs index be2d2f7110ca..a90f399e4b8c 100644 --- a/src/meta/model_v2/src/source.rs +++ b/src/meta/model_v2/src/source.rs @@ -19,8 +19,8 @@ use sea_orm::ActiveValue::Set; use serde::{Deserialize, Serialize}; use crate::{ - ColumnCatalogArray, ConnectionId, I32Array, Property, SourceId, StreamSourceInfo, TableId, - WatermarkDescArray, + ColumnCatalogArray, ConnectionId, I32Array, Property, SecretRef, SourceId, StreamSourceInfo, + TableId, WatermarkDescArray, }; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] @@ -39,6 +39,8 @@ pub struct Model { pub optional_associated_table_id: Option, pub connection_id: Option, pub version: i64, + // `secret_ref` stores the mapping info mapping from property name to secret id and type. + pub secret_ref: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -101,6 +103,7 @@ impl From for ActiveModel { optional_associated_table_id: Set(optional_associated_table_id), connection_id: Set(source.connection_id.map(|id| id as _)), version: Set(source.version as _), + secret_ref: Set(None), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 4fb5d086a060..b6997240f772 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -201,9 +201,9 @@ impl From> for PbSource { impl From> for PbSink { fn from(value: ObjectModel) -> Self { - let mut secret_ref_map: BTreeMap = BTreeMap::new(); + let mut secret_ref_map = BTreeMap::new(); if let Some(secret_ref) = value.0.secret_ref { - secret_ref_map = secret_ref.into_inner(); + secret_ref_map = secret_ref.to_protobuf(); } Self { id: value.0.sink_id as _, diff --git a/src/prost/build.rs b/src/prost/build.rs index 961dbe196944..4e939f46abb6 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -66,6 +66,7 @@ fn main() -> Result<(), Box> { ".plan_common.ExternalTableDesc", ".hummock.CompactTask", ".catalog.StreamSourceInfo", + ".catalog.SecretRef", ".catalog.Source", ".catalog.Sink", ".catalog.View", @@ -111,6 +112,7 @@ fn main() -> Result<(), Box> { // The requirement is from Source node -> SourceCatalog -> WatermarkDesc -> expr .type_attribute("catalog.WatermarkDesc", "#[derive(Eq, Hash)]") .type_attribute("catalog.StreamSourceInfo", "#[derive(Eq, Hash)]") + .type_attribute("catalog.SecretRef", "#[derive(Eq, Hash)]") .type_attribute("catalog.IndexColumnProperties", "#[derive(Eq, Hash)]") .type_attribute("expr.ExprNode", "#[derive(Eq, Hash)]") .type_attribute("data.DataType", "#[derive(Eq, Hash)]") From 955cbb9d2d6e386f499ad494c8156a3a3d68820e Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 18 Jun 2024 08:37:42 +0800 Subject: [PATCH 13/17] perf(sink): don't compact chunk for blackhole sink (#17167) Signed-off-by: xxchan --- src/connector/src/sink/mod.rs | 4 ++++ src/stream/src/executor/sink.rs | 10 ++++++++-- 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index bdd923f786ec..8102f03355e8 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -443,6 +443,10 @@ impl SinkImpl { pub fn is_sink_into_table(&self) -> bool { matches!(self, SinkImpl::Table(_)) } + + pub fn is_blackhole(&self) -> bool { + matches!(self, SinkImpl::BlackHole(_)) + } } pub fn build_sink(param: SinkParam) -> Result { diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index e8dfc09e9130..f7c7770c0bd7 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -183,6 +183,8 @@ impl SinkExecutor { let re_construct_with_sink_pk = need_advance_delete && self.sink_param.sink_type == SinkType::Upsert && !self.sink_param.downstream_pk.is_empty(); + // Don't compact chunk for blackhole sink for better benchmark performance. + let compact_chunk = !self.sink.is_blackhole(); let processed_input = Self::process_msg( input, self.sink_param.sink_type, @@ -193,6 +195,7 @@ impl SinkExecutor { self.input_data_types, self.sink_param.downstream_pk.clone(), metrics.sink_chunk_buffer_size, + compact_chunk, ); if self.sink.is_sink_into_table() { @@ -298,6 +301,7 @@ impl SinkExecutor { input_data_types: Vec, down_stream_pk: Vec, sink_chunk_buffer_size_metrics: LabelGuardedIntGauge<3>, + compact_chunk: bool, ) { // need to buffer chunks during one barrier if need_advance_delete || re_construct_with_sink_pk { @@ -362,10 +366,12 @@ impl SinkExecutor { for msg in input { match msg? { Message::Watermark(w) => yield Message::Watermark(w), - Message::Chunk(chunk) => { + Message::Chunk(mut chunk) => { // Compact the chunk to eliminate any useless intermediate result (e.g. UPDATE // V->V). - let chunk = merge_chunk_row(chunk, &stream_key); + if compact_chunk { + chunk = merge_chunk_row(chunk, &stream_key); + } match sink_type { SinkType::AppendOnly => yield Message::Chunk(chunk), SinkType::ForceAppendOnly => { From 4e5a73125ecb568acac05743806f7234b812f0fe Mon Sep 17 00:00:00 2001 From: ZENOTME <43447882+ZENOTME@users.noreply.github.com> Date: Tue, 18 Jun 2024 10:35:58 +0800 Subject: [PATCH 14/17] chore: add e2e test for iceberg source scan (#17281) Co-authored-by: ZENOTME --- e2e_test/iceberg/main.py | 3 +++ ..._no_partition_append_only_table_verify.slt | 25 +++++++++++++++++++ ...ink_partition_append_only_table_verify.slt | 25 +++++++++++++++++++ ...nge_partition_append_only_table_verify.slt | 25 +++++++++++++++++++ .../test_case/no_partition_append_only.toml | 2 ++ .../test_case/partition_append_only.toml | 2 ++ .../range_partition_append_only.toml | 2 ++ 7 files changed, 84 insertions(+) create mode 100644 e2e_test/iceberg/test_case/iceberg_sink_no_partition_append_only_table_verify.slt create mode 100644 e2e_test/iceberg/test_case/iceberg_sink_partition_append_only_table_verify.slt create mode 100644 e2e_test/iceberg/test_case/iceberg_sink_range_partition_append_only_table_verify.slt diff --git a/e2e_test/iceberg/main.py b/e2e_test/iceberg/main.py index 03066033c6d2..c64ce725f6c3 100644 --- a/e2e_test/iceberg/main.py +++ b/e2e_test/iceberg/main.py @@ -129,6 +129,7 @@ def drop_table(args, drop_sqls): verify_sql = test_case.get("verify_sql") print(f"verify_sql:{verify_sql}") verify_data = test_case.get("verify_data") + verify_slt = test_case.get("verify_slt") cmp_sqls = test_case.get("cmp_sqls") drop_sqls = test_case["drop_sqls"] config = configparser.ConfigParser() @@ -146,6 +147,8 @@ def drop_table(args, drop_sqls): verify_result(config, verify_sql, verify_schema, verify_data) if cmp_sqls is not None and cmp_sqls != "" and len(cmp_sqls) == 2: compare_sql(config, cmp_sqls) + if verify_slt is not None and verify_slt != "": + execute_slt(config, verify_slt) if drop_sqls is not None and drop_sqls != "": drop_table(config, drop_sqls) diff --git a/e2e_test/iceberg/test_case/iceberg_sink_no_partition_append_only_table_verify.slt b/e2e_test/iceberg/test_case/iceberg_sink_no_partition_append_only_table_verify.slt new file mode 100644 index 000000000000..74629053344b --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_sink_no_partition_append_only_table_verify.slt @@ -0,0 +1,25 @@ +statement ok +CREATE SOURCE iceberg_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + catalog.type = 'storage', + warehouse.path = 's3://icebergdata/demo', + database.name = 'demo_db', + table.name = 'no_partition_append_only_table', +); + +query I +SELECT id from iceberg_source ORDER by id; +---- +1 +2 +3 +4 +5 + +statement ok +DROP SOURCE iceberg_source diff --git a/e2e_test/iceberg/test_case/iceberg_sink_partition_append_only_table_verify.slt b/e2e_test/iceberg/test_case/iceberg_sink_partition_append_only_table_verify.slt new file mode 100644 index 000000000000..4e6beb709f92 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_sink_partition_append_only_table_verify.slt @@ -0,0 +1,25 @@ +statement ok +CREATE SOURCE iceberg_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + catalog.type = 'storage', + warehouse.path = 's3://icebergdata/demo', + database.name = 'demo_db', + table.name = 'partition_append_only_table', +); + +query I +SELECT id from iceberg_source ORDER by id; +---- +1 +2 +3 +4 +5 + +statement ok +DROP SOURCE iceberg_source diff --git a/e2e_test/iceberg/test_case/iceberg_sink_range_partition_append_only_table_verify.slt b/e2e_test/iceberg/test_case/iceberg_sink_range_partition_append_only_table_verify.slt new file mode 100644 index 000000000000..9d03d99aada1 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_sink_range_partition_append_only_table_verify.slt @@ -0,0 +1,25 @@ +statement ok +CREATE SOURCE iceberg_source +WITH ( + connector = 'iceberg', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + catalog.type = 'storage', + warehouse.path = 's3://icebergdata/demo', + database.name = 'demo_db', + table.name = 'range_partition_append_only_table', +); + +query I +SELECT id from iceberg_source ORDER by id; +---- +1 +2 +3 +4 +5 + +statement ok +DROP SOURCE iceberg_source diff --git a/e2e_test/iceberg/test_case/no_partition_append_only.toml b/e2e_test/iceberg/test_case/no_partition_append_only.toml index 58f5900586d0..7d2952c50875 100644 --- a/e2e_test/iceberg/test_case/no_partition_append_only.toml +++ b/e2e_test/iceberg/test_case/no_partition_append_only.toml @@ -33,6 +33,8 @@ verify_data = """ 5,5,5000,5.5,5.55,5-5,true,2022-03-15,2022-03-15 05:00:00+00:00,2022-03-15 05:00:00,none """ +verify_slt = 'test_case/iceberg_sink_no_partition_append_only_table_verify.slt' + drop_sqls = [ 'DROP TABLE IF EXISTS demo_db.no_partition_append_only_table', 'DROP SCHEMA IF EXISTS demo_db' diff --git a/e2e_test/iceberg/test_case/partition_append_only.toml b/e2e_test/iceberg/test_case/partition_append_only.toml index 9fd55ee0b004..b36dfb98e369 100644 --- a/e2e_test/iceberg/test_case/partition_append_only.toml +++ b/e2e_test/iceberg/test_case/partition_append_only.toml @@ -35,6 +35,8 @@ verify_data = """ 5,5,5000,5.5,5.55,5-5,true,2022-03-15,2022-03-15 05:00:00+00:00,2022-03-15 05:00:00,none """ +verify_slt = 'test_case/iceberg_sink_partition_append_only_table_verify.slt' + drop_sqls = [ 'DROP TABLE IF EXISTS demo_db.partition_append_only_table', 'DROP SCHEMA IF EXISTS demo_db' diff --git a/e2e_test/iceberg/test_case/range_partition_append_only.toml b/e2e_test/iceberg/test_case/range_partition_append_only.toml index d8c12d744584..d54407f91dc1 100644 --- a/e2e_test/iceberg/test_case/range_partition_append_only.toml +++ b/e2e_test/iceberg/test_case/range_partition_append_only.toml @@ -35,6 +35,8 @@ verify_data = """ 5,5,5000,5.5,5.55,5-5,true,2022-03-15,2022-03-15 05:00:00+00:00,2022-03-15 05:00:00,none """ +verify_slt = 'test_case/iceberg_sink_range_partition_append_only_table_verify.slt' + drop_sqls = [ 'DROP TABLE IF EXISTS demo_db.range_partition_append_only_table', 'DROP SCHEMA IF EXISTS demo_db' From ba80ab6b2767472928b35caeee64dccbbdd56bc2 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 18 Jun 2024 13:34:41 +0800 Subject: [PATCH 15/17] feat(optimizer): grow stack for complicated plan in more places (#17224) Signed-off-by: Bugen Zhao --- src/frontend/src/binder/bind_param.rs | 19 +----- src/frontend/src/expr/expr_rewriter.rs | 60 +++++++++++++------ src/frontend/src/expr/expr_visitor.rs | 53 +++++++++++----- src/frontend/src/expr/mod.rs | 8 ++- .../plan_expr_rewriter/const_eval_rewriter.rs | 19 ++---- .../plan_expr_visitor/expr_counter.rs | 17 +----- src/frontend/src/optimizer/plan_node/mod.rs | 6 +- .../src/optimizer/plan_rewriter/mod.rs | 19 ++++-- .../src/optimizer/plan_visitor/mod.rs | 19 ++++-- 9 files changed, 128 insertions(+), 92 deletions(-) diff --git a/src/frontend/src/binder/bind_param.rs b/src/frontend/src/binder/bind_param.rs index 6c3be04d4ee9..b4bbaf420e0c 100644 --- a/src/frontend/src/binder/bind_param.rs +++ b/src/frontend/src/binder/bind_param.rs @@ -21,7 +21,7 @@ use risingwave_common::types::{Datum, ScalarImpl}; use super::statement::RewriteExprsRecursive; use super::BoundStatement; use crate::error::{ErrorCode, Result}; -use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; +use crate::expr::{default_rewrite_expr, Expr, ExprImpl, ExprRewriter, Literal}; /// Rewrites parameter expressions to literals. pub(crate) struct ParamRewriter { @@ -47,22 +47,7 @@ impl ExprRewriter for ParamRewriter { if self.error.is_some() { return expr; } - match expr { - ExprImpl::InputRef(inner) => self.rewrite_input_ref(*inner), - ExprImpl::Literal(inner) => self.rewrite_literal(*inner), - ExprImpl::FunctionCall(inner) => self.rewrite_function_call(*inner), - ExprImpl::FunctionCallWithLambda(inner) => { - self.rewrite_function_call_with_lambda(*inner) - } - ExprImpl::AggCall(inner) => self.rewrite_agg_call(*inner), - ExprImpl::Subquery(inner) => self.rewrite_subquery(*inner), - ExprImpl::CorrelatedInputRef(inner) => self.rewrite_correlated_input_ref(*inner), - ExprImpl::TableFunction(inner) => self.rewrite_table_function(*inner), - ExprImpl::WindowFunction(inner) => self.rewrite_window_function(*inner), - ExprImpl::UserDefinedFunction(inner) => self.rewrite_user_defined_function(*inner), - ExprImpl::Parameter(inner) => self.rewrite_parameter(*inner), - ExprImpl::Now(inner) => self.rewrite_now(*inner), - } + default_rewrite_expr(self, expr) } fn rewrite_subquery(&mut self, mut subquery: crate::expr::Subquery) -> ExprImpl { diff --git a/src/frontend/src/expr/expr_rewriter.rs b/src/frontend/src/expr/expr_rewriter.rs index 4d5b960d654d..6300f9d5e885 100644 --- a/src/frontend/src/expr/expr_rewriter.rs +++ b/src/frontend/src/expr/expr_rewriter.rs @@ -12,33 +12,59 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::util::recursive::{tracker, Recurse}; + use super::{ AggCall, CorrelatedInputRef, ExprImpl, FunctionCall, FunctionCallWithLambda, InputRef, Literal, - Parameter, Subquery, TableFunction, UserDefinedFunction, WindowFunction, + Parameter, Subquery, TableFunction, UserDefinedFunction, WindowFunction, EXPR_DEPTH_THRESHOLD, + EXPR_TOO_DEEP_NOTICE, }; use crate::expr::Now; +use crate::session::current::notice_to_user; + +/// The default implementation of [`ExprRewriter::rewrite_expr`] that simply dispatches to other +/// methods based on the type of the expression. +/// +/// You can use this function as a helper to reduce boilerplate code when implementing the trait. +// TODO: This is essentially a mimic of `super` pattern from OO languages. Ideally, we should +// adopt the style proposed in https://github.com/risingwavelabs/risingwave/issues/13477. +pub fn default_rewrite_expr( + rewriter: &mut R, + expr: ExprImpl, +) -> ExprImpl { + // TODO: Implementors may choose to not use this function at all, in which case we will fail + // to track the recursion and grow the stack as necessary. The current approach is only a + // best-effort attempt to prevent stack overflow. + tracker!().recurse(|t| { + if t.depth_reaches(EXPR_DEPTH_THRESHOLD) { + notice_to_user(EXPR_TOO_DEEP_NOTICE); + } + + match expr { + ExprImpl::InputRef(inner) => rewriter.rewrite_input_ref(*inner), + ExprImpl::Literal(inner) => rewriter.rewrite_literal(*inner), + ExprImpl::FunctionCall(inner) => rewriter.rewrite_function_call(*inner), + ExprImpl::FunctionCallWithLambda(inner) => { + rewriter.rewrite_function_call_with_lambda(*inner) + } + ExprImpl::AggCall(inner) => rewriter.rewrite_agg_call(*inner), + ExprImpl::Subquery(inner) => rewriter.rewrite_subquery(*inner), + ExprImpl::CorrelatedInputRef(inner) => rewriter.rewrite_correlated_input_ref(*inner), + ExprImpl::TableFunction(inner) => rewriter.rewrite_table_function(*inner), + ExprImpl::WindowFunction(inner) => rewriter.rewrite_window_function(*inner), + ExprImpl::UserDefinedFunction(inner) => rewriter.rewrite_user_defined_function(*inner), + ExprImpl::Parameter(inner) => rewriter.rewrite_parameter(*inner), + ExprImpl::Now(inner) => rewriter.rewrite_now(*inner), + } + }) +} /// By default, `ExprRewriter` simply traverses the expression tree and leaves nodes unchanged. /// Implementations can override a subset of methods and perform transformation on some particular /// types of expression. pub trait ExprRewriter { fn rewrite_expr(&mut self, expr: ExprImpl) -> ExprImpl { - match expr { - ExprImpl::InputRef(inner) => self.rewrite_input_ref(*inner), - ExprImpl::Literal(inner) => self.rewrite_literal(*inner), - ExprImpl::FunctionCall(inner) => self.rewrite_function_call(*inner), - ExprImpl::FunctionCallWithLambda(inner) => { - self.rewrite_function_call_with_lambda(*inner) - } - ExprImpl::AggCall(inner) => self.rewrite_agg_call(*inner), - ExprImpl::Subquery(inner) => self.rewrite_subquery(*inner), - ExprImpl::CorrelatedInputRef(inner) => self.rewrite_correlated_input_ref(*inner), - ExprImpl::TableFunction(inner) => self.rewrite_table_function(*inner), - ExprImpl::WindowFunction(inner) => self.rewrite_window_function(*inner), - ExprImpl::UserDefinedFunction(inner) => self.rewrite_user_defined_function(*inner), - ExprImpl::Parameter(inner) => self.rewrite_parameter(*inner), - ExprImpl::Now(inner) => self.rewrite_now(*inner), - } + default_rewrite_expr(self, expr) } fn rewrite_function_call(&mut self, func_call: FunctionCall) -> ExprImpl { diff --git a/src/frontend/src/expr/expr_visitor.rs b/src/frontend/src/expr/expr_visitor.rs index 4e0484397ab9..64b5c61b565d 100644 --- a/src/frontend/src/expr/expr_visitor.rs +++ b/src/frontend/src/expr/expr_visitor.rs @@ -12,10 +12,48 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::util::recursive::{tracker, Recurse}; + use super::{ AggCall, CorrelatedInputRef, ExprImpl, FunctionCall, FunctionCallWithLambda, InputRef, Literal, Now, Parameter, Subquery, TableFunction, UserDefinedFunction, WindowFunction, + EXPR_DEPTH_THRESHOLD, EXPR_TOO_DEEP_NOTICE, }; +use crate::session::current::notice_to_user; + +/// The default implementation of [`ExprVisitor::visit_expr`] that simply dispatches to other +/// methods based on the type of the expression. +/// +/// You can use this function as a helper to reduce boilerplate code when implementing the trait. +// TODO: This is essentially a mimic of `super` pattern from OO languages. Ideally, we should +// adopt the style proposed in https://github.com/risingwavelabs/risingwave/issues/13477. +pub fn default_visit_expr(visitor: &mut V, expr: &ExprImpl) { + // TODO: Implementors may choose to not use this function at all, in which case we will fail + // to track the recursion and grow the stack as necessary. The current approach is only a + // best-effort attempt to prevent stack overflow. + tracker!().recurse(|t| { + if t.depth_reaches(EXPR_DEPTH_THRESHOLD) { + notice_to_user(EXPR_TOO_DEEP_NOTICE); + } + + match expr { + ExprImpl::InputRef(inner) => visitor.visit_input_ref(inner), + ExprImpl::Literal(inner) => visitor.visit_literal(inner), + ExprImpl::FunctionCall(inner) => visitor.visit_function_call(inner), + ExprImpl::FunctionCallWithLambda(inner) => { + visitor.visit_function_call_with_lambda(inner) + } + ExprImpl::AggCall(inner) => visitor.visit_agg_call(inner), + ExprImpl::Subquery(inner) => visitor.visit_subquery(inner), + ExprImpl::CorrelatedInputRef(inner) => visitor.visit_correlated_input_ref(inner), + ExprImpl::TableFunction(inner) => visitor.visit_table_function(inner), + ExprImpl::WindowFunction(inner) => visitor.visit_window_function(inner), + ExprImpl::UserDefinedFunction(inner) => visitor.visit_user_defined_function(inner), + ExprImpl::Parameter(inner) => visitor.visit_parameter(inner), + ExprImpl::Now(inner) => visitor.visit_now(inner), + } + }) +} /// Traverse an expression tree. /// @@ -27,20 +65,7 @@ use super::{ /// subqueries are not traversed. pub trait ExprVisitor { fn visit_expr(&mut self, expr: &ExprImpl) { - match expr { - ExprImpl::InputRef(inner) => self.visit_input_ref(inner), - ExprImpl::Literal(inner) => self.visit_literal(inner), - ExprImpl::FunctionCall(inner) => self.visit_function_call(inner), - ExprImpl::FunctionCallWithLambda(inner) => self.visit_function_call_with_lambda(inner), - ExprImpl::AggCall(inner) => self.visit_agg_call(inner), - ExprImpl::Subquery(inner) => self.visit_subquery(inner), - ExprImpl::CorrelatedInputRef(inner) => self.visit_correlated_input_ref(inner), - ExprImpl::TableFunction(inner) => self.visit_table_function(inner), - ExprImpl::WindowFunction(inner) => self.visit_window_function(inner), - ExprImpl::UserDefinedFunction(inner) => self.visit_user_defined_function(inner), - ExprImpl::Parameter(inner) => self.visit_parameter(inner), - ExprImpl::Now(inner) => self.visit_now(inner), - } + default_visit_expr(self, expr) } fn visit_function_call(&mut self, func_call: &FunctionCall) { func_call diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 03be40f955d7..d14d99766bcc 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -53,8 +53,8 @@ mod utils; pub use agg_call::AggCall; pub use correlated_input_ref::{CorrelatedId, CorrelatedInputRef, Depth}; pub use expr_mutator::ExprMutator; -pub use expr_rewriter::ExprRewriter; -pub use expr_visitor::ExprVisitor; +pub use expr_rewriter::{default_rewrite_expr, ExprRewriter}; +pub use expr_visitor::{default_visit_expr, ExprVisitor}; pub use function_call::{is_row_function, FunctionCall, FunctionCallDisplay}; pub use function_call_with_lambda::FunctionCallWithLambda; pub use input_ref::{input_ref_to_column_indices, InputRef, InputRefDisplay}; @@ -74,6 +74,10 @@ pub use user_defined_function::UserDefinedFunction; pub use utils::*; pub use window_function::WindowFunction; +const EXPR_DEPTH_THRESHOLD: usize = 30; +const EXPR_TOO_DEEP_NOTICE: &str = "Some expression is too complicated. \ +Consider simplifying or splitting the query if you encounter any issues."; + /// the trait of bound expressions pub trait Expr: Into { /// Get the return type of the expr diff --git a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs index 43b76891f356..0844bdb33a85 100644 --- a/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs +++ b/src/frontend/src/optimizer/plan_expr_rewriter/const_eval_rewriter.rs @@ -13,7 +13,7 @@ // limitations under the License. use crate::error::RwError; -use crate::expr::{Expr, ExprImpl, ExprRewriter, Literal}; +use crate::expr::{default_rewrite_expr, Expr, ExprImpl, ExprRewriter, Literal}; pub(crate) struct ConstEvalRewriter { pub(crate) error: Option, @@ -31,21 +31,10 @@ impl ExprRewriter for ConstEvalRewriter { expr } } + } else if let ExprImpl::Parameter(_) = expr { + unreachable!("Parameter should not appear here. It will be replaced by a literal before this step.") } else { - match expr { - ExprImpl::InputRef(inner) => self.rewrite_input_ref(*inner), - ExprImpl::Literal(inner) => self.rewrite_literal(*inner), - ExprImpl::FunctionCall(inner) => self.rewrite_function_call(*inner), - ExprImpl::FunctionCallWithLambda(inner) => self.rewrite_function_call_with_lambda(*inner), - ExprImpl::AggCall(inner) => self.rewrite_agg_call(*inner), - ExprImpl::Subquery(inner) => self.rewrite_subquery(*inner), - ExprImpl::CorrelatedInputRef(inner) => self.rewrite_correlated_input_ref(*inner), - ExprImpl::TableFunction(inner) => self.rewrite_table_function(*inner), - ExprImpl::WindowFunction(inner) => self.rewrite_window_function(*inner), - ExprImpl::UserDefinedFunction(inner) => self.rewrite_user_defined_function(*inner), - ExprImpl::Parameter(_) => unreachable!("Parameter should not appear here. It will be replaced by a literal before this step."), - ExprImpl::Now(inner) => self.rewrite_now(*inner), - } + default_rewrite_expr(self, expr) } } } diff --git a/src/frontend/src/optimizer/plan_expr_visitor/expr_counter.rs b/src/frontend/src/optimizer/plan_expr_visitor/expr_counter.rs index 68ce5b93b044..b636218338c2 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/expr_counter.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/expr_counter.rs @@ -14,7 +14,7 @@ use std::collections::HashMap; -use crate::expr::{ExprImpl, ExprType, ExprVisitor, FunctionCall}; +use crate::expr::{default_visit_expr, ExprImpl, ExprType, ExprVisitor, FunctionCall}; /// `ExprCounter` is used by `CseRewriter`. #[derive(Default)] @@ -35,20 +35,7 @@ impl ExprVisitor for CseExprCounter { return; } - match expr { - ExprImpl::InputRef(inner) => self.visit_input_ref(inner), - ExprImpl::Literal(inner) => self.visit_literal(inner), - ExprImpl::FunctionCall(inner) => self.visit_function_call(inner), - ExprImpl::FunctionCallWithLambda(inner) => self.visit_function_call_with_lambda(inner), - ExprImpl::AggCall(inner) => self.visit_agg_call(inner), - ExprImpl::Subquery(inner) => self.visit_subquery(inner), - ExprImpl::CorrelatedInputRef(inner) => self.visit_correlated_input_ref(inner), - ExprImpl::TableFunction(inner) => self.visit_table_function(inner), - ExprImpl::WindowFunction(inner) => self.visit_window_function(inner), - ExprImpl::UserDefinedFunction(inner) => self.visit_user_defined_function(inner), - ExprImpl::Parameter(inner) => self.visit_parameter(inner), - ExprImpl::Now(inner) => self.visit_now(inner), - } + default_visit_expr(self, expr); } fn visit_function_call(&mut self, func_call: &FunctionCall) { diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index d7187357e3fa..2567cbf01c6e 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -696,8 +696,10 @@ impl dyn PlanNode { } } -const PLAN_DEPTH_THRESHOLD: usize = 30; -const PLAN_TOO_DEEP_NOTICE: &str = "The plan is too deep. \ +/// Recursion depth threshold for plan node visitor to send notice to user. +pub const PLAN_DEPTH_THRESHOLD: usize = 30; +/// Notice message for plan node visitor to send to user when the depth threshold is reached. +pub const PLAN_TOO_DEEP_NOTICE: &str = "The plan is too deep. \ Consider simplifying or splitting the query if you encounter any issues."; impl dyn PlanNode { diff --git a/src/frontend/src/optimizer/plan_rewriter/mod.rs b/src/frontend/src/optimizer/plan_rewriter/mod.rs index 81c0809bae86..360c61d3121b 100644 --- a/src/frontend/src/optimizer/plan_rewriter/mod.rs +++ b/src/frontend/src/optimizer/plan_rewriter/mod.rs @@ -56,11 +56,20 @@ macro_rules! def_rewriter { pub trait PlanRewriter { paste! { fn rewrite(&mut self, plan: PlanRef) -> PlanRef{ - match plan.node_type() { - $( - PlanNodeType::[<$convention $name>] => self.[](plan.downcast_ref::<[<$convention $name>]>().unwrap()), - )* - } + use risingwave_common::util::recursive::{tracker, Recurse}; + use crate::session::current::notice_to_user; + + tracker!().recurse(|t| { + if t.depth_reaches(PLAN_DEPTH_THRESHOLD) { + notice_to_user(PLAN_TOO_DEEP_NOTICE); + } + + match plan.node_type() { + $( + PlanNodeType::[<$convention $name>] => self.[](plan.downcast_ref::<[<$convention $name>]>().unwrap()), + )* + } + }) } $( diff --git a/src/frontend/src/optimizer/plan_visitor/mod.rs b/src/frontend/src/optimizer/plan_visitor/mod.rs index 6156454fd3e8..63a0484cfdfd 100644 --- a/src/frontend/src/optimizer/plan_visitor/mod.rs +++ b/src/frontend/src/optimizer/plan_visitor/mod.rs @@ -93,11 +93,20 @@ macro_rules! def_visitor { paste! { fn visit(&mut self, plan: PlanRef) -> Self::Result { - match plan.node_type() { - $( - PlanNodeType::[<$convention $name>] => self.[](plan.downcast_ref::<[<$convention $name>]>().unwrap()), - )* - } + use risingwave_common::util::recursive::{tracker, Recurse}; + use crate::session::current::notice_to_user; + + tracker!().recurse(|t| { + if t.depth_reaches(PLAN_DEPTH_THRESHOLD) { + notice_to_user(PLAN_TOO_DEEP_NOTICE); + } + + match plan.node_type() { + $( + PlanNodeType::[<$convention $name>] => self.[](plan.downcast_ref::<[<$convention $name>]>().unwrap()), + )* + } + }) } $( From c1b3f5a46645a1ad0de33279e622c78efa1946c5 Mon Sep 17 00:00:00 2001 From: "Zhanxiang (Patrick) Huang" Date: Tue, 18 Jun 2024 13:49:25 +0800 Subject: [PATCH 16/17] chore: remove the object_store prefix in the object store config naming (#17290) --- src/common/src/config.rs | 160 +++++++++++++++--- src/config/example.toml | 12 +- .../src/object/opendal_engine/fs.rs | 2 +- .../src/object/opendal_engine/opendal_s3.rs | 4 +- src/object_store/src/object/s3.rs | 12 +- .../src/hummock/compactor/compactor_runner.rs | 2 +- 6 files changed, 152 insertions(+), 40 deletions(-) diff --git a/src/common/src/config.rs b/src/common/src/config.rs index d8bb65e90b06..0dc6b48d2d8d 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -1029,8 +1029,12 @@ for_all_params!(define_system_config); /// The subsections `[storage.object_store]`. #[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] pub struct ObjectStoreConfig { - #[serde(default = "default::object_store_config::object_store_set_atomic_write_dir")] - pub object_store_set_atomic_write_dir: bool, + // alias is for backward compatibility + #[serde( + default = "default::object_store_config::set_atomic_write_dir", + alias = "object_store_set_atomic_write_dir" + )] + pub set_atomic_write_dir: bool, /// Retry and timeout configuration /// Description retry strategy driven by exponential back-off @@ -1045,25 +1049,36 @@ pub struct ObjectStoreConfig { impl ObjectStoreConfig { pub fn set_atomic_write_dir(&mut self) { - self.object_store_set_atomic_write_dir = true; + self.set_atomic_write_dir = true; } } /// The subsections `[storage.object_store.s3]`. #[derive(Clone, Debug, Serialize, Deserialize, DefaultFromSerde)] pub struct S3ObjectStoreConfig { - #[serde(default = "default::object_store_config::s3::object_store_keepalive_ms")] - pub object_store_keepalive_ms: Option, - #[serde(default = "default::object_store_config::s3::object_store_recv_buffer_size")] - pub object_store_recv_buffer_size: Option, - #[serde(default = "default::object_store_config::s3::object_store_send_buffer_size")] - pub object_store_send_buffer_size: Option, - #[serde(default = "default::object_store_config::s3::object_store_nodelay")] - pub object_store_nodelay: Option, - /// For backwards compatibility, users should use `S3ObjectStoreDeveloperConfig` instead. + // alias is for backward compatibility + #[serde( + default = "default::object_store_config::s3::keepalive_ms", + alias = "object_store_keepalive_ms" + )] + pub keepalive_ms: Option, + #[serde( + default = "default::object_store_config::s3::recv_buffer_size", + alias = "object_store_recv_buffer_size" + )] + pub recv_buffer_size: Option, + #[serde( + default = "default::object_store_config::s3::send_buffer_size", + alias = "object_store_send_buffer_size" + )] + pub send_buffer_size: Option, #[serde( - default = "default::object_store_config::s3::developer::object_store_retry_unknown_service_error" + default = "default::object_store_config::s3::nodelay", + alias = "object_store_nodelay" )] + pub nodelay: Option, + /// For backwards compatibility, users should use `S3ObjectStoreDeveloperConfig` instead. + #[serde(default = "default::object_store_config::s3::developer::retry_unknown_service_error")] pub retry_unknown_service_error: bool, #[serde(default = "default::object_store_config::s3::identity_resolution_timeout_s")] pub identity_resolution_timeout_s: u64, @@ -1076,15 +1091,17 @@ pub struct S3ObjectStoreConfig { pub struct S3ObjectStoreDeveloperConfig { /// Whether to retry s3 sdk error from which no error metadata is provided. #[serde( - default = "default::object_store_config::s3::developer::object_store_retry_unknown_service_error" + default = "default::object_store_config::s3::developer::retry_unknown_service_error", + alias = "object_store_retry_unknown_service_error" )] - pub object_store_retry_unknown_service_error: bool, + pub retry_unknown_service_error: bool, /// An array of error codes that should be retried. /// e.g. `["SlowDown", "TooManyRequests"]` #[serde( - default = "default::object_store_config::s3::developer::object_store_retryable_service_error_codes" + default = "default::object_store_config::s3::developer::retryable_service_error_codes", + alias = "object_store_retryable_service_error_codes" )] - pub object_store_retryable_service_error_codes: Vec, + pub retryable_service_error_codes: Vec, #[serde(default = "default::object_store_config::s3::developer::use_opendal")] pub use_opendal: bool, @@ -1904,7 +1921,7 @@ pub mod default { const DEFAULT_REQ_BACKOFF_MAX_DELAY_MS: u64 = 10 * 1000; // 10s const DEFAULT_REQ_MAX_RETRY_ATTEMPTS: usize = 3; - pub fn object_store_set_atomic_write_dir() -> bool { + pub fn set_atomic_write_dir() -> bool { false } @@ -1992,19 +2009,19 @@ pub mod default { const DEFAULT_KEEPALIVE_MS: u64 = 600 * 1000; // 10min - pub fn object_store_keepalive_ms() -> Option { + pub fn keepalive_ms() -> Option { Some(DEFAULT_KEEPALIVE_MS) // 10min } - pub fn object_store_recv_buffer_size() -> Option { + pub fn recv_buffer_size() -> Option { Some(1 << 21) // 2m } - pub fn object_store_send_buffer_size() -> Option { + pub fn send_buffer_size() -> Option { None } - pub fn object_store_nodelay() -> Option { + pub fn nodelay() -> Option { Some(true) } @@ -2017,11 +2034,11 @@ pub mod default { const RW_USE_OPENDAL_FOR_S3: &str = "RW_USE_OPENDAL_FOR_S3"; - pub fn object_store_retry_unknown_service_error() -> bool { + pub fn retry_unknown_service_error() -> bool { false } - pub fn object_store_retryable_service_error_codes() -> Vec { + pub fn retryable_service_error_codes() -> Vec { vec!["SlowDown".into(), "TooManyRequests".into()] } @@ -2336,4 +2353,99 @@ mod tests { } } } + + #[test] + fn test_object_store_configs_backward_compatibility() { + // Define configs with the old name and make sure it still works + { + let config: RwConfig = toml::from_str( + r#" + [storage.object_store] + object_store_set_atomic_write_dir = true + + [storage.object_store.s3] + object_store_keepalive_ms = 1 + object_store_send_buffer_size = 1 + object_store_recv_buffer_size = 1 + object_store_nodelay = false + + [storage.object_store.s3.developer] + object_store_retry_unknown_service_error = true + object_store_retryable_service_error_codes = ['dummy'] + + + "#, + ) + .unwrap(); + + assert!(config.storage.object_store.set_atomic_write_dir); + assert_eq!(config.storage.object_store.s3.keepalive_ms, Some(1)); + assert_eq!(config.storage.object_store.s3.send_buffer_size, Some(1)); + assert_eq!(config.storage.object_store.s3.recv_buffer_size, Some(1)); + assert_eq!(config.storage.object_store.s3.nodelay, Some(false)); + assert!( + config + .storage + .object_store + .s3 + .developer + .retry_unknown_service_error + ); + assert_eq!( + config + .storage + .object_store + .s3 + .developer + .retryable_service_error_codes, + vec!["dummy".to_string()] + ); + } + + // Define configs with the new name and make sure it works + { + let config: RwConfig = toml::from_str( + r#" + [storage.object_store] + set_atomic_write_dir = true + + [storage.object_store.s3] + keepalive_ms = 1 + send_buffer_size = 1 + recv_buffer_size = 1 + nodelay = false + + [storage.object_store.s3.developer] + retry_unknown_service_error = true + retryable_service_error_codes = ['dummy'] + + + "#, + ) + .unwrap(); + + assert!(config.storage.object_store.set_atomic_write_dir); + assert_eq!(config.storage.object_store.s3.keepalive_ms, Some(1)); + assert_eq!(config.storage.object_store.s3.send_buffer_size, Some(1)); + assert_eq!(config.storage.object_store.s3.recv_buffer_size, Some(1)); + assert_eq!(config.storage.object_store.s3.nodelay, Some(false)); + assert!( + config + .storage + .object_store + .s3 + .developer + .retry_unknown_service_error + ); + assert_eq!( + config + .storage + .object_store + .s3 + .developer + .retryable_service_error_codes, + vec!["dummy".to_string()] + ); + } + } } diff --git a/src/config/example.toml b/src/config/example.toml index b35590c85059..27bbea13ade1 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -190,7 +190,7 @@ recent_filter_layers = 6 recent_filter_rotate_interval_ms = 10000 [storage.object_store] -object_store_set_atomic_write_dir = false +set_atomic_write_dir = false [storage.object_store.retry] req_backoff_interval_ms = 1000 @@ -214,15 +214,15 @@ list_attempt_timeout_ms = 600000 list_retry_attempts = 3 [storage.object_store.s3] -object_store_keepalive_ms = 600000 -object_store_recv_buffer_size = 2097152 -object_store_nodelay = true +keepalive_ms = 600000 +recv_buffer_size = 2097152 +nodelay = true retry_unknown_service_error = false identity_resolution_timeout_s = 5 [storage.object_store.s3.developer] -object_store_retry_unknown_service_error = false -object_store_retryable_service_error_codes = ["SlowDown", "TooManyRequests"] +retry_unknown_service_error = false +retryable_service_error_codes = ["SlowDown", "TooManyRequests"] use_opendal = false [system] diff --git a/src/object_store/src/object/opendal_engine/fs.rs b/src/object_store/src/object/opendal_engine/fs.rs index 4f2715d6ccfb..ecb1131f0def 100644 --- a/src/object_store/src/object/opendal_engine/fs.rs +++ b/src/object_store/src/object/opendal_engine/fs.rs @@ -29,7 +29,7 @@ impl OpendalObjectStore { // Create fs backend builder. let mut builder = Fs::default(); builder.root(&root); - if config.object_store_set_atomic_write_dir { + if config.set_atomic_write_dir { let atomic_write_dir = format!("{}/{}", root, ATOMIC_WRITE_DIR); builder.atomic_write_dir(&atomic_write_dir); } diff --git a/src/object_store/src/object/opendal_engine/opendal_s3.rs b/src/object_store/src/object/opendal_engine/opendal_s3.rs index 7a51cbb36955..e86a209f4f3f 100644 --- a/src/object_store/src/object/opendal_engine/opendal_s3.rs +++ b/src/object_store/src/object/opendal_engine/opendal_s3.rs @@ -95,11 +95,11 @@ impl OpendalObjectStore { pub fn new_http_client(config: &ObjectStoreConfig) -> ObjectResult { let mut client_builder = reqwest::ClientBuilder::new(); - if let Some(keepalive_ms) = config.s3.object_store_keepalive_ms.as_ref() { + if let Some(keepalive_ms) = config.s3.keepalive_ms.as_ref() { client_builder = client_builder.tcp_keepalive(Duration::from_millis(*keepalive_ms)); } - if let Some(nodelay) = config.s3.object_store_nodelay.as_ref() { + if let Some(nodelay) = config.s3.nodelay.as_ref() { client_builder = client_builder.tcp_nodelay(*nodelay); } diff --git a/src/object_store/src/object/s3.rs b/src/object_store/src/object/s3.rs index f1f569cb7d36..3ed5fc01ba40 100644 --- a/src/object_store/src/object/s3.rs +++ b/src/object_store/src/object/s3.rs @@ -618,19 +618,19 @@ impl S3ObjectStore { let mut http = hyper::client::HttpConnector::new(); // connection config - if let Some(keepalive_ms) = config.s3.object_store_keepalive_ms.as_ref() { + if let Some(keepalive_ms) = config.s3.keepalive_ms.as_ref() { http.set_keepalive(Some(Duration::from_millis(*keepalive_ms))); } - if let Some(nodelay) = config.s3.object_store_nodelay.as_ref() { + if let Some(nodelay) = config.s3.nodelay.as_ref() { http.set_nodelay(*nodelay); } - if let Some(recv_buffer_size) = config.s3.object_store_recv_buffer_size.as_ref() { + if let Some(recv_buffer_size) = config.s3.recv_buffer_size.as_ref() { http.set_recv_buffer_size(Some(*recv_buffer_size)); } - if let Some(send_buffer_size) = config.s3.object_store_send_buffer_size.as_ref() { + if let Some(send_buffer_size) = config.s3.send_buffer_size.as_ref() { http.set_send_buffer_size(Some(*send_buffer_size)); } @@ -1043,7 +1043,7 @@ where Some(SdkError::ServiceError(e)) => { let retry = match e.err().code() { None => { - if config.s3.developer.object_store_retry_unknown_service_error + if config.s3.developer.retry_unknown_service_error || config.s3.retry_unknown_service_error { tracing::warn!(target: "unknown_service_error", "{e:?} occurs, retry S3 get_object request."); @@ -1056,7 +1056,7 @@ where if config .s3 .developer - .object_store_retryable_service_error_codes + .retryable_service_error_codes .iter() .any(|s| s.as_str().eq_ignore_ascii_case(code)) { diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 41a55518158d..0336f6f542fa 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -380,7 +380,7 @@ pub async fn compact( .storage_opts .object_store_config .s3 - .object_store_recv_buffer_size + .recv_buffer_size .unwrap_or(6 * 1024 * 1024) as u64, capacity as u64, ) * compact_task.splits.len() as u64; From ade66c7faede483a85b7213df58cdd84368bf50d Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Tue, 18 Jun 2024 14:02:24 +0800 Subject: [PATCH 17/17] refactor(storage): deserialize output columns only (#17260) --- src/storage/src/row_serde/value_serde.rs | 58 ++++++++++++- .../src/table/batch_table/storage_table.rs | 84 ++++++++++++------- 2 files changed, 109 insertions(+), 33 deletions(-) diff --git a/src/storage/src/row_serde/value_serde.rs b/src/storage/src/row_serde/value_serde.rs index c4d4ef8b808f..3a9156ca16d1 100644 --- a/src/storage/src/row_serde/value_serde.rs +++ b/src/storage/src/row_serde/value_serde.rs @@ -97,7 +97,8 @@ impl ValueRowSerdeNew for ColumnAwareSerde { } } - let column_with_default = table_columns.iter().enumerate().filter_map(|(i, c)| { + let partial_columns = value_indices.iter().map(|idx| &table_columns[*idx]); + let column_with_default = partial_columns.enumerate().filter_map(|(i, c)| { if let Some(GeneratedOrDefaultColumn::DefaultColumn(DefaultColumnDesc { snapshot_value, expr, @@ -305,4 +306,59 @@ mod tests { // drop all columns is now allowed assert!(try_drop_invalid_columns(&row_bytes, &HashSet::new()).is_none()); } + + #[test] + fn test_deserialize_partial_columns() { + let column_ids = vec![ColumnId::new(0), ColumnId::new(1), ColumnId::new(2)]; + let row1 = OwnedRow::new(vec![ + Some(Int16(5)), + Some(Utf8("abc".into())), + Some(Utf8("ABC".into())), + ]); + let serializer = column_aware_row_encoding::Serializer::new(&column_ids); + let row_bytes = serializer.serialize(row1); + + let deserializer = column_aware_row_encoding::Deserializer::new( + &[ColumnId::new(2), ColumnId::new(0)], + Arc::from(vec![DataType::Varchar, DataType::Int16].into_boxed_slice()), + std::iter::empty(), + ); + let decoded = deserializer.deserialize(&row_bytes[..]); + assert_eq!( + decoded.unwrap(), + vec![Some(Utf8("ABC".into())), Some(Int16(5))] + ); + } + + #[test] + fn test_deserialize_partial_columns_with_default_columns() { + let column_ids = vec![ColumnId::new(0), ColumnId::new(1), ColumnId::new(2)]; + let row1 = OwnedRow::new(vec![ + Some(Int16(5)), + Some(Utf8("abc".into())), + Some(Utf8("ABC".into())), + ]); + let serializer = column_aware_row_encoding::Serializer::new(&column_ids); + let row_bytes = serializer.serialize(row1); + + // default column of ColumnId::new(3) + let default_columns = vec![(1, Some(Utf8("new column".into())))]; + + let deserializer = column_aware_row_encoding::Deserializer::new( + &[ColumnId::new(2), ColumnId::new(3), ColumnId::new(0)], + Arc::from( + vec![DataType::Varchar, DataType::Varchar, DataType::Int16].into_boxed_slice(), + ), + default_columns.into_iter(), + ); + let decoded = deserializer.deserialize(&row_bytes[..]); + assert_eq!( + decoded.unwrap(), + vec![ + Some(Utf8("ABC".into())), + Some(Utf8("new column".into())), + Some(Int16(5)) + ] + ); + } } diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 7dc697d84a13..e5f2801b0502 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::ops::Bound::{self, Excluded, Included, Unbounded}; -use std::ops::{Index, RangeBounds}; +use std::ops::RangeBounds; use std::sync::Arc; use auto_enums::auto_enum; @@ -30,6 +30,7 @@ use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, TableId, TableOption}; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{self, OwnedRow, Row, RowExt}; +use risingwave_common::types::ToOwnedDatum; use risingwave_common::util::row_serde::*; use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; @@ -82,7 +83,8 @@ pub struct StorageTableInner { /// Mapping from column id to column index for deserializing the row. mapping: Arc, - /// Row deserializer to deserialize the whole value in storage to a row. + /// Row deserializer to deserialize the value in storage to a row. + /// The row can be either complete or partial, depending on whether the row encoding is versioned. row_serde: Arc, /// Indices of primary key. @@ -247,29 +249,45 @@ impl StorageTableInner { } } - let output_row_in_value_indices = value_output_indices - .iter() - .map(|&di| value_indices.iter().position(|&pi| di == pi).unwrap()) - .collect_vec(); let output_row_in_key_indices = key_output_indices .iter() .map(|&di| pk_indices.iter().position(|&pi| di == pi).unwrap()) .collect_vec(); let schema = Schema::new(output_columns.iter().map(Into::into).collect()); - let mapping = ColumnMapping::new(output_row_in_value_indices); - let pk_data_types = pk_indices .iter() .map(|i| table_columns[*i].data_type.clone()) .collect(); let pk_serializer = OrderedRowSerde::new(pk_data_types, order_types); - - let row_serde = { + let (row_serde, mapping) = { if versioned { - ColumnAwareSerde::new(value_indices.into(), table_columns.into()).into() + let value_output_indices_dedup = value_output_indices + .iter() + .unique() + .copied() + .collect::>(); + let output_row_in_value_output_indices_dedup = value_output_indices + .iter() + .map(|&di| { + value_output_indices_dedup + .iter() + .position(|&pi| di == pi) + .unwrap() + }) + .collect_vec(); + let mapping = ColumnMapping::new(output_row_in_value_output_indices_dedup); + let serde = + ColumnAwareSerde::new(value_output_indices_dedup.into(), table_columns.into()); + (serde.into(), mapping) } else { - BasicSerde::new(value_indices.into(), table_columns.into()).into() + let output_row_in_value_indices = value_output_indices + .iter() + .map(|&di| value_indices.iter().position(|&pi| di == pi).unwrap()) + .collect_vec(); + let mapping = ColumnMapping::new(output_row_in_value_indices); + let serde = BasicSerde::new(value_indices.into(), table_columns.into()); + (serde.into(), mapping) } }; @@ -367,11 +385,10 @@ impl StorageTableInner { if let Some(value) = self.store.get(serialized_pk, epoch, read_options).await? { // Refer to [`StorageTableInnerIterInner::new`] for necessity of `validate_read_epoch`. self.store.validate_read_epoch(wait_epoch)?; - let full_row = self.row_serde.deserialize(&value)?; - let result_row_in_value = self - .mapping - .project(OwnedRow::new(full_row)) - .into_owned_row(); + + let row = self.row_serde.deserialize(&value)?; + let result_row_in_value = self.mapping.project(OwnedRow::new(row)); + match &self.key_output_indices { Some(key_output_indices) => { let result_row_in_key = @@ -386,20 +403,23 @@ impl StorageTableInner { .unwrap(); result_row_vec.push( result_row_in_value - .index(*item_position_in_value_indices) - .clone(), + .datum_at(*item_position_in_value_indices) + .to_owned_datum(), ); } else { let item_position_in_pk_indices = key_output_indices.iter().position(|p| idx == p).unwrap(); - result_row_vec - .push(result_row_in_key.index(item_position_in_pk_indices).clone()); + result_row_vec.push( + result_row_in_key + .datum_at(item_position_in_pk_indices) + .to_owned_datum(), + ); } } let result_row = OwnedRow::new(result_row_vec); Ok(Some(result_row)) } - None => Ok(Some(result_row_in_value)), + None => Ok(Some(result_row_in_value.into_owned_row())), } } else { Ok(None) @@ -875,11 +895,8 @@ impl StorageTableInnerIterInner { .await? { let (table_key, value) = (k.user_key.table_key, v); - let full_row = self.row_deserializer.deserialize(value)?; - let result_row_in_value = self - .mapping - .project(OwnedRow::new(full_row)) - .into_owned_row(); + let row = self.row_deserializer.deserialize(value)?; + let result_row_in_value = self.mapping.project(OwnedRow::new(row)); match &self.key_output_indices { Some(key_output_indices) => { let result_row_in_key = match self.pk_serializer.clone() { @@ -901,14 +918,17 @@ impl StorageTableInnerIterInner { .unwrap(); result_row_vec.push( result_row_in_value - .index(*item_position_in_value_indices) - .clone(), + .datum_at(*item_position_in_value_indices) + .to_owned_datum(), ); } else { let item_position_in_pk_indices = key_output_indices.iter().position(|p| idx == p).unwrap(); - result_row_vec - .push(result_row_in_key.index(item_position_in_pk_indices).clone()); + result_row_vec.push( + result_row_in_key + .datum_at(item_position_in_pk_indices) + .to_owned_datum(), + ); } } let row = OwnedRow::new(result_row_vec); @@ -922,7 +942,7 @@ impl StorageTableInnerIterInner { None => { yield KeyedRow { vnode_prefixed_key: table_key.copy_into(), - row: result_row_in_value, + row: result_row_in_value.into_owned_row(), } } }