From ee6378b5f81716217d2881046cb02bd64394ae64 Mon Sep 17 00:00:00 2001 From: August Date: Mon, 25 Dec 2023 16:46:11 +0800 Subject: [PATCH 01/15] feat: [Part 1] add more functions for sql-based controllers and some bug fix (#14155) --- .../migration/src/m20230908_072257_init.rs | 14 +- src/meta/model_v2/src/function.rs | 10 + src/meta/model_v2/src/index.rs | 18 +- src/meta/model_v2/src/lib.rs | 15 + src/meta/model_v2/src/sink.rs | 40 +- src/meta/model_v2/src/table.rs | 74 +- src/meta/src/controller/catalog.rs | 767 +++++++++++++++++- src/meta/src/controller/cluster.rs | 60 +- src/meta/src/controller/fragment.rs | 268 +++++- src/meta/src/controller/mod.rs | 38 +- src/meta/src/controller/user.rs | 21 +- src/meta/src/controller/utils.rs | 63 +- 12 files changed, 1269 insertions(+), 119 deletions(-) diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index 4a4cb601f2a06..ee853876bc0f9 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -395,6 +395,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Actor::ParallelUnitId).integer().not_null()) .col(ColumnDef::new(Actor::UpstreamActorIds).json()) .col(ColumnDef::new(Actor::VnodeBitmap).json()) + .col(ColumnDef::new(Actor::ExprContext).json().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_actor_fragment_id") @@ -538,7 +539,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Table::StreamKey).json().not_null()) .col(ColumnDef::new(Table::AppendOnly).boolean().not_null()) .col(ColumnDef::new(Table::Properties).json().not_null()) - .col(ColumnDef::new(Table::FragmentId).integer().not_null()) + .col(ColumnDef::new(Table::FragmentId).integer()) .col(ColumnDef::new(Table::VnodeColIndex).integer()) .col(ColumnDef::new(Table::RowIdIndex).integer()) .col(ColumnDef::new(Table::ValueIndices).json().not_null()) @@ -562,10 +563,8 @@ impl MigrationTrait for Migration { .boolean() .not_null(), ) - .col(ColumnDef::new(Table::JobStatus).string().not_null()) - .col(ColumnDef::new(Table::CreateType).string().not_null()) .col(ColumnDef::new(Table::Description).string()) - .col(ColumnDef::new(Table::Version).json().not_null()) + .col(ColumnDef::new(Table::Version).json()) .foreign_key( &mut ForeignKey::create() .name("FK_table_object_id") @@ -623,7 +622,6 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Sink::DbName).string().not_null()) .col(ColumnDef::new(Sink::SinkFromName).string().not_null()) .col(ColumnDef::new(Sink::SinkFormatDesc).json()) - .col(ColumnDef::new(Sink::JobStatus).string().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_sink_object_id") @@ -672,7 +670,6 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Index::PrimaryTableId).integer().not_null()) .col(ColumnDef::new(Index::IndexItems).json().not_null()) .col(ColumnDef::new(Index::OriginalColumns).json().not_null()) - .col(ColumnDef::new(Index::JobStatus).string().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_index_object_id") @@ -969,6 +966,7 @@ enum Actor { ParallelUnitId, UpstreamActorIds, VnodeBitmap, + ExprContext, } #[derive(DeriveIden)] @@ -1021,8 +1019,6 @@ enum Table { DmlFragmentId, Cardinality, CleanedByWatermark, - JobStatus, - CreateType, Description, Version, } @@ -1060,7 +1056,6 @@ enum Sink { DbName, SinkFromName, SinkFormatDesc, - JobStatus, } #[derive(DeriveIden)] @@ -1090,7 +1085,6 @@ enum Index { PrimaryTableId, IndexItems, OriginalColumns, - JobStatus, } #[derive(DeriveIden)] diff --git a/src/meta/model_v2/src/function.rs b/src/meta/model_v2/src/function.rs index c4774b177eabc..2013a58403e62 100644 --- a/src/meta/model_v2/src/function.rs +++ b/src/meta/model_v2/src/function.rs @@ -74,6 +74,16 @@ impl From for FunctionKind { } } +impl From for Kind { + fn from(value: FunctionKind) -> Self { + match value { + FunctionKind::Scalar => Self::Scalar(Default::default()), + FunctionKind::Table => Self::Table(Default::default()), + FunctionKind::Aggregate => Self::Aggregate(Default::default()), + } + } +} + impl From for ActiveModel { fn from(function: PbFunction) -> Self { Self { diff --git a/src/meta/model_v2/src/index.rs b/src/meta/model_v2/src/index.rs index c85a896914240..8b10c58e04674 100644 --- a/src/meta/model_v2/src/index.rs +++ b/src/meta/model_v2/src/index.rs @@ -12,9 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_pb::catalog::PbIndex; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue::Set; -use crate::{ExprNodeArray, I32Array, IndexId, JobStatus, TableId}; +use crate::{ExprNodeArray, I32Array, IndexId, TableId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "index")] @@ -26,7 +28,6 @@ pub struct Model { pub primary_table_id: TableId, pub index_items: ExprNodeArray, pub original_columns: I32Array, - pub job_status: JobStatus, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -64,3 +65,16 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(pb_index: PbIndex) -> Self { + Self { + index_id: Set(pb_index.id as _), + name: Set(pb_index.name), + index_table_id: Set(pb_index.index_table_id as _), + primary_table_id: Set(pb_index.primary_table_id as _), + index_items: Set(pb_index.index_item.into()), + original_columns: Set(pb_index.original_columns.into()), + } + } +} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index afbb24d24204c..ffb994d06f2e3 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -123,12 +123,27 @@ impl From for PbCreateType { } } +impl From for CreateType { + fn from(create_type: PbCreateType) -> Self { + match create_type { + PbCreateType::Background => Self::Background, + PbCreateType::Foreground => Self::Foreground, + PbCreateType::Unspecified => unreachable!("Unspecified create type"), + } + } +} + /// Defines struct with a single pb field that derives `FromJsonQueryResult`, it will helps to map json value stored in database to Pb struct. macro_rules! derive_from_json_struct { ($struct_name:ident, $field_type:ty) => { #[derive(Clone, Debug, PartialEq, FromJsonQueryResult, Serialize, Deserialize, Default)] pub struct $struct_name(pub $field_type); impl Eq for $struct_name {} + impl From<$field_type> for $struct_name { + fn from(value: $field_type) -> Self { + Self(value) + } + } impl $struct_name { pub fn into_inner(self) -> $field_type { diff --git a/src/meta/model_v2/src/sink.rs b/src/meta/model_v2/src/sink.rs index 21ac172246703..7f2f6cc5f9a0b 100644 --- a/src/meta/model_v2/src/sink.rs +++ b/src/meta/model_v2/src/sink.rs @@ -12,12 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_pb::catalog::PbSinkType; +use risingwave_pb::catalog::{PbSink, PbSinkType}; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue::Set; use crate::{ - ColumnCatalogArray, ColumnOrderArray, ConnectionId, I32Array, JobStatus, Property, - SinkFormatDesc, SinkId, + ColumnCatalogArray, ColumnOrderArray, ConnectionId, I32Array, Property, SinkFormatDesc, SinkId, }; #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] @@ -41,6 +41,17 @@ impl From for PbSinkType { } } +impl From for SinkType { + fn from(sink_type: PbSinkType) -> Self { + match sink_type { + PbSinkType::AppendOnly => Self::AppendOnly, + PbSinkType::ForceAppendOnly => Self::ForceAppendOnly, + PbSinkType::Upsert => Self::Upsert, + PbSinkType::Unspecified => unreachable!("Unspecified sink type"), + } + } +} + #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "sink")] pub struct Model { @@ -58,7 +69,6 @@ pub struct Model { pub db_name: String, pub sink_from_name: String, pub sink_format_desc: Option, - pub job_status: JobStatus, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -94,3 +104,25 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(pb_sink: PbSink) -> Self { + let sink_type = pb_sink.sink_type(); + + Self { + sink_id: Set(pb_sink.id as _), + name: Set(pb_sink.name), + columns: Set(pb_sink.columns.into()), + plan_pk: Set(pb_sink.plan_pk.into()), + distribution_key: Set(pb_sink.distribution_key.into()), + downstream_pk: Set(pb_sink.downstream_pk.into()), + sink_type: Set(sink_type.into()), + properties: Set(pb_sink.properties.into()), + definition: Set(pb_sink.definition), + connection_id: Set(pb_sink.connection_id.map(|x| x as _)), + db_name: Set(pb_sink.db_name), + sink_from_name: Set(pb_sink.sink_from_name), + sink_format_desc: Set(pb_sink.format_desc.map(|x| x.into())), + } + } +} diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 9b1538c059af3..1141c6adb48ab 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -13,12 +13,14 @@ // limitations under the License. use risingwave_pb::catalog::table::PbTableType; -use risingwave_pb::catalog::PbHandleConflictBehavior; +use risingwave_pb::catalog::{PbHandleConflictBehavior, PbTable}; use sea_orm::entity::prelude::*; +use sea_orm::ActiveValue::Set; +use sea_orm::NotSet; use crate::{ - Cardinality, ColumnCatalogArray, ColumnOrderArray, CreateType, FragmentId, I32Array, JobStatus, - ObjectId, Property, SourceId, TableId, TableVersion, + Cardinality, ColumnCatalogArray, ColumnOrderArray, FragmentId, I32Array, ObjectId, Property, + SourceId, TableId, TableVersion, }; #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] @@ -45,6 +47,18 @@ impl From for PbTableType { } } +impl From for TableType { + fn from(table_type: PbTableType) -> Self { + match table_type { + PbTableType::Table => Self::Table, + PbTableType::MaterializedView => Self::MaterializedView, + PbTableType::Index => Self::Index, + PbTableType::Internal => Self::Internal, + PbTableType::Unspecified => unreachable!("Unspecified table type"), + } + } +} + #[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum HandleConflictBehavior { @@ -66,6 +80,19 @@ impl From for PbHandleConflictBehavior { } } +impl From for HandleConflictBehavior { + fn from(handle_conflict_behavior: PbHandleConflictBehavior) -> Self { + match handle_conflict_behavior { + PbHandleConflictBehavior::Overwrite => Self::Overwrite, + PbHandleConflictBehavior::Ignore => Self::Ignore, + PbHandleConflictBehavior::NoCheck => Self::NoCheck, + PbHandleConflictBehavior::Unspecified => { + unreachable!("Unspecified handle conflict behavior") + } + } + } +} + #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "table")] pub struct Model { @@ -81,7 +108,7 @@ pub struct Model { pub stream_key: I32Array, pub append_only: bool, pub properties: Property, - pub fragment_id: FragmentId, + pub fragment_id: Option, pub vnode_col_index: Option, pub row_id_index: Option, pub value_indices: I32Array, @@ -93,10 +120,8 @@ pub struct Model { pub dml_fragment_id: Option, pub cardinality: Option, pub cleaned_by_watermark: bool, - pub job_status: JobStatus, - pub create_type: CreateType, pub description: Option, - pub version: TableVersion, + pub version: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -156,3 +181,38 @@ impl Related for Entity { } impl ActiveModelBehavior for ActiveModel {} + +impl From for ActiveModel { + fn from(pb_table: PbTable) -> Self { + let table_type = pb_table.table_type(); + let handle_pk_conflict_behavior = pb_table.handle_pk_conflict_behavior(); + + Self { + table_id: Set(pb_table.id as _), + name: Set(pb_table.name), + optional_associated_source_id: NotSet, + table_type: Set(table_type.into()), + belongs_to_job_id: Set(None), + columns: Set(pb_table.columns.into()), + pk: Set(pb_table.pk.into()), + distribution_key: Set(pb_table.distribution_key.into()), + stream_key: Set(pb_table.stream_key.into()), + append_only: Set(pb_table.append_only), + properties: Set(pb_table.properties.into()), + fragment_id: NotSet, + vnode_col_index: Set(pb_table.vnode_col_index.map(|x| x as i32)), + row_id_index: Set(pb_table.row_id_index.map(|x| x as i32)), + value_indices: Set(pb_table.value_indices.into()), + definition: Set(pb_table.definition), + handle_pk_conflict_behavior: Set(handle_pk_conflict_behavior.into()), + read_prefix_len_hint: Set(pb_table.read_prefix_len_hint as _), + watermark_indices: Set(pb_table.watermark_indices.into()), + dist_key_in_pk: Set(pb_table.dist_key_in_pk.into()), + dml_fragment_id: NotSet, + cardinality: Set(pb_table.cardinality.map(|x| x.into())), + cleaned_by_watermark: Set(pb_table.cleaned_by_watermark), + description: Set(pb_table.description), + version: Set(pb_table.version.map(|v| v.into())), + } + } +} diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 52dbcd3df0426..d724e94af007b 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -12,48 +12,56 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::iter; use std::sync::Arc; use anyhow::anyhow; use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::catalog::{DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; +use risingwave_common::catalog::{TableOption, DEFAULT_SCHEMA_NAME, SYSTEM_SCHEMAS}; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::table::TableType; use risingwave_meta_model_v2::{ - connection, database, function, index, object, object_dependency, schema, sink, source, table, - user_privilege, view, ColumnCatalogArray, ConnectionId, DatabaseId, FunctionId, IndexId, - ObjectId, PrivateLinkService, SchemaId, SourceId, TableId, UserId, + connection, database, function, index, object, object_dependency, schema, sink, source, + streaming_job, table, user_privilege, view, ColumnCatalogArray, ConnectionId, CreateType, + DatabaseId, FunctionId, IndexId, JobStatus, ObjectId, PrivateLinkService, Property, SchemaId, + SourceId, TableId, UserId, }; +use risingwave_pb::catalog::table::PbTableType; use risingwave_pb::catalog::{ PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, }; +use risingwave_pb::meta::cancel_creating_jobs_request::PbCreatingJobInfo; use risingwave_pb::meta::relation::PbRelationInfo; use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Operation as NotificationOperation, }; use risingwave_pb::meta::{PbRelation, PbRelationGroup, PbTableFragments}; -use sea_orm::sea_query::SimpleExpr; +use risingwave_pb::user::PbUserInfo; +use sea_orm::sea_query::{Expr, SimpleExpr}; use sea_orm::ActiveValue::Set; use sea_orm::{ ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, - IntoActiveModel, JoinType, QueryFilter, QuerySelect, RelationTrait, TransactionTrait, Value, + IntoActiveModel, JoinType, PaginatorTrait, QueryFilter, QuerySelect, RelationTrait, + TransactionTrait, Value, }; -use tokio::sync::RwLock; +use tokio::sync::{RwLock, RwLockReadGuard}; use crate::controller::rename::{alter_relation_rename, alter_relation_rename_refs}; use crate::controller::utils::{ check_connection_name_duplicate, check_database_name_duplicate, check_function_signature_duplicate, check_relation_name_duplicate, check_schema_name_duplicate, ensure_object_id, ensure_object_not_refer, ensure_schema_empty, ensure_user_id, - get_referring_objects, get_referring_objects_cascade, list_user_info_by_ids, PartialObject, + get_fragment_mappings, get_referring_objects, get_referring_objects_cascade, + get_user_privilege, list_user_info_by_ids, PartialObject, }; use crate::controller::ObjectModel; -use crate::manager::{MetaSrvEnv, NotificationVersion, StreamingJob, IGNORED_NOTIFICATION_VERSION}; +use crate::manager::{Catalog, MetaSrvEnv, NotificationVersion, IGNORED_NOTIFICATION_VERSION}; use crate::rpc::ddl_controller::DropMode; +use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; pub type CatalogControllerRef = Arc; @@ -67,6 +75,7 @@ pub struct CatalogController { #[derive(Clone, Default)] pub struct ReleaseContext { pub(crate) streaming_jobs: Vec, + pub(crate) state_table_ids: Vec, pub(crate) source_ids: Vec, pub(crate) connections: Vec, } @@ -83,9 +92,15 @@ impl CatalogController { }), }) } + + /// Used in `NotificationService::subscribe`. + /// Need to pay attention to the order of acquiring locks to prevent deadlock problems. + pub async fn get_inner_read_guard(&self) -> RwLockReadGuard<'_, CatalogControllerInner> { + self.inner.read().await + } } -pub(crate) struct CatalogControllerInner { +pub struct CatalogControllerInner { pub(crate) db: DatabaseConnection, } @@ -114,11 +129,7 @@ impl CatalogController { } impl CatalogController { - pub fn snapshot(&self) -> MetaResult<()> { - todo!("snapshot") - } - - async fn create_object( + pub(crate) async fn create_object( txn: &DatabaseTransaction, obj_type: ObjectType, owner_id: UserId, @@ -198,6 +209,18 @@ impl CatalogController { .all(&txn) .await?; + let state_table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .filter( + table::Column::BelongsToJobId + .is_in(streaming_jobs.clone()) + .or(table::Column::TableId.is_in(streaming_jobs.clone())), + ) + .into_tuple() + .all(&txn) + .await?; + let source_ids: Vec = Object::find() .select_only() .column(object::Column::Oid) @@ -256,6 +279,7 @@ impl CatalogController { Ok(( ReleaseContext { streaming_jobs, + state_table_ids, source_ids, connections, }, @@ -355,6 +379,223 @@ impl CatalogController { Ok(version) } + pub async fn list_background_creating_mviews(&self) -> MetaResult> { + let inner = self.inner.read().await; + let tables = Table::find() + .join(JoinType::LeftJoin, table::Relation::Object1.def()) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter( + table::Column::TableType + .eq(TableType::MaterializedView) + .and( + streaming_job::Column::CreateType + .eq(CreateType::Background) + .and(streaming_job::Column::JobStatus.eq(JobStatus::Creating)), + ), + ) + .all(&inner.db) + .await?; + Ok(tables) + } + + pub async fn has_any_streaming_jobs(&self) -> MetaResult { + let inner = self.inner.read().await; + let count = streaming_job::Entity::find().count(&inner.db).await?; + Ok(count > 0) + } + + pub async fn clean_foreground_creating_jobs(&self) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + let creating_job_ids: Vec = streaming_job::Entity::find() + .select_only() + .column(streaming_job::Column::JobId) + .filter( + streaming_job::Column::CreateType + .eq(CreateType::Foreground) + .and(streaming_job::Column::JobStatus.eq(JobStatus::Creating)), + ) + .into_tuple() + .all(&txn) + .await?; + if creating_job_ids.is_empty() { + return Ok(ReleaseContext::default()); + } + + let associated_source_ids: Vec = Table::find() + .select_only() + .column(table::Column::OptionalAssociatedSourceId) + .filter( + table::Column::TableId + .is_in(creating_job_ids.clone()) + .and(table::Column::OptionalAssociatedSourceId.is_not_null()), + ) + .into_tuple() + .all(&txn) + .await?; + + let state_table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .filter( + table::Column::BelongsToJobId + .is_in(creating_job_ids.clone()) + .or(table::Column::TableId.is_in(creating_job_ids.clone())), + ) + .into_tuple() + .all(&txn) + .await?; + + // get all fragment mappings. + let mut fragment_mappings = vec![]; + for job_id in &creating_job_ids { + let mappings = get_fragment_mappings(&txn, *job_id).await?; + fragment_mappings.extend(mappings); + } + + let res = Object::delete_many() + .filter(object::Column::Oid.is_in(creating_job_ids.clone())) + .exec(&txn) + .await?; + assert!(res.rows_affected > 0); + txn.commit().await?; + + // notify delete of fragment mappings. + self.notify_fragment_mapping(NotificationOperation::Delete, fragment_mappings) + .await; + + Ok(ReleaseContext { + streaming_jobs: creating_job_ids, + state_table_ids, + source_ids: associated_source_ids, + ..Default::default() + }) + } + + /// `finish_streaming_job` marks job related objects as `Created` and notify frontend. + pub async fn finish_streaming_job(&self, job_id: ObjectId) -> MetaResult { + let inner = self.inner.write().await; + let txn = inner.db.begin().await?; + + let job_type = Object::find_by_id(job_id) + .select_only() + .column(object::Column::ObjType) + .into_tuple() + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("streaming job", job_id))?; + + // update `created_at` as now(). + let res = Object::update_many() + .col_expr(object::Column::CreatedAt, Expr::current_timestamp().into()) + .filter(object::Column::Oid.eq(job_id)) + .exec(&txn) + .await?; + if res.rows_affected == 0 { + return Err(MetaError::catalog_id_not_found("streaming job", job_id)); + } + + // mark the target stream job as `Created`. + let job = streaming_job::ActiveModel { + job_id: Set(job_id), + job_status: Set(JobStatus::Created), + ..Default::default() + }; + job.update(&txn).await?; + + // notify frontend: job, internal tables. + let internal_table_objs = Table::find() + .find_also_related(Object) + .filter(table::Column::BelongsToJobId.eq(job_id)) + .all(&txn) + .await?; + let mut relations = internal_table_objs + .into_iter() + .map(|(table, obj)| PbRelation { + relation_info: Some(PbRelationInfo::Table( + ObjectModel(table, obj.unwrap()).into(), + )), + }) + .collect_vec(); + + match job_type { + ObjectType::Table => { + let (table, obj) = Table::find_by_id(job_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", job_id))?; + if let Some(source_id) = table.optional_associated_source_id { + let (src, obj) = Source::find_by_id(source_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("source", source_id))?; + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Source( + ObjectModel(src, obj.unwrap()).into(), + )), + }); + } + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Table( + ObjectModel(table, obj.unwrap()).into(), + )), + }); + } + ObjectType::Sink => { + let (sink, obj) = Sink::find_by_id(job_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("sink", job_id))?; + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Sink( + ObjectModel(sink, obj.unwrap()).into(), + )), + }); + } + ObjectType::Index => { + let (index, obj) = Index::find_by_id(job_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("index", job_id))?; + { + let (table, obj) = Table::find_by_id(index.index_table_id) + .find_also_related(Object) + .one(&txn) + .await? + .ok_or_else(|| { + MetaError::catalog_id_not_found("table", index.index_table_id) + })?; + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Table( + ObjectModel(table, obj.unwrap()).into(), + )), + }); + } + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Index( + ObjectModel(index, obj.unwrap()).into(), + )), + }); + } + _ => unreachable!("invalid job type: {:?}", job_type), + } + + txn.commit().await?; + + let version = self + .notify_frontend( + NotificationOperation::Add, + NotificationInfo::RelationGroup(PbRelationGroup { relations }), + ) + .await; + + Ok(version) + } + pub fn create_stream_job( &self, _stream_job: &StreamingJob, @@ -364,7 +605,11 @@ impl CatalogController { todo!() } - pub async fn create_source(&self, mut pb_source: PbSource) -> MetaResult { + pub async fn create_source( + &self, + mut pb_source: PbSource, + source_manager_ref: Option, + ) -> MetaResult { let inner = self.inner.write().await; let owner_id = pb_source.owner as _; let txn = inner.db.begin().await?; @@ -390,6 +635,14 @@ impl CatalogController { pb_source.id = source_obj.oid as _; let source: source::ActiveModel = pb_source.clone().into(); source.insert(&txn).await?; + + if let Some(src_manager) = source_manager_ref { + let ret = src_manager.register_source(&pb_source).await; + if let Err(e) = ret { + txn.rollback().await?; + return Err(e); + } + } txn.commit().await?; let version = self @@ -529,10 +782,11 @@ impl CatalogController { pub async fn drop_connection( &self, connection_id: ConnectionId, - ) -> MetaResult { + ) -> MetaResult<(NotificationVersion, PbConnection)> { let inner = self.inner.write().await; let txn = inner.db.begin().await?; - let connection_obj = Object::find_by_id(connection_id) + let (conn, conn_obj) = Connection::find_by_id(connection_id) + .find_also_related(Object) .one(&txn) .await? .ok_or_else(|| MetaError::catalog_id_not_found("connection", connection_id))?; @@ -556,19 +810,16 @@ impl CatalogController { txn.commit().await?; + let pb_connection: PbConnection = ObjectModel(conn, conn_obj.unwrap()).into(); + self.notify_users_update(user_infos).await; let version = self .notify_frontend( NotificationOperation::Delete, - NotificationInfo::Connection(PbConnection { - id: connection_id as _, - schema_id: connection_obj.schema_id.unwrap() as _, - database_id: connection_obj.database_id.unwrap() as _, - ..Default::default() - }), + NotificationInfo::Connection(pb_connection.clone()), ) .await; - Ok(version) + Ok((version, pb_connection)) } pub async fn create_view(&self, mut pb_view: PbView) -> MetaResult { @@ -932,6 +1183,7 @@ impl CatalogController { .filter(|obj| obj.obj_type == ObjectType::Table || obj.obj_type == ObjectType::Sink) .map(|obj| obj.oid) .collect_vec(); + let mut to_drop_state_table_ids = to_drop_table_ids.clone().collect_vec(); // todo: record index dependency info in the object dependency table. let to_drop_index_ids = to_drop_objects .iter() @@ -986,6 +1238,7 @@ impl CatalogController { .all(&txn) .await?; + to_drop_state_table_ids.extend(to_drop_internal_table_objs.iter().map(|obj| obj.oid)); to_drop_objects.extend(to_drop_internal_table_objs); } @@ -1072,6 +1325,7 @@ impl CatalogController { Ok(( ReleaseContext { streaming_jobs: to_drop_streaming_jobs, + state_table_ids: to_drop_state_table_ids, source_ids: to_drop_source_ids, connections: vec![], }, @@ -1316,6 +1570,467 @@ impl CatalogController { .await; Ok(version) } + + pub async fn list_databases(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_databases().await + } + + pub async fn list_all_state_tables(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_all_state_tables().await + } + + pub async fn list_all_state_table_ids(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_all_state_table_ids().await + } + + pub async fn list_readonly_table_ids(&self, schema_id: SchemaId) -> MetaResult> { + let inner = self.inner.read().await; + let table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .join(JoinType::InnerJoin, table::Relation::Object1.def()) + .filter( + object::Column::SchemaId + .eq(schema_id) + .and(table::Column::TableType.ne(TableType::Table)), + ) + .into_tuple() + .all(&inner.db) + .await?; + Ok(table_ids) + } + + pub async fn list_dml_table_ids(&self, schema_id: SchemaId) -> MetaResult> { + let inner = self.inner.read().await; + let table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .join(JoinType::InnerJoin, table::Relation::Object1.def()) + .filter( + object::Column::SchemaId + .eq(schema_id) + .and(table::Column::TableType.eq(TableType::Table)), + ) + .into_tuple() + .all(&inner.db) + .await?; + Ok(table_ids) + } + + pub async fn list_tables_by_type(&self, table_type: TableType) -> MetaResult> { + let inner = self.inner.read().await; + let table_objs = Table::find() + .find_also_related(Object) + .filter(table::Column::TableType.eq(table_type)) + .all(&inner.db) + .await?; + Ok(table_objs + .into_iter() + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + + pub async fn list_sources(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_sources().await + } + + pub async fn list_source_ids(&self, schema_id: SchemaId) -> MetaResult> { + let inner = self.inner.read().await; + let source_ids: Vec = Source::find() + .select_only() + .column(source::Column::SourceId) + .join(JoinType::InnerJoin, source::Relation::Object.def()) + .filter(object::Column::SchemaId.eq(schema_id)) + .into_tuple() + .all(&inner.db) + .await?; + Ok(source_ids) + } + + pub async fn list_sinks(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_sinks().await + } + + pub async fn list_views(&self) -> MetaResult> { + let inner = self.inner.read().await; + inner.list_views().await + } + + pub async fn get_table_by_name( + &self, + database_name: &str, + table_name: &str, + ) -> MetaResult> { + let inner = self.inner.read().await; + let table_obj = Table::find() + .find_also_related(Object) + .join(JoinType::InnerJoin, object::Relation::Database.def()) + .filter( + table::Column::Name + .eq(table_name) + .and(database::Column::Name.eq(database_name)), + ) + .one(&inner.db) + .await?; + Ok(table_obj.map(|(table, obj)| ObjectModel(table, obj.unwrap()).into())) + } + + pub async fn get_table_by_ids(&self, table_ids: Vec) -> MetaResult> { + let inner = self.inner.read().await; + let table_objs = Table::find() + .find_also_related(Object) + .filter(table::Column::TableId.is_in(table_ids)) + .all(&inner.db) + .await?; + Ok(table_objs + .into_iter() + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + + pub async fn find_creating_streaming_job_ids( + &self, + infos: Vec, + ) -> MetaResult> { + let inner = self.inner.read().await; + + type JobKey = (DatabaseId, SchemaId, String); + + // Index table is already included if we still assign the same name for index table as the index. + let creating_tables: Vec<(ObjectId, String, DatabaseId, SchemaId)> = Table::find() + .select_only() + .columns([table::Column::TableId, table::Column::Name]) + .columns([object::Column::DatabaseId, object::Column::SchemaId]) + .join(JoinType::InnerJoin, table::Relation::Object1.def()) + .join(JoinType::InnerJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Creating)) + .into_tuple() + .all(&inner.db) + .await?; + let creating_sinks: Vec<(ObjectId, String, DatabaseId, SchemaId)> = Sink::find() + .select_only() + .columns([sink::Column::SinkId, sink::Column::Name]) + .columns([object::Column::DatabaseId, object::Column::SchemaId]) + .join(JoinType::InnerJoin, sink::Relation::Object.def()) + .join(JoinType::InnerJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Creating)) + .into_tuple() + .all(&inner.db) + .await?; + + let mut job_mapping: HashMap = creating_tables + .into_iter() + .chain(creating_sinks.into_iter()) + .map(|(id, name, database_id, schema_id)| ((database_id, schema_id, name), id)) + .collect(); + + Ok(infos + .into_iter() + .flat_map(|info| { + job_mapping.remove(&( + info.database_id as _, + info.schema_id as _, + info.name.clone(), + )) + }) + .collect()) + } + + pub async fn list_connections(&self) -> MetaResult> { + let inner = self.inner.read().await; + let conn_objs = Connection::find() + .find_also_related(Object) + .all(&inner.db) + .await?; + Ok(conn_objs + .into_iter() + .map(|(conn, obj)| ObjectModel(conn, obj.unwrap()).into()) + .collect()) + } + + pub async fn get_all_table_options(&self) -> MetaResult> { + let inner = self.inner.read().await; + let table_options: Vec<(TableId, Property)> = Table::find() + .select_only() + .columns([table::Column::TableId, table::Column::Properties]) + .into_tuple::<(TableId, Property)>() + .all(&inner.db) + .await?; + + Ok(table_options + .into_iter() + .map(|(id, property)| (id, TableOption::build_table_option(&property.into_inner()))) + .collect()) + } + + pub async fn get_table_name_type_mapping( + &self, + ) -> MetaResult> { + let inner = self.inner.read().await; + let table_name_types: Vec<(TableId, String, TableType)> = Table::find() + .select_only() + .columns([ + table::Column::TableId, + table::Column::Name, + table::Column::TableType, + ]) + .into_tuple() + .all(&inner.db) + .await?; + Ok(table_name_types + .into_iter() + .map(|(id, name, table_type)| { + ( + id, + ( + name, + PbTableType::from(table_type).as_str_name().to_string(), + ), + ) + }) + .collect()) + } + + pub async fn get_created_table_ids(&self) -> MetaResult> { + let inner = self.inner.read().await; + + // created table ids. + let mut table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .join(JoinType::LeftJoin, table::Relation::Object1.def()) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) + .into_tuple() + .all(&inner.db) + .await?; + + // internal table ids. + let internal_table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .filter(table::Column::BelongsToJobId.is_in(table_ids.clone())) + .into_tuple() + .all(&inner.db) + .await?; + table_ids.extend(internal_table_ids); + + Ok(table_ids) + } +} + +impl CatalogControllerInner { + pub async fn snapshot(&self) -> MetaResult<(Catalog, Vec)> { + let databases = self.list_databases().await?; + let schemas = self.list_schemas().await?; + let tables = self.list_tables().await?; + let sources = self.list_sources().await?; + let sinks = self.list_sinks().await?; + let indexes = self.list_indexes().await?; + let views = self.list_views().await?; + let functions = self.list_functions().await?; + let connections = self.list_connections().await?; + + let users = self.list_users().await?; + + Ok(( + ( + databases, + schemas, + tables, + sources, + sinks, + indexes, + views, + functions, + connections, + ), + users, + )) + } + + async fn list_databases(&self) -> MetaResult> { + let db_objs = Database::find() + .find_also_related(Object) + .all(&self.db) + .await?; + Ok(db_objs + .into_iter() + .map(|(db, obj)| ObjectModel(db, obj.unwrap()).into()) + .collect()) + } + + async fn list_schemas(&self) -> MetaResult> { + let schema_objs = Schema::find() + .find_also_related(Object) + .all(&self.db) + .await?; + + Ok(schema_objs + .into_iter() + .map(|(schema, obj)| ObjectModel(schema, obj.unwrap()).into()) + .collect()) + } + + async fn list_users(&self) -> MetaResult> { + let mut user_infos: Vec = User::find() + .all(&self.db) + .await? + .into_iter() + .map(Into::into) + .collect(); + + for user_info in &mut user_infos { + user_info.grant_privileges = get_user_privilege(user_info.id as _, &self.db).await?; + } + Ok(user_infos) + } + + /// `list_all_tables` return all tables and internal tables. + pub async fn list_all_state_tables(&self) -> MetaResult> { + let table_objs = Table::find() + .find_also_related(Object) + .all(&self.db) + .await?; + + Ok(table_objs + .into_iter() + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + + /// `list_all_tables` return all ids of state tables. + pub async fn list_all_state_table_ids(&self) -> MetaResult> { + let table_ids: Vec = Table::find() + .select_only() + .column(table::Column::TableId) + .into_tuple() + .all(&self.db) + .await?; + Ok(table_ids) + } + + /// `list_tables` return all `CREATED` tables and internal tables that belong to `CREATED` streaming jobs. + async fn list_tables(&self) -> MetaResult> { + let table_objs = Table::find() + .find_also_related(Object) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) + .all(&self.db) + .await?; + + let created_streaming_job_ids: Vec = StreamingJob::find() + .select_only() + .column(streaming_job::Column::JobId) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) + .into_tuple() + .all(&self.db) + .await?; + + let internal_table_objs = Table::find() + .find_also_related(Object) + .filter( + table::Column::TableType + .eq(TableType::Internal) + .and(table::Column::BelongsToJobId.is_in(created_streaming_job_ids)), + ) + .all(&self.db) + .await?; + + Ok(table_objs + .into_iter() + .chain(internal_table_objs.into_iter()) + .map(|(table, obj)| ObjectModel(table, obj.unwrap()).into()) + .collect()) + } + + /// `list_sources` return all sources and `CREATED` ones if contains any streaming jobs. + async fn list_sources(&self) -> MetaResult> { + let source_objs = Source::find() + .find_also_related(Object) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter( + streaming_job::Column::JobStatus + .is_null() + .or(streaming_job::Column::JobStatus.eq(JobStatus::Created)), + ) + .all(&self.db) + .await?; + // TODO: filter out inner connector source that are still under creating. + + Ok(source_objs + .into_iter() + .map(|(source, obj)| ObjectModel(source, obj.unwrap()).into()) + .collect()) + } + + /// `list_sinks` return all `CREATED` sinks. + async fn list_sinks(&self) -> MetaResult> { + let sink_objs = Sink::find() + .find_also_related(Object) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) + .all(&self.db) + .await?; + + Ok(sink_objs + .into_iter() + .map(|(sink, obj)| ObjectModel(sink, obj.unwrap()).into()) + .collect()) + } + + async fn list_views(&self) -> MetaResult> { + let view_objs = View::find().find_also_related(Object).all(&self.db).await?; + + Ok(view_objs + .into_iter() + .map(|(view, obj)| ObjectModel(view, obj.unwrap()).into()) + .collect()) + } + + /// `list_indexes` return all `CREATED` indexes. + async fn list_indexes(&self) -> MetaResult> { + let index_objs = Index::find() + .find_also_related(Object) + .join(JoinType::LeftJoin, object::Relation::StreamingJob.def()) + .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) + .all(&self.db) + .await?; + + Ok(index_objs + .into_iter() + .map(|(index, obj)| ObjectModel(index, obj.unwrap()).into()) + .collect()) + } + + async fn list_connections(&self) -> MetaResult> { + let conn_objs = Connection::find() + .find_also_related(Object) + .all(&self.db) + .await?; + + Ok(conn_objs + .into_iter() + .map(|(conn, obj)| ObjectModel(conn, obj.unwrap()).into()) + .collect()) + } + + async fn list_functions(&self) -> MetaResult> { + let func_objs = Function::find() + .find_also_related(Object) + .all(&self.db) + .await?; + + Ok(func_objs + .into_iter() + .map(|(func, obj)| ObjectModel(func, obj.unwrap()).into()) + .collect()) + } } #[cfg(test)] @@ -1482,7 +2197,7 @@ mod tests { .to_string(), ..Default::default() }; - mgr.create_source(pb_source).await?; + mgr.create_source(pb_source, None).await?; let source_id: SourceId = Source::find() .select_only() .column(source::Column::SourceId) diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 0a2b3ec616669..ccff7fd93f26e 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -20,7 +20,6 @@ use std::sync::Arc; use std::time::{Duration, SystemTime}; use itertools::Itertools; -use risingwave_common::hash::ParallelUnitId; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; @@ -47,7 +46,9 @@ use tokio::sync::oneshot::Sender; use tokio::sync::{RwLock, RwLockReadGuard}; use tokio::task::JoinHandle; -use crate::manager::{LocalNotification, MetaSrvEnv, WorkerKey, META_NODE_ID}; +use crate::manager::{ + LocalNotification, MetaSrvEnv, StreamingClusterInfo, WorkerKey, META_NODE_ID, +}; use crate::{MetaError, MetaResult}; pub type ClusterControllerRef = Arc; @@ -118,11 +119,11 @@ impl ClusterController { /// Used in `NotificationService::subscribe`. /// Need to pay attention to the order of acquiring locks to prevent deadlock problems. - pub async fn get_inner_guard(&self) -> RwLockReadGuard<'_, ClusterControllerInner> { + pub async fn get_inner_read_guard(&self) -> RwLockReadGuard<'_, ClusterControllerInner> { self.inner.read().await } - pub async fn count_worker_by_type(&self) -> MetaResult> { + pub async fn count_worker_by_type(&self) -> MetaResult> { self.inner.read().await.count_worker_by_type().await } @@ -208,7 +209,7 @@ impl ClusterController { &self, worker_id: WorkerId, info: Vec, - ) { + ) -> MetaResult<()> { tracing::trace!(target: "events::meta::server_heartbeat", worker_id = worker_id, "receive heartbeat"); self.inner .write() @@ -217,7 +218,7 @@ impl ClusterController { } pub fn start_heartbeat_checker( - cluster_controller: ClusterController, + cluster_controller: ClusterControllerRef, check_interval: Duration, ) -> (JoinHandle<()>, Sender<()>) { let (shutdown_tx, mut shutdown_rx) = tokio::sync::oneshot::channel(); @@ -260,6 +261,7 @@ impl ClusterController { .column(worker::Column::WorkerType) .column(worker::Column::Host) .column(worker::Column::Port) + .filter(worker::Column::WorkerId.is_in(worker_to_delete.clone())) .into_tuple::<(WorkerType, String, i32)>() .all(&inner.db) .await @@ -376,9 +378,9 @@ pub struct WorkerExtraInfo { expire_at: Option, started_at: Option, // Monotonic increasing id since meta node bootstrap. - info_version_id: u64, + pub(crate) info_version_id: u64, // GC watermark. - hummock_gc_watermark: Option, + pub(crate) hummock_gc_watermark: Option, resource: Option, } @@ -411,6 +413,19 @@ impl WorkerExtraInfo { } } +// TODO: remove this when we deprecate model v1. +impl From for WorkerExtraInfo { + fn from(worker: crate::model::Worker) -> Self { + Self { + expire_at: Some(worker.expire_at), + started_at: worker.started_at, + info_version_id: worker.info_version_id, + hummock_gc_watermark: worker.hummock_gc_watermark, + resource: worker.resource, + } + } +} + fn timestamp_now_sec() -> u64 { SystemTime::now() .duration_since(SystemTime::UNIX_EPOCH) @@ -439,19 +454,6 @@ fn meta_node_info(host: &str, started_at: Option) -> PbWorkerNode { } } -/// The cluster info used for scheduling a streaming job. -#[derive(Debug, Clone)] -pub struct StreamingClusterInfo { - /// All **active** compute nodes in the cluster. - pub worker_nodes: HashMap, - - /// All parallel units of the **active** compute nodes in the cluster. - pub parallel_units: HashMap, - - /// All unschedulable parallel units of compute nodes in the cluster. - pub unschedulable_parallel_units: HashMap, -} - pub struct ClusterControllerInner { db: DatabaseConnection, /// Record for tracking available machine ids, one is available. @@ -492,8 +494,8 @@ impl ClusterControllerInner { }) } - pub async fn count_worker_by_type(&self) -> MetaResult> { - let workers: Vec<(WorkerType, i32)> = Worker::find() + pub async fn count_worker_by_type(&self) -> MetaResult> { + let workers: Vec<(WorkerType, i64)> = Worker::find() .select_only() .column(worker::Column::WorkerType) .column_as(worker::Column::WorkerId.count(), "count") @@ -658,8 +660,8 @@ impl ClusterControllerInner { add_property.worker_node_parallelism as _, ))), is_streaming: Set(add_property.is_streaming), - is_serving: Set(add_property.is_streaming), - is_unschedulable: Set(add_property.is_streaming), + is_serving: Set(add_property.is_serving), + is_unschedulable: Set(add_property.is_unschedulable), }; WorkerProperty::insert(property).exec(&txn).await?; } @@ -744,10 +746,16 @@ impl ClusterControllerInner { worker_id: WorkerId, ttl: Duration, info: Vec, - ) { + ) -> MetaResult<()> { if let Some(worker_info) = self.worker_extra_info.get_mut(&worker_id) { worker_info.update_ttl(ttl); worker_info.update_hummock_info(info); + Ok(()) + } else { + Err(MetaError::invalid_worker( + worker_id as u32, + "worker not found", + )) } } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 27ebd7f6e7d97..d2748b3ced1c9 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -20,17 +20,23 @@ use itertools::Itertools; use risingwave_common::bail; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; -use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, StreamingJob}; +use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; use risingwave_meta_model_v2::{ - actor, actor_dispatcher, fragment, ActorId, ConnectorSplits, ExprContext, FragmentId, - FragmentVnodeMapping, I32Array, ObjectId, StreamNode, TableId, VnodeBitmap, WorkerId, + actor, actor_dispatcher, fragment, sink, streaming_job, ActorId, ConnectorSplits, ExprContext, + FragmentId, FragmentVnodeMapping, I32Array, JobStatus, ObjectId, SinkId, StreamNode, TableId, + VnodeBitmap, WorkerId, }; use risingwave_pb::common::PbParallelUnit; use risingwave_pb::ddl_service::PbTableJobType; +use risingwave_pb::meta::subscribe_response::{ + Info as NotificationInfo, Operation as NotificationOperation, +}; use risingwave_pb::meta::table_fragments::actor_status::PbActorState; use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use risingwave_pb::meta::table_fragments::{PbActorStatus, PbFragment, PbState}; -use risingwave_pb::meta::{FragmentParallelUnitMapping, PbTableFragments}; +use risingwave_pb::meta::{ + FragmentParallelUnitMapping, PbFragmentParallelUnitMapping, PbTableFragments, +}; use risingwave_pb::source::PbConnectorSplits; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ @@ -39,13 +45,16 @@ use risingwave_pb::stream_plan::{ use sea_orm::sea_query::{Expr, Value}; use sea_orm::ActiveValue::Set; use sea_orm::{ - ColumnTrait, EntityTrait, JoinType, ModelTrait, PaginatorTrait, QueryFilter, QuerySelect, - RelationTrait, TransactionTrait, + ActiveModelTrait, ColumnTrait, EntityTrait, JoinType, ModelTrait, PaginatorTrait, QueryFilter, + QuerySelect, RelationTrait, TransactionTrait, }; use crate::controller::catalog::{CatalogController, CatalogControllerInner}; -use crate::controller::utils::{get_actor_dispatchers, get_parallel_unit_mapping}; -use crate::manager::ActorInfos; +use crate::controller::utils::{ + get_actor_dispatchers, get_parallel_unit_mapping, FragmentDesc, PartialActorLocation, + PartialFragmentStateTables, +}; +use crate::manager::{ActorInfos, LocalNotification}; use crate::stream::SplitAssignment; use crate::MetaResult; @@ -70,6 +79,50 @@ impl CatalogControllerInner { } impl CatalogController { + pub(crate) async fn notify_fragment_mapping( + &self, + operation: NotificationOperation, + fragment_mappings: Vec, + ) { + let fragment_ids = fragment_mappings + .iter() + .map(|mapping| mapping.fragment_id) + .collect_vec(); + // notify all fragment mappings to frontend. + for fragment_mapping in fragment_mappings { + self.env + .notification_manager() + .notify_frontend( + operation, + NotificationInfo::ParallelUnitMapping(fragment_mapping), + ) + .await; + } + + // update serving vnode mappings. + match operation { + NotificationOperation::Add | NotificationOperation::Update => { + self.env + .notification_manager() + .notify_local_subscribers(LocalNotification::FragmentMappingsUpsert( + fragment_ids, + )) + .await; + } + NotificationOperation::Delete => { + self.env + .notification_manager() + .notify_local_subscribers(LocalNotification::FragmentMappingsDelete( + fragment_ids, + )) + .await; + } + op => { + tracing::warn!("unexpected fragment mapping op: {}", op.as_str_name()); + } + } + } + #[allow(clippy::type_complexity)] pub fn extract_fragment_and_actors_from_table_fragments( PbTableFragments { @@ -255,7 +308,7 @@ impl CatalogController { Vec, HashMap>, )>, - parallel_units_map: HashMap, + parallel_units_map: &HashMap, ) -> MetaResult { let mut pb_fragments = HashMap::new(); let mut pb_actor_splits = HashMap::new(); @@ -263,7 +316,7 @@ impl CatalogController { for (fragment, actors, actor_dispatcher) in fragments { let (fragment, fragment_actor_status, fragment_actor_splits) = - Self::compose_fragment(fragment, actors, actor_dispatcher, ¶llel_units_map)?; + Self::compose_fragment(fragment, actors, actor_dispatcher, parallel_units_map)?; pb_fragments.insert(fragment.fragment_id, fragment); @@ -277,7 +330,7 @@ impl CatalogController { fragments: pb_fragments, actor_status: pb_actor_status, actor_splits: pb_actor_splits, - ctx, + ctx: Some(ctx.unwrap_or_default()), }; Ok(table_fragments) @@ -341,7 +394,7 @@ impl CatalogController { visit_stream_node(&mut nodes, |body| { if let NodeBody::Merge(m) = body && let Some(upstream_actor_ids) = - upstream_fragment_actors.get(&(m.upstream_fragment_id as _)) + upstream_fragment_actors.get(&(m.upstream_fragment_id as _)) { m.upstream_actor_id = upstream_actor_ids.iter().map(|id| *id as _).collect(); @@ -556,10 +609,24 @@ impl CatalogController { job_info.job_status.into(), job_info.timezone.map(|tz| PbStreamContext { timezone: tz }), fragment_info, - parallel_units_map, + ¶llel_units_map, ) } + pub async fn list_streaming_job_states(&self) -> MetaResult> { + let inner = self.inner.read().await; + let job_states: Vec<(ObjectId, JobStatus)> = StreamingJob::find() + .select_only() + .columns([ + streaming_job::Column::JobId, + streaming_job::Column::JobStatus, + ]) + .into_tuple() + .all(&inner.db) + .await?; + Ok(job_states) + } + /// Get all actor ids in the target streaming jobs. pub async fn get_job_actor_mapping( &self, @@ -607,10 +674,49 @@ impl CatalogController { Ok(count > 0) } - pub fn table_fragments(&self) -> MetaResult> { - unimplemented!( - "This function is too heavy, we should avoid using it and implement others on demand." - ) + // TODO: This function is too heavy, we should avoid using it and implement others on demand. + pub async fn table_fragments(&self) -> MetaResult> { + let inner = self.inner.read().await; + let jobs = StreamingJob::find().all(&inner.db).await?; + let parallel_units_map = get_parallel_unit_mapping(&inner.db).await?; + let mut table_fragments = BTreeMap::new(); + for job in jobs { + let fragment_actors = Fragment::find() + .find_with_related(Actor) + .filter(fragment::Column::JobId.eq(job.job_id)) + .all(&inner.db) + .await?; + let mut actor_dispatchers = get_actor_dispatchers( + &inner.db, + fragment_actors + .iter() + .flat_map(|(_, actors)| actors.iter().map(|actor| actor.actor_id)) + .collect(), + ) + .await?; + let mut fragment_info = vec![]; + for (fragment, actors) in fragment_actors { + let mut dispatcher_info = HashMap::new(); + for actor in &actors { + if let Some(dispatchers) = actor_dispatchers.remove(&actor.actor_id) { + dispatcher_info.insert(actor.actor_id, dispatchers); + } + } + fragment_info.push((fragment, actors, dispatcher_info)); + } + table_fragments.insert( + job.job_id as ObjectId, + Self::compose_table_fragments( + job.job_id as _, + job.job_status.into(), + job.timezone.map(|tz| PbStreamContext { timezone: tz }), + fragment_info, + ¶llel_units_map, + )?, + ); + } + + Ok(table_fragments) } /// Check if the fragment type mask is injectable. @@ -623,9 +729,89 @@ impl CatalogController { != 0 } + pub async fn list_actor_locations(&self) -> MetaResult> { + let inner = self.inner.read().await; + let actor_locations: Vec = + Actor::find().into_partial_model().all(&inner.db).await?; + Ok(actor_locations) + } + + pub async fn list_fragment_descs(&self) -> MetaResult> { + let inner = self.inner.read().await; + let fragment_descs: Vec = Fragment::find() + .select_only() + .columns([ + fragment::Column::FragmentId, + fragment::Column::JobId, + fragment::Column::FragmentTypeMask, + fragment::Column::DistributionType, + fragment::Column::StateTableIds, + fragment::Column::UpstreamFragmentId, + ]) + .column_as(Expr::col(actor::Column::ActorId).count(), "parallelism") + .join(JoinType::LeftJoin, fragment::Relation::Actor.def()) + .group_by(fragment::Column::FragmentId) + .into_model() + .all(&inner.db) + .await?; + Ok(fragment_descs) + } + + pub async fn list_sink_actor_mapping( + &self, + ) -> MetaResult)>> { + let inner = self.inner.read().await; + let sink_id_names: Vec<(SinkId, String)> = Sink::find() + .select_only() + .columns([sink::Column::SinkId, sink::Column::Name]) + .into_tuple() + .all(&inner.db) + .await?; + let (sink_ids, _): (Vec<_>, Vec<_>) = sink_id_names.iter().cloned().unzip(); + let sink_name_mapping: HashMap = sink_id_names.into_iter().collect(); + + let actor_with_type: Vec<(ActorId, SinkId, i32)> = Actor::find() + .select_only() + .column(actor::Column::ActorId) + .columns([fragment::Column::JobId, fragment::Column::FragmentTypeMask]) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) + .filter(fragment::Column::JobId.is_in(sink_ids)) + .into_tuple() + .all(&inner.db) + .await?; + + let mut sink_actor_mapping = HashMap::new(); + for (actor_id, sink_id, type_mask) in actor_with_type { + if type_mask & PbFragmentTypeFlag::Sink as i32 != 0 { + sink_actor_mapping + .entry(sink_id) + .or_insert_with(|| (sink_name_mapping.get(&sink_id).unwrap().clone(), vec![])) + .1 + .push(actor_id); + } + } + + Ok(sink_actor_mapping) + } + + pub async fn list_fragment_state_tables(&self) -> MetaResult> { + let inner = self.inner.read().await; + let fragment_state_tables: Vec = Fragment::find() + .select_only() + .columns([ + fragment::Column::FragmentId, + fragment::Column::JobId, + fragment::Column::StateTableIds, + ]) + .into_partial_model() + .all(&inner.db) + .await?; + Ok(fragment_state_tables) + } + /// Used in [`crate::barrier::GlobalBarrierManager`], load all actor that need to be sent or /// collected - pub async fn load_all_actor( + pub async fn load_all_actors( &self, parallel_units_map: &HashMap, check_state: impl Fn(PbActorState, ObjectId, ActorId) -> bool, @@ -648,6 +834,7 @@ impl CatalogController { for (actor_id, status, parallel_unit_id, job_id, type_mask) in actor_info { let status = PbActorState::from(status); + // FIXME: since worker might have gone, it's not safe to unwrap here. let worker_id = parallel_units_map .get(&(parallel_unit_id as _)) .unwrap() @@ -672,21 +859,58 @@ impl CatalogController { }) } - pub async fn migrate_actors(&self, plan: HashMap) -> MetaResult<()> { + pub async fn migrate_actors(&self, plan: HashMap) -> MetaResult<()> { let inner = self.inner.read().await; let txn = inner.db.begin().await?; - for (from_pu_id, to_pu_id) in plan { + for (from_pu_id, to_pu_id) in &plan { Actor::update_many() .col_expr( actor::Column::ParallelUnitId, - Expr::value(Value::Int(Some(to_pu_id.id as _))), + Expr::value(Value::Int(Some(*to_pu_id))), ) - .filter(actor::Column::ParallelUnitId.eq(from_pu_id as i32)) + .filter(actor::Column::ParallelUnitId.eq(*from_pu_id)) .exec(&txn) .await?; } + + let mut fragment_mapping: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + .select_only() + .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .join(JoinType::InnerJoin, fragment::Relation::Actor.def()) + .filter(actor::Column::ParallelUnitId.is_in(plan.values().cloned().collect::>())) + .into_tuple() + .all(&txn) + .await?; + // TODO: we'd better not store vnode mapping in fragment table and derive it from actors. + for (fragment_id, vnode_mapping) in &mut fragment_mapping { + vnode_mapping.0.data.iter_mut().for_each(|id| { + if let Some(new_id) = plan.get(&(*id as i32)) { + *id = *new_id as u32; + } + }); + fragment::ActiveModel { + fragment_id: Set(*fragment_id), + vnode_mapping: Set(vnode_mapping.clone()), + ..Default::default() + } + .update(&txn) + .await?; + } + txn.commit().await?; + self.notify_fragment_mapping( + NotificationOperation::Update, + fragment_mapping + .into_iter() + .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { + fragment_id: fragment_id as _, + mapping: Some(mapping.into_inner()), + }) + .collect(), + ) + .await; + Ok(()) } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 6a8fdac04e5fe..48c6b6d43172b 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -17,14 +17,14 @@ use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::{ - connection, database, index, object, schema, sink, source, table, view, + connection, database, function, index, object, schema, sink, source, table, view, }; use risingwave_pb::catalog::connection::PbInfo as PbConnectionInfo; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableType}; use risingwave_pb::catalog::{ - PbConnection, PbCreateType, PbDatabase, PbHandleConflictBehavior, PbIndex, PbSchema, PbSink, - PbSinkType, PbSource, PbStreamJobStatus, PbTable, PbView, + PbConnection, PbCreateType, PbDatabase, PbFunction, PbHandleConflictBehavior, PbIndex, + PbSchema, PbSink, PbSinkType, PbSource, PbStreamJobStatus, PbTable, PbView, }; use sea_orm::{DatabaseConnection, ModelTrait}; @@ -107,7 +107,7 @@ impl From> for PbTable { append_only: value.0.append_only, owner: value.1.owner_id as _, properties: value.0.properties.0, - fragment_id: value.0.fragment_id as u32, + fragment_id: value.0.fragment_id.unwrap_or_default() as u32, vnode_col_index: value.0.vnode_col_index.map(|index| index as _), row_id_index: value.0.row_id_index.map(|index| index as _), value_indices: value.0.value_indices.0, @@ -127,14 +127,14 @@ impl From> for PbTable { Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, ), cleaned_by_watermark: value.0.cleaned_by_watermark, - stream_job_status: PbStreamJobStatus::from(value.0.job_status) as _, - create_type: PbCreateType::from(value.0.create_type) as _, - version: Some(value.0.version.0), + stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. + create_type: PbCreateType::Foreground as _, + version: value.0.version.map(|v| v.into_inner()), optional_associated_source_id: value .0 .optional_associated_source_id .map(|id| PbOptionalAssociatedSourceId::AssociatedSourceId(id as _)), - description: None, + description: value.0.description, // TODO: fix it for model v2. incoming_sinks: vec![], } @@ -198,7 +198,7 @@ impl From> for PbSink { ), db_name: value.0.db_name, sink_from_name: value.0.sink_from_name, - stream_job_status: PbStreamJobStatus::from(value.0.job_status) as _, + stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. format_desc: value.0.sink_format_desc.map(|desc| desc.0), // todo: fix this for model v2 target_table: None, @@ -224,7 +224,7 @@ impl From> for PbIndex { created_at_epoch: Some( Epoch::from_unix_millis(value.1.created_at.timestamp_millis() as _).0, ), - stream_job_status: PbStreamJobStatus::from(value.0.job_status) as _, + stream_job_status: PbStreamJobStatus::Created as _, // todo: deprecate it. } } } @@ -257,3 +257,21 @@ impl From> for PbConnection { } } } + +impl From> for PbFunction { + fn from(value: ObjectModel) -> Self { + Self { + id: value.0.function_id as _, + schema_id: value.1.schema_id.unwrap() as _, + database_id: value.1.database_id.unwrap() as _, + name: value.0.name, + owner: value.1.owner_id as _, + arg_types: value.0.arg_types.into_inner(), + return_type: Some(value.0.return_type.into_inner()), + language: value.0.language, + link: value.0.link, + identifier: value.0.identifier, + kind: Some(value.0.kind.into()), + } + } +} diff --git a/src/meta/src/controller/user.rs b/src/meta/src/controller/user.rs index fb5cfb4d14e30..967a5d232498b 100644 --- a/src/meta/src/controller/user.rs +++ b/src/meta/src/controller/user.rs @@ -54,7 +54,7 @@ impl CatalogController { version } - async fn create_user(&self, pb_user: PbUserInfo) -> MetaResult { + pub async fn create_user(&self, pb_user: PbUserInfo) -> MetaResult { let inner = self.inner.write().await; let txn = inner.db.begin().await?; check_user_name_duplicate(&pb_user.name, &txn).await?; @@ -94,7 +94,7 @@ impl CatalogController { Ok(version) } - async fn update_user( + pub async fn update_user( &self, update_user: PbUserInfo, update_fields: &[PbUpdateField], @@ -158,7 +158,7 @@ impl CatalogController { Ok(user) } - async fn drop_user(&self, user_id: UserId) -> MetaResult { + pub async fn drop_user(&self, user_id: UserId) -> MetaResult { let inner = self.inner.write().await; let txn = inner.db.begin().await?; let user = User::find_by_id(user_id) @@ -271,13 +271,16 @@ impl CatalogController { } // insert privileges - let user_privileges = user_ids.iter().flat_map(|user_id| { - privileges.iter().map(|p| { - let mut p = p.clone(); - p.user_id = Set(*user_id); - p + let user_privileges = user_ids + .iter() + .flat_map(|user_id| { + privileges.iter().map(|p| { + let mut p = p.clone(); + p.user_id = Set(*user_id); + p + }) }) - }); + .collect_vec(); for privilege in user_privileges { let mut on_conflict = OnConflict::columns([ user_privilege::Column::UserId, diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 5567c77ef0768..46655dc13fa13 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -17,15 +17,19 @@ use std::collections::HashMap; use anyhow::anyhow; use itertools::Itertools; use risingwave_meta_model_migration::WithQuery; +use risingwave_meta_model_v2::actor::ActorStatus; +use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::object::ObjectType; use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ - actor_dispatcher, connection, database, function, index, object, object_dependency, schema, - sink, source, table, user, user_privilege, view, worker_property, ActorId, DataTypeArray, - DatabaseId, I32Array, ObjectId, PrivilegeId, SchemaId, UserId, WorkerId, + actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, + schema, sink, source, table, user, user_privilege, view, worker_property, ActorId, + DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, PrivilegeId, + SchemaId, UserId, WorkerId, }; use risingwave_pb::catalog::{PbConnection, PbFunction}; use risingwave_pb::common::PbParallelUnit; +use risingwave_pb::meta::PbFragmentParallelUnitMapping; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; use risingwave_pb::user::{PbGrantPrivilege, PbUserInfo}; use sea_orm::sea_query::{ @@ -122,6 +126,34 @@ pub struct PartialObject { pub database_id: Option, } +#[derive(Clone, DerivePartialModel, FromQueryResult)] +#[sea_orm(entity = "Fragment")] +pub struct PartialFragmentStateTables { + pub fragment_id: FragmentId, + pub job_id: ObjectId, + pub state_table_ids: I32Array, +} + +#[derive(Clone, DerivePartialModel, FromQueryResult)] +#[sea_orm(entity = "Actor")] +pub struct PartialActorLocation { + pub actor_id: ActorId, + pub fragment_id: FragmentId, + pub parallel_unit_id: i32, + pub status: ActorStatus, +} + +#[derive(FromQueryResult)] +pub struct FragmentDesc { + pub fragment_id: FragmentId, + pub job_id: ObjectId, + pub fragment_type_mask: i32, + pub distribution_type: DistributionType, + pub state_table_ids: I32Array, + pub upstream_fragment_id: I32Array, + pub parallelism: i64, +} + /// List all objects that are using the given one in a cascade way. It runs a recursive CTE to find all the dependencies. pub async fn get_referring_objects_cascade( obj_id: ObjectId, @@ -632,3 +664,28 @@ where .map(|(actor_id, actor_dispatcher)| (actor_id, actor_dispatcher.collect())) .collect()) } + +/// `get_fragment_parallel_unit_mappings` returns the fragment vnode mappings of the given job. +pub async fn get_fragment_mappings( + db: &C, + job_id: ObjectId, +) -> MetaResult> +where + C: ConnectionTrait, +{ + let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + .select_only() + .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .filter(fragment::Column::JobId.eq(job_id)) + .into_tuple() + .all(db) + .await?; + + Ok(fragment_mappings + .into_iter() + .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { + fragment_id: fragment_id as _, + mapping: Some(mapping.into_inner()), + }) + .collect()) +} From 4ad19403b10dcd9b8595694f06e73c2f843d374f Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 25 Dec 2023 17:43:23 +0800 Subject: [PATCH 02/15] feat(storage): pass per-vnode watermark to hummock (#13429) --- .../hummock_sdk/src/table_watermark.rs | 115 +++- .../hummock_test/src/hummock_storage_tests.rs | 497 +++++++++++++++++- src/storage/hummock_trace/src/opts.rs | 5 +- .../event_handler/hummock_event_handler.rs | 12 +- .../src/hummock/event_handler/uploader.rs | 1 - .../hummock/store/local_hummock_storage.rs | 13 +- src/storage/src/hummock/store/version.rs | 29 +- src/storage/src/mem_table.rs | 6 +- src/storage/src/store.rs | 65 ++- .../log_store_impl/kv_log_store/reader.rs | 2 +- .../log_store_impl/kv_log_store/serde.rs | 24 +- .../log_store_impl/kv_log_store/writer.rs | 27 +- src/stream/src/common/table/state_table.rs | 68 +-- 13 files changed, 774 insertions(+), 90 deletions(-) diff --git a/src/storage/hummock_sdk/src/table_watermark.rs b/src/storage/hummock_sdk/src/table_watermark.rs index e4967b6e39508..2f76cff6cc5e4 100644 --- a/src/storage/hummock_sdk/src/table_watermark.rs +++ b/src/storage/hummock_sdk/src/table_watermark.rs @@ -23,7 +23,7 @@ use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; use risingwave_pb::hummock::{PbTableWatermarks, PbVnodeWatermark}; -use tracing::debug; +use tracing::{debug, warn}; use crate::key::{prefix_slice_with_vnode, vnode_range, TableKey, TableKeyRange}; use crate::HummockEpoch; @@ -82,6 +82,10 @@ impl TableWatermarksIndex { } } + pub fn index(&self) -> &HashMap> { + &self.index + } + pub fn read_watermark(&self, vnode: VirtualNode, epoch: HummockEpoch) -> Option { self.index.get(&vnode).and_then(|epoch_watermarks| { epoch_watermarks @@ -171,6 +175,54 @@ impl TableWatermarksIndex { } } + pub fn filter_regress_watermarks(&self, watermarks: &mut Vec) { + let mut ret = Vec::with_capacity(watermarks.len()); + for watermark in watermarks.drain(..) { + let mut regress_vnodes = None; + for vnode in watermark.vnode_bitmap.iter_vnodes() { + if let Some(prev_watermark) = self.latest_watermark(vnode) { + let is_regress = match self.direction() { + WatermarkDirection::Ascending => prev_watermark > watermark.watermark, + WatermarkDirection::Descending => prev_watermark < watermark.watermark, + }; + if is_regress { + warn!( + "table watermark regress: {:?} {} {:?} {:?}", + self.direction(), + vnode, + watermark.watermark, + prev_watermark + ); + regress_vnodes + .get_or_insert_with(|| BitmapBuilder::zeroed(VirtualNode::COUNT)) + .set(vnode.to_index(), true); + } + } + } + if let Some(regress_vnodes) = regress_vnodes { + let mut bitmap_builder = None; + for vnode in watermark.vnode_bitmap.iter_vnodes() { + let vnode_index = vnode.to_index(); + if !regress_vnodes.is_set(vnode_index) { + bitmap_builder + .get_or_insert_with(|| BitmapBuilder::zeroed(VirtualNode::COUNT)) + .set(vnode_index, true); + } + } + if let Some(bitmap_builder) = bitmap_builder { + ret.push(VnodeWatermark::new( + Arc::new(bitmap_builder.finish()), + watermark.watermark, + )); + } + } else { + // no vnode has regress watermark + ret.push(watermark); + } + } + *watermarks = ret; + } + pub fn direction(&self) -> WatermarkDirection { self.watermark_direction } @@ -238,7 +290,7 @@ impl WatermarkDirection { } } -#[derive(Clone, Debug)] +#[derive(Clone, Debug, PartialEq)] pub struct VnodeWatermark { vnode_bitmap: Arc, watermark: Bytes, @@ -826,6 +878,9 @@ mod tests { prefixed_range_with_vnode(range, TEST_SINGLE_VNODE) } + /// Build and return a watermark index with the following watermarks + /// EPOCH1 bitmap(0, 1, 2, 3) watermark1 + /// EPOCH2 bitmap(1, 2, 3, 4) watermark2 fn build_and_test_watermark_index( direction: WatermarkDirection, watermark1: Bytes, @@ -1030,4 +1085,60 @@ mod tests { } } } + + #[test] + fn test_filter_regress_watermark() { + let watermark1 = Bytes::from_static(b"watermark1"); + let watermark2 = Bytes::from_static(b"watermark2"); + let watermark3 = Bytes::from_static(b"watermark3"); + let index = build_and_test_watermark_index( + WatermarkDirection::Ascending, + watermark1.clone(), + watermark2.clone(), + watermark3.clone(), + ); + + let mut new_watermarks = vec![ + // Partial regress + VnodeWatermark { + vnode_bitmap: build_bitmap(0..2), + watermark: watermark1.clone(), + }, + // All not regress + VnodeWatermark { + vnode_bitmap: build_bitmap(2..4), + watermark: watermark3.clone(), + }, + // All regress + VnodeWatermark { + vnode_bitmap: build_bitmap(4..5), + watermark: watermark1.clone(), + }, + // All newly set vnode + VnodeWatermark { + vnode_bitmap: build_bitmap(5..6), + watermark: watermark3.clone(), + }, + ]; + + index.filter_regress_watermarks(&mut new_watermarks); + + assert_eq!( + new_watermarks, + vec![ + VnodeWatermark { + vnode_bitmap: build_bitmap(0..1), + watermark: watermark1, + }, + VnodeWatermark { + vnode_bitmap: build_bitmap(2..4), + watermark: watermark3.clone(), + }, + VnodeWatermark { + vnode_bitmap: build_bitmap(5..6), + watermark: watermark3, + }, + ] + ); + } } diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 5826ef5c8bb65..9c5e7fac402a9 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -12,18 +12,26 @@ // See the License for the specific language governing permissions and // limitations under the License. use std::ops::Bound::{Excluded, Included, Unbounded}; +use std::ops::Range; use std::sync::Arc; use bytes::{BufMut, Bytes}; use futures::TryStreamExt; +use itertools::Itertools; use parking_lot::RwLock; +use risingwave_common::buffer::BitmapBuilder; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; -use risingwave_hummock_sdk::key::{FullKey, TableKey, TABLE_PREFIX_LEN}; +use risingwave_common::range::RangeBoundsExt; +use risingwave_hummock_sdk::key::{ + gen_key_from_bytes, prefix_slice_with_vnode, FullKey, TableKey, UserKey, TABLE_PREFIX_LEN, +}; +use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection}; use risingwave_rpc_client::HummockMetaClient; +use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use risingwave_storage::hummock::store::version::{read_filter_for_batch, read_filter_for_local}; -use risingwave_storage::hummock::CachePolicy; +use risingwave_storage::hummock::{CachePolicy, HummockStorage, LocalHummockStorage}; use risingwave_storage::storage_value::StorageValue; use risingwave_storage::store::*; use risingwave_storage::StateStore; @@ -1780,3 +1788,488 @@ async fn test_get_with_min_epoch() { assert!(v.is_none()); } } + +#[tokio::test] +async fn test_table_watermark() { + const TEST_TABLE_ID: TableId = TableId { table_id: 233 }; + let test_env = prepare_hummock_test_env().await; + test_env.register_table_id(TEST_TABLE_ID).await; + let mut local1 = test_env + .storage + .new_local(NewLocalOptions::for_test(TEST_TABLE_ID)) + .await; + + let mut local2 = test_env + .storage + .new_local(NewLocalOptions::for_test(TEST_TABLE_ID)) + .await; + + let vnode1 = VirtualNode::from_index(1); + let vnode_bitmap1 = { + let mut builder = BitmapBuilder::zeroed(VirtualNode::COUNT); + builder.set(1, true); + builder.finish() + }; + let vnode2 = VirtualNode::from_index(2); + let vnode_bitmap2 = { + let mut builder = BitmapBuilder::zeroed(VirtualNode::COUNT); + builder.set(2, true); + builder.finish() + }; + + let epoch1 = (31 * 1000) << 16; + local1.init_for_test(epoch1).await.unwrap(); + local2.init_for_test(epoch1).await.unwrap(); + + fn gen_inner_key(index: usize) -> Bytes { + Bytes::copy_from_slice(format!("key_{:05}", index).as_bytes()) + } + + fn gen_key(vnode: VirtualNode, index: usize) -> TableKey { + gen_key_from_bytes(vnode, &gen_inner_key(index)) + } + + fn gen_val(index: usize) -> Bytes { + Bytes::copy_from_slice(format!("val_{}", index).as_bytes()) + } + + fn gen_range() -> Range { + 0..30 + } + + fn gen_batch( + vnode: VirtualNode, + index: impl Iterator, + ) -> Vec<(TableKey, Bytes)> { + index + .map(|index| (gen_key(vnode, index), gen_val(index))) + .collect_vec() + } + + let epoch1_indexes = || gen_range().filter(|index| index % 3 == 0); + + // epoch 1 write + let batch1_epoch1 = gen_batch(vnode1, epoch1_indexes()); + let batch2_epoch1 = gen_batch(vnode2, epoch1_indexes()); + + for (local, batch) in [(&mut local1, batch1_epoch1), (&mut local2, batch2_epoch1)] { + for (key, value) in batch { + local.insert(key, value, None).unwrap(); + } + } + + // test read after write + { + for (local, vnode) in [(&local1, vnode1), (&local2, vnode2)] { + for index in epoch1_indexes() { + let value = risingwave_storage::store::LocalStateStore::get( + local, + gen_key(vnode, index), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap(); + assert_eq!(value.unwrap(), gen_val(index)); + } + let result = risingwave_storage::store::LocalStateStore::iter( + local, + RangeBoundsExt::map(&gen_range(), |index| gen_key(vnode, *index)), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .map_ok(|(full_key, value)| (full_key.user_key, value)) + .try_collect::>() + .await + .unwrap(); + let expected = epoch1_indexes() + .map(|index| { + ( + UserKey::new(TEST_TABLE_ID, gen_key(vnode, index)), + gen_val(index), + ) + }) + .collect_vec(); + assert_eq!(expected, result); + } + } + + let watermark1 = 10; + + let epoch2 = (32 * 1000) << 16; + for (local, vnode_bitmap) in [ + (&mut local1, vnode_bitmap1.clone()), + (&mut local2, vnode_bitmap2.clone()), + ] { + local.flush(vec![]).await.unwrap(); + local.seal_current_epoch( + epoch2, + SealCurrentEpochOptions::new( + vec![VnodeWatermark::new( + Arc::new(vnode_bitmap), + gen_inner_key(watermark1), + )], + WatermarkDirection::Ascending, + ), + ); + } + + // test read after seal with watermark1 + { + for (local, vnode) in [(&local1, vnode1), (&local2, vnode2)] { + for index in epoch1_indexes() { + let value = risingwave_storage::store::LocalStateStore::get( + local, + gen_key(vnode, index), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap(); + if index < watermark1 { + assert!(value.is_none()); + } else { + assert_eq!(value.unwrap(), gen_val(index)); + } + } + + // iter full range + { + let result = risingwave_storage::store::LocalStateStore::iter( + local, + RangeBoundsExt::map(&gen_range(), |index| gen_key(vnode, *index)), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .map_ok(|(full_key, value)| (full_key.user_key, value)) + .try_collect::>() + .await + .unwrap(); + let expected = epoch1_indexes() + .filter(|index| index >= &watermark1) + .map(|index| { + ( + UserKey::new(TEST_TABLE_ID, gen_key(vnode, index)), + gen_val(index), + ) + }) + .collect_vec(); + assert_eq!(expected, result); + } + + // iter below watermark + { + let result = risingwave_storage::store::LocalStateStore::iter( + local, + ( + Included(gen_key(vnode, 0)), + Included(gen_key(vnode, watermark1 - 1)), + ), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .try_collect::>() + .await + .unwrap(); + assert!(result.is_empty()); + } + } + } + + let epoch2_indexes = || { + gen_range() + .filter(|index| index % 3 == 1) + .filter(|index| index >= &watermark1) + }; + + // epoch 2 write + let batch1_epoch2 = gen_batch(vnode1, epoch2_indexes()); + let batch2_epoch2 = gen_batch(vnode2, epoch2_indexes()); + + let epoch3 = (33 * 1000) << 16; + + for (local, batch) in [(&mut local1, batch1_epoch2), (&mut local2, batch2_epoch2)] { + for (key, value) in batch { + local.insert(key, value, None).unwrap(); + } + local.flush(vec![]).await.unwrap(); + local.seal_current_epoch(epoch3, SealCurrentEpochOptions::no_watermark()); + } + + let indexes_after_epoch2 = || gen_range().filter(|index| index % 3 == 0 || index % 3 == 1); + + let test_after_epoch2 = |local1: LocalHummockStorage, local2: LocalHummockStorage| async { + for (local, vnode) in [(&local1, vnode1), (&local2, vnode2)] { + for index in indexes_after_epoch2() { + let value = risingwave_storage::store::LocalStateStore::get( + local, + gen_key(vnode, index), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap(); + if index < watermark1 { + assert!(value.is_none()); + } else { + assert_eq!(value.unwrap(), gen_val(index)); + } + } + + // iter full range + { + let result = risingwave_storage::store::LocalStateStore::iter( + local, + RangeBoundsExt::map(&gen_range(), |index| gen_key(vnode, *index)), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .map_ok(|(full_key, value)| (full_key.user_key, value)) + .try_collect::>() + .await + .unwrap(); + let expected = indexes_after_epoch2() + .filter(|index| index >= &watermark1) + .map(|index| { + ( + UserKey::new(TEST_TABLE_ID, gen_key(vnode, index)), + gen_val(index), + ) + }) + .collect_vec(); + assert_eq!(expected, result); + } + + // iter below watermark + { + let result = risingwave_storage::store::LocalStateStore::iter( + local, + ( + Included(gen_key(vnode, 0)), + Included(gen_key(vnode, watermark1 - 1)), + ), + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .try_collect::>() + .await + .unwrap(); + assert!(result.is_empty()); + } + } + (local1, local2) + }; + + let (local1, local2) = test_after_epoch2(local1, local2).await; + + let check_version_table_watermark = |version: PinnedVersion| { + let table_watermarks = version.table_watermark_index().get(&TEST_TABLE_ID).unwrap(); + assert_eq!(WatermarkDirection::Ascending, table_watermarks.direction()); + let index = table_watermarks.index(); + assert_eq!(2, index.len()); + let vnode1_watermark = index.get(&vnode1).unwrap(); + assert_eq!(1, vnode1_watermark.len()); + assert_eq!( + &gen_inner_key(watermark1), + vnode1_watermark.get(&epoch1).unwrap() + ); + let vnode2_watermark = index.get(&vnode2).unwrap(); + assert_eq!(1, vnode2_watermark.len()); + assert_eq!( + &gen_inner_key(watermark1), + vnode2_watermark.get(&epoch1).unwrap() + ); + }; + + test_env.commit_epoch(epoch1).await; + test_env.storage.try_wait_epoch_for_test(epoch1).await; + + let test_global_read = |storage: HummockStorage, epoch: u64| async move { + // inner vnode read + for vnode in [vnode1, vnode2] { + for index in indexes_after_epoch2() { + let value = storage + .get( + gen_key(vnode, index), + epoch, + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap(); + if index < watermark1 { + assert!(value.is_none()); + } else { + assert_eq!(value.unwrap(), gen_val(index)); + } + } + + // iter full range + { + let result = storage + .iter( + RangeBoundsExt::map(&gen_range(), |index| gen_key(vnode, *index)), + epoch, + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .map_ok(|(full_key, value)| (full_key.user_key, value)) + .try_collect::>() + .await + .unwrap(); + let expected = indexes_after_epoch2() + .filter(|index| index >= &watermark1) + .map(|index| { + ( + UserKey::new(TEST_TABLE_ID, gen_key(vnode, index)), + gen_val(index), + ) + }) + .collect_vec(); + assert_eq!(expected, result); + } + + // iter below watermark + { + let result = storage + .iter( + ( + Included(gen_key(vnode, 0)), + Included(gen_key(vnode, watermark1 - 1)), + ), + epoch, + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .try_collect::>() + .await + .unwrap(); + assert!(result.is_empty()); + } + } + + // cross vnode read + let result = storage + .iter( + ( + Included(TableKey(prefix_slice_with_vnode( + vnode1, + &gen_inner_key(gen_range().start), + ))), + Included(TableKey(prefix_slice_with_vnode( + vnode2, + &gen_inner_key(gen_range().end), + ))), + ), + epoch, + ReadOptions { + table_id: TEST_TABLE_ID, + ..Default::default() + }, + ) + .await + .unwrap() + .map_ok(|(full_key, value)| (full_key.user_key, value)) + .try_collect::>() + .await + .unwrap(); + let expected = [vnode1, vnode2] + .into_iter() + .flat_map(|vnode| { + gen_range() + .filter(|index| index % 3 == 0 || index % 3 == 1) + .filter(|index| index >= &watermark1) + .map(move |index| { + ( + UserKey::new(TEST_TABLE_ID, gen_key(vnode, index)), + gen_val(index), + ) + }) + }) + .collect_vec(); + assert_eq!(expected, result); + }; + + test_global_read(test_env.storage.clone(), epoch2).await; + + check_version_table_watermark(test_env.storage.get_pinned_version()); + + let (local1, local2) = test_after_epoch2(local1, local2).await; + + test_env.commit_epoch(epoch2).await; + test_env.storage.try_wait_epoch_for_test(epoch2).await; + + test_global_read(test_env.storage.clone(), epoch2).await; + + check_version_table_watermark(test_env.storage.get_pinned_version()); + + let (mut local1, mut local2) = test_after_epoch2(local1, local2).await; + + let epoch4 = (34 * 1000) << 16; + + for (local, vnode_bitmap) in [ + (&mut local1, vnode_bitmap1.clone()), + (&mut local2, vnode_bitmap2.clone()), + ] { + // regress watermark + local.seal_current_epoch( + epoch4, + SealCurrentEpochOptions::new( + vec![VnodeWatermark::new( + Arc::new(vnode_bitmap), + gen_inner_key(5), + )], + WatermarkDirection::Ascending, + ), + ); + } + + test_global_read(test_env.storage.clone(), epoch3).await; + + let (local1, local2) = test_after_epoch2(local1, local2).await; + + test_env.commit_epoch(epoch3).await; + test_env.storage.try_wait_epoch_for_test(epoch3).await; + + check_version_table_watermark(test_env.storage.get_pinned_version()); + + let (_local1, _local2) = test_after_epoch2(local1, local2).await; + + test_global_read(test_env.storage.clone(), epoch3).await; +} diff --git a/src/storage/hummock_trace/src/opts.rs b/src/storage/hummock_trace/src/opts.rs index b9eb6fd3ed699..dfa3ba46ac4e7 100644 --- a/src/storage/hummock_trace/src/opts.rs +++ b/src/storage/hummock_trace/src/opts.rs @@ -219,4 +219,7 @@ pub struct TracedInitOptions { } #[derive(Debug, Clone, PartialEq, Eq, Decode, Encode)] -pub struct TracedSealCurrentEpochOptions {} +pub struct TracedSealCurrentEpochOptions { + // The watermark is serialized into protobuf + pub table_watermarks: Option<(bool, Vec>)>, +} diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index f8fc574a54ad0..498c41c6e635d 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -578,7 +578,17 @@ impl HummockEventHandler { } } - HummockEvent::LocalSealEpoch { .. } => {} + HummockEvent::LocalSealEpoch { + epoch, + opts, + table_id, + .. + } => { + if let Some((direction, watermarks)) = opts.table_watermarks { + self.uploader + .add_table_watermarks(epoch, table_id, watermarks, direction) + } + } #[cfg(any(test, feature = "test"))] HummockEvent::FlushEvent(sender) => { diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index ba6cb511259b7..66357753fd038 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -759,7 +759,6 @@ impl HummockUploader { .push_front(imm); } - #[expect(dead_code)] pub(crate) fn add_table_watermarks( &mut self, epoch: u64, diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 2586941210286..72c24cc5b1cb7 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -394,7 +394,7 @@ impl LocalStateStore for LocalHummockStorage { Ok(()) } - fn seal_current_epoch(&mut self, next_epoch: u64, opts: SealCurrentEpochOptions) { + fn seal_current_epoch(&mut self, next_epoch: u64, mut opts: SealCurrentEpochOptions) { assert!(!self.is_dirty()); let prev_epoch = self .epoch @@ -407,6 +407,17 @@ impl LocalStateStore for LocalHummockStorage { next_epoch, prev_epoch ); + if let Some((direction, watermarks)) = &mut opts.table_watermarks { + let mut read_version = self.read_version.write(); + read_version.filter_regress_watermarks(watermarks); + if !watermarks.is_empty() { + read_version.update(VersionUpdate::NewTableWatermark { + direction: *direction, + epoch: prev_epoch, + vnode_watermarks: watermarks.clone(), + }); + } + } self.event_sender .send(HummockEvent::LocalSealEpoch { instance_id: self.instance_id(), diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 5496e94b9e751..8fcd81d6b75de 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -27,7 +27,9 @@ use risingwave_hummock_sdk::key::{ bound_table_key_range, is_empty_key_range, FullKey, TableKey, TableKeyRange, UserKey, }; use risingwave_hummock_sdk::key_range::KeyRangeCommon; -use risingwave_hummock_sdk::table_watermark::{ReadTableWatermark, TableWatermarksIndex}; +use risingwave_hummock_sdk::table_watermark::{ + ReadTableWatermark, TableWatermarksIndex, VnodeWatermark, WatermarkDirection, +}; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::{HummockVersionDelta, LevelType, SstableInfo}; use sync_point::sync_point; @@ -124,6 +126,11 @@ pub enum VersionUpdate { Staging(StagingData), CommittedDelta(HummockVersionDelta), CommittedSnapshot(CommittedVersion), + NewTableWatermark { + direction: WatermarkDirection, + epoch: HummockEpoch, + vnode_watermarks: Vec, + }, } #[derive(Clone)] @@ -407,6 +414,16 @@ impl HummockReadVersion { } } } + VersionUpdate::NewTableWatermark { + direction, + epoch, + vnode_watermarks, + } => self + .table_watermarks + .get_or_insert_with(|| { + TableWatermarksIndex::new(direction, self.committed.max_committed_epoch()) + }) + .add_epoch_watermark(epoch, &vnode_watermarks, direction), } } @@ -418,6 +435,16 @@ impl HummockReadVersion { &self.committed } + /// We have assumption that the watermark is increasing monotonically. Therefore, + /// here if the upper layer usage has passed an regressed watermark, we should + /// filter out the regressed watermark. Currently the kv log store may write + /// regressed watermark + pub fn filter_regress_watermarks(&self, watermarks: &mut Vec) { + if let Some(watermark_index) = &self.table_watermarks { + watermark_index.filter_regress_watermarks(watermarks) + } + } + pub fn clear_uncommitted(&mut self) { self.staging.imm.clear(); self.staging.merged_imm.clear(); diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index 25342f2b163e2..eca66bfba2f74 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -26,6 +26,7 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::estimate_size::{EstimateSize, KvSize}; use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; use thiserror::Error; +use tracing::warn; use crate::error::{StorageError, StorageResult}; use crate::hummock::iterator::{FromRustIterator, RustIteratorBuilder}; @@ -600,7 +601,7 @@ impl LocalStateStore for MemtableLocalState Ok(()) } - fn seal_current_epoch(&mut self, next_epoch: u64, _opts: SealCurrentEpochOptions) { + fn seal_current_epoch(&mut self, next_epoch: u64, opts: SealCurrentEpochOptions) { assert!(!self.is_dirty()); let prev_epoch = self .epoch @@ -612,6 +613,9 @@ impl LocalStateStore for MemtableLocalState next_epoch, prev_epoch ); + if opts.table_watermarks.is_some() { + warn!("table watermark only supported in hummock state store"); + } } async fn try_flush(&mut self) -> StorageResult<()> { diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index 7564df7a7dfc4..cf5211d7069e5 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -21,10 +21,13 @@ use std::sync::Arc; use bytes::Bytes; use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; +use prost::Message; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::util::epoch::{Epoch, EpochPair}; use risingwave_hummock_sdk::key::{FullKey, TableKey, TableKeyRange}; -use risingwave_hummock_sdk::table_watermark::TableWatermarks; +use risingwave_hummock_sdk::table_watermark::{ + TableWatermarks, VnodeWatermark, WatermarkDirection, +}; use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo}; use risingwave_hummock_trace::{ TracedInitOptions, TracedNewLocalOptions, TracedPrefetchOptions, TracedReadOptions, @@ -503,30 +506,64 @@ impl From for InitOptions { } #[derive(Clone, Debug)] -pub struct SealCurrentEpochOptions {} +pub struct SealCurrentEpochOptions { + pub table_watermarks: Option<(WatermarkDirection, Vec)>, +} impl From for TracedSealCurrentEpochOptions { - fn from(_value: SealCurrentEpochOptions) -> Self { - TracedSealCurrentEpochOptions {} + fn from(value: SealCurrentEpochOptions) -> Self { + TracedSealCurrentEpochOptions { + table_watermarks: value.table_watermarks.map(|(direction, watermarks)| { + ( + direction == WatermarkDirection::Ascending, + watermarks + .iter() + .map(|watermark| Message::encode_to_vec(&watermark.to_protobuf())) + .collect(), + ) + }), + } } } -impl TryInto for TracedSealCurrentEpochOptions { - type Error = anyhow::Error; - - fn try_into(self) -> Result { - Ok(SealCurrentEpochOptions {}) +impl From for SealCurrentEpochOptions { + fn from(value: TracedSealCurrentEpochOptions) -> SealCurrentEpochOptions { + SealCurrentEpochOptions { + table_watermarks: value.table_watermarks.map(|(is_ascending, watermarks)| { + ( + if is_ascending { + WatermarkDirection::Ascending + } else { + WatermarkDirection::Descending + }, + watermarks + .iter() + .map(|serialized_watermark| { + Message::decode(serialized_watermark.as_slice()) + .map(|pb| VnodeWatermark::from_protobuf(&pb)) + .expect("should not failed") + }) + .collect(), + ) + }), + } } } impl SealCurrentEpochOptions { - #[expect(clippy::new_without_default)] - pub fn new() -> Self { - Self {} + pub fn new(watermarks: Vec, direction: WatermarkDirection) -> Self { + Self { + table_watermarks: Some((direction, watermarks)), + } + } + + pub fn no_watermark() -> Self { + Self { + table_watermarks: None, + } } - #[cfg(any(test, feature = "test"))] pub fn for_test() -> Self { - Self::new() + Self::no_watermark() } } diff --git a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs index dede4e21a61dc..8cb126f64b7fc 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs @@ -482,6 +482,6 @@ impl LogReader for KvLogStoreReader { } self.rx.rewind(); - Ok((true, Some(self.serde.vnodes().clone()))) + Ok((true, Some((**self.serde.vnodes()).clone()))) } } diff --git a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs index 22d5958d7409b..7ff0661f9795d 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -183,8 +183,8 @@ impl LogStoreRowSerde { self.vnodes = vnodes; } - pub(crate) fn vnodes(&self) -> &Bitmap { - self.vnodes.as_ref() + pub(crate) fn vnodes(&self) -> &Arc { + &self.vnodes } pub(crate) fn encode_epoch(epoch: u64) -> i64 { @@ -272,14 +272,16 @@ impl LogStoreRowSerde { pub(crate) fn serialize_truncation_offset_watermark( &self, - vnode: VirtualNode, offset: ReaderTruncationOffsetType, ) -> Bytes { let (epoch, seq_id) = offset; - let curr_offset = self.serialize_log_store_pk(vnode, epoch, seq_id); - let ret = Bytes::from(next_key(&curr_offset)); - assert!(!ret.is_empty()); - ret + Bytes::from(next_key(&serialize_pk( + [ + Some(ScalarImpl::Int64(Self::encode_epoch(epoch))), + seq_id.map(ScalarImpl::Int32), + ], + &self.pk_serde, + ))) } } @@ -781,9 +783,7 @@ mod tests { fn remove_vnode_prefix(key: &Bytes) -> Bytes { key.slice(VirtualNode::SIZE..) } - let delete_range_right1 = remove_vnode_prefix( - &serde.serialize_truncation_offset_watermark(DEFAULT_VNODE, (epoch, None)), - ); + let delete_range_right1 = serde.serialize_truncation_offset_watermark((epoch, None)); for (op, row) in stream_chunk.rows() { let (_, key, value) = serde.serialize_data_row(epoch, seq_id, op, row); @@ -820,9 +820,7 @@ mod tests { seq_id = 1; epoch += 1; - let delete_range_right2 = remove_vnode_prefix( - &serde.serialize_truncation_offset_watermark(DEFAULT_VNODE, (epoch, None)), - ); + let delete_range_right2 = serde.serialize_truncation_offset_watermark((epoch, None)); for (op, row) in stream_chunk.rows() { let (_, key, value) = serde.serialize_data_row(epoch, seq_id, op, row); diff --git a/src/stream/src/common/log_store_impl/kv_log_store/writer.rs b/src/stream/src/common/log_store_impl/kv_log_store/writer.rs index 7b215e772324f..fa380a77b6aee 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/writer.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/writer.rs @@ -12,11 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Bound::{Excluded, Included}; use std::sync::Arc; use anyhow::anyhow; -use bytes::Bytes; +use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; @@ -24,6 +23,7 @@ use risingwave_common::estimate_size::EstimateSize; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_connector::sink::log_store::{LogStoreResult, LogWriter}; +use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection}; use risingwave_storage::store::{InitOptions, LocalStateStore, SealCurrentEpochOptions}; use tokio::sync::watch; @@ -150,19 +150,20 @@ impl LogWriter for KvLogStoreWriter { Ok(()) })?; flush_info.report(&self.metrics); - let mut delete_range = Vec::with_capacity(self.serde.vnodes().count_ones()); + let mut watermark = None; if let Some(truncation_offset) = self.tx.pop_truncation(epoch) { - for vnode in self.serde.vnodes().iter_vnodes() { - let range_begin = Bytes::from(vnode.to_be_bytes().to_vec()); - let range_end = self - .serde - .serialize_truncation_offset_watermark(vnode, truncation_offset); - delete_range.push((Included(range_begin), Excluded(range_end))); - } + watermark = Some(VnodeWatermark::new( + self.serde.vnodes().clone(), + self.serde + .serialize_truncation_offset_watermark(truncation_offset), + )); } - self.state_store.flush(delete_range).await?; - self.state_store - .seal_current_epoch(next_epoch, SealCurrentEpochOptions::new()); + self.state_store.flush(vec![]).await?; + let watermark = watermark.into_iter().collect_vec(); + self.state_store.seal_current_epoch( + next_epoch, + SealCurrentEpochOptions::new(watermark, WatermarkDirection::Ascending), + ); self.tx.barrier(epoch, is_checkpoint, next_epoch); self.seq_id = FIRST_SEQ_ID; Ok(()) diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index b5cf00eceb78c..d887684686977 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -40,9 +40,10 @@ use risingwave_common::util::row_serde::OrderedRowSerde; use risingwave_common::util::sort_util::OrderType; use risingwave_common::util::value_encoding::BasicSerde; use risingwave_hummock_sdk::key::{ - end_bound_of_prefix, next_key, prefixed_range_with_vnode, range_of_prefix, + end_bound_of_prefix, prefixed_range_with_vnode, range_of_prefix, start_bound_of_excluded_prefix, TableKey, TableKeyRange, }; +use risingwave_hummock_sdk::table_watermark::{VnodeWatermark, WatermarkDirection}; use risingwave_pb::catalog::Table; use risingwave_storage::error::{ErrorKind, StorageError, StorageResult}; use risingwave_storage::hummock::CachePolicy; @@ -1040,7 +1041,7 @@ where if !self.is_dirty() { // If the state table is not modified, go fast path. self.local_store - .seal_current_epoch(new_epoch.curr, SealCurrentEpochOptions::new()); + .seal_current_epoch(new_epoch.curr, SealCurrentEpochOptions::no_watermark()); return Ok(()); } else { self.seal_current_epoch(new_epoch.curr) @@ -1109,7 +1110,7 @@ where // per epoch. self.watermark_buffer_strategy.tick(); self.local_store - .seal_current_epoch(new_epoch.curr, SealCurrentEpochOptions::new()); + .seal_current_epoch(new_epoch.curr, SealCurrentEpochOptions::no_watermark()); } /// Write to state store. @@ -1119,8 +1120,6 @@ where trace!(table_id = %self.table_id, watermark = ?watermark, "state cleaning"); }); - let mut delete_ranges = Vec::new(); - let prefix_serializer = if self.pk_indices().is_empty() { None } else { @@ -1156,10 +1155,11 @@ where ) }); + let mut seal_watermark: Option<(WatermarkDirection, VnodeWatermark)> = None; + // Compute Delete Ranges if should_clean_watermark && let Some(watermark_suffix) = watermark_suffix - && let Some(first_byte) = watermark_suffix.first() { trace!(table_id = %self.table_id, watermark = ?watermark_suffix, vnodes = ?{ self.vnodes.iter_vnodes().collect_vec() @@ -1172,36 +1172,21 @@ where .unwrap() .is_ascending() { - // We either serialize null into `0u8`, data into `(1u8 || scalar)`, or serialize null - // into `1u8`, data into `(0u8 || scalar)`. We do not want to delete null - // here, so `range_begin_suffix` cannot be `vec![]` when null is represented as `0u8`. - let range_begin_suffix = vec![*first_byte]; - for vnode in self.vnodes.iter_vnodes() { - let mut range_begin = vnode.to_be_bytes().to_vec(); - let mut range_end = range_begin.clone(); - range_begin.extend(&range_begin_suffix); - range_end.extend(&watermark_suffix); - delete_ranges.push(( - Bound::Included(Bytes::from(range_begin)), - Bound::Excluded(Bytes::from(range_end)), - )); - } + seal_watermark = Some(( + WatermarkDirection::Ascending, + VnodeWatermark::new( + self.vnodes.clone(), + Bytes::copy_from_slice(watermark_suffix.as_ref()) + ) + )); } else { - assert_ne!(*first_byte, u8::MAX); - let following_bytes = next_key(&watermark_suffix[1..]); - if !following_bytes.is_empty() { - for vnode in self.vnodes.iter_vnodes() { - let mut range_begin = vnode.to_be_bytes().to_vec(); - let mut range_end = range_begin.clone(); - range_begin.push(*first_byte); - range_begin.extend(&following_bytes); - range_end.push(first_byte + 1); - delete_ranges.push(( - Bound::Included(Bytes::from(range_begin)), - Bound::Excluded(Bytes::from(range_end)), - )); - } - } + seal_watermark = Some(( + WatermarkDirection::Descending, + VnodeWatermark::new( + self.vnodes.clone(), + Bytes::copy_from_slice(watermark_suffix.as_ref()) + ) + )); } } self.prev_cleaned_watermark = watermark; @@ -1213,13 +1198,18 @@ where // 2. Mark the cache as not_synced, so we can still refill it later. // 3. When refilling the cache, // we just refill from the largest value of the cache, as the lower bound. - if USE_WATERMARK_CACHE && !delete_ranges.is_empty() { + if USE_WATERMARK_CACHE && seal_watermark.is_some() { self.watermark_cache.clear(); } - self.local_store.flush(delete_ranges).await?; - self.local_store - .seal_current_epoch(next_epoch, SealCurrentEpochOptions::new()); + self.local_store.flush(vec![]).await?; + let seal_opt = match seal_watermark { + Some((direction, watermark)) => { + SealCurrentEpochOptions::new(vec![watermark], direction) + } + None => SealCurrentEpochOptions::no_watermark(), + }; + self.local_store.seal_current_epoch(next_epoch, seal_opt); Ok(()) } From 38b6ba7e5d496e3e8e2517dfc9bb95839abbbe75 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 26 Dec 2023 11:02:56 +0800 Subject: [PATCH 03/15] chore(deps): Bump xorf from 0.10.2 to 0.11.0 (#14194) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- Cargo.lock | 4 ++-- src/storage/Cargo.toml | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 0c8b66655c0ec..26c5c012368ae 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -12480,9 +12480,9 @@ checksum = "4d25c75bf9ea12c4040a97f829154768bbbce366287e2dc044af160cd79a13fd" [[package]] name = "xorf" -version = "0.10.2" +version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7d36478bcf71152a2f9f6cf9bc48273333f32780c769ef90e13d464ab778db5f" +checksum = "cf24c008fe464f5d8f58b8d16a1ab7e930bd73b2a6933ff8704c414b2bed7f92" dependencies = [ "libm", "rand", diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index e1cff95c761ea..bb135641dfa26 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -70,7 +70,7 @@ tokio-retry = "0.3" tonic = { workspace = true } tracing = "0.1" tracing-futures = { version = "0.2", features = ["futures-03"] } -xorf = "0.10.2" +xorf = "0.11.0" xxhash-rust = { version = "0.8.7", features = ["xxh32", "xxh64"] } zstd = { version = "0.13", default-features = false } From 01bbc0812ef9df8f17f0bf8d2d28dc27c01d0615 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 26 Dec 2023 12:12:07 +0800 Subject: [PATCH 04/15] refactor(source): remove unnecessary Box for NexmarkProperties (#14183) --- src/connector/src/source/nexmark/mod.rs | 12 +++++------- src/connector/src/source/nexmark/source/reader.rs | 8 ++++---- src/connector/with_options_source.yaml | 2 +- 3 files changed, 10 insertions(+), 12 deletions(-) diff --git a/src/connector/src/source/nexmark/mod.rs b/src/connector/src/source/nexmark/mod.rs index 474eeaf6f0619..0c63a96dfe683 100644 --- a/src/connector/src/source/nexmark/mod.rs +++ b/src/connector/src/source/nexmark/mod.rs @@ -43,11 +43,9 @@ const fn none() -> Option { None } -pub type NexmarkProperties = Box; - #[serde_as] #[derive(Clone, Debug, Deserialize, WithOptions)] -pub struct NexmarkPropertiesInner { +pub struct NexmarkProperties { #[serde_as(as = "DisplayFromStr")] #[serde(rename = "nexmark.split.num", default = "identity_i32::<1>")] pub split_num: i32, @@ -233,15 +231,15 @@ fn default_event_num() -> u64 { u64::MAX } -impl Default for NexmarkPropertiesInner { +impl Default for NexmarkProperties { fn default() -> Self { let v = serde_json::to_value(HashMap::::new()).unwrap(); - NexmarkPropertiesInner::deserialize(v).unwrap() + NexmarkProperties::deserialize(v).unwrap() } } -impl From<&NexmarkPropertiesInner> for NexmarkConfig { - fn from(value: &NexmarkPropertiesInner) -> Self { +impl From<&NexmarkProperties> for NexmarkConfig { + fn from(value: &NexmarkProperties) -> Self { // 2015-07-15 00:00:00 pub const BASE_TIME: u64 = 1_436_918_400_000; diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index 28dc76eadeb48..c8e7b0c41d7ea 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -77,7 +77,7 @@ impl SplitReader for NexmarkSplitReader { let offset = split.start_offset.unwrap_or(split_index); let assigned_split = split; - let mut generator = EventGenerator::new(NexmarkConfig::from(&*properties)) + let mut generator = EventGenerator::new(NexmarkConfig::from(&properties)) .with_offset(offset) .with_step(split_num); // If the user doesn't specify the event type in the source definition, then the user @@ -188,18 +188,18 @@ mod tests { use anyhow::Result; use super::*; - use crate::source::nexmark::{NexmarkPropertiesInner, NexmarkSplitEnumerator}; + use crate::source::nexmark::{NexmarkProperties, NexmarkSplitEnumerator}; use crate::source::{SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] async fn test_nexmark_split_reader() -> Result<()> { - let props = Box::new(NexmarkPropertiesInner { + let props = NexmarkProperties { split_num: 2, min_event_gap_in_ns: 0, table_type: Some(EventType::Bid), max_chunk_size: 5, ..Default::default() - }); + }; let mut enumerator = NexmarkSplitEnumerator::new(props.clone(), SourceEnumeratorContext::default().into()) diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 25e5ade6567d9..586ff624611ae 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -225,7 +225,7 @@ NatsProperties: - name: stream field_type: String required: true -NexmarkPropertiesInner: +NexmarkProperties: fields: - name: nexmark.split.num field_type: i32 From 94c4c9112bd256e78a2af66147b056100126c7e2 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Dec 2023 14:40:59 +0800 Subject: [PATCH 05/15] refactor(storage): replace PbHummockVersion with new HummockVersion struct (#14101) --- Cargo.lock | 4 + .../cmd_impl/hummock/list_version_deltas.rs | 2 +- src/ctl/src/cmd_impl/hummock/pause_resume.rs | 18 +- src/ctl/src/cmd_impl/hummock/sst_dump.rs | 3 +- src/frontend/Cargo.toml | 1 + .../rw_catalog/rw_hummock_version.rs | 2 +- src/frontend/src/meta_client.rs | 10 +- src/frontend/src/test_utils.rs | 3 +- src/jni_core/src/hummock_iterator.rs | 6 +- src/meta/model_v2/Cargo.toml | 1 + .../model_v2/src/hummock_version_delta.rs | 6 +- src/meta/service/Cargo.toml | 1 + src/meta/service/src/hummock_service.rs | 35 +-- src/meta/service/src/notification_service.rs | 2 +- .../backup_restore/meta_snapshot_builder.rs | 11 +- .../meta_snapshot_builder_v2.rs | 11 +- src/meta/src/backup_restore/restore.rs | 10 +- src/meta/src/barrier/mod.rs | 21 +- src/meta/src/hummock/manager/checkpoint.rs | 58 ++-- .../manager/compaction_group_manager.rs | 12 +- src/meta/src/hummock/manager/context.rs | 3 +- src/meta/src/hummock/manager/gc.rs | 5 +- src/meta/src/hummock/manager/mod.rs | 45 ++- src/meta/src/hummock/manager/tests.rs | 26 +- src/meta/src/hummock/manager/versioning.rs | 11 +- src/meta/src/hummock/metrics_utils.rs | 12 +- .../src/hummock/mock_hummock_meta_client.rs | 9 +- src/meta/src/hummock/model/ext/hummock.rs | 7 +- src/meta/src/hummock/model/version_delta.rs | 11 +- src/meta/src/hummock/test_utils.rs | 5 +- src/meta/src/model/mod.rs | 3 +- src/rpc_client/src/hummock_meta_client.rs | 4 +- src/rpc_client/src/meta_client.rs | 44 +-- src/storage/backup/src/lib.rs | 3 +- src/storage/backup/src/meta_snapshot.rs | 2 +- src/storage/backup/src/meta_snapshot_v1.rs | 8 +- src/storage/backup/src/meta_snapshot_v2.rs | 8 +- src/storage/hummock_sdk/Cargo.toml | 1 + .../compaction_group/hummock_version_ext.rs | 49 ++-- src/storage/hummock_sdk/src/lib.rs | 1 + src/storage/hummock_sdk/src/table_stats.rs | 4 +- .../hummock_sdk/src/table_watermark.rs | 259 +++++++++--------- src/storage/hummock_sdk/src/version.rs | 192 +++++++++++++ .../hummock_test/src/compactor_tests.rs | 4 +- .../src/mock_notification_client.rs | 2 +- .../hummock_test/src/sync_point_tests.rs | 1 - src/storage/hummock_test/src/test_utils.rs | 16 +- .../event_handler/hummock_event_handler.rs | 14 +- src/storage/src/hummock/event_handler/mod.rs | 10 +- .../src/hummock/event_handler/uploader.rs | 3 +- .../src/hummock/hummock_meta_client.rs | 5 +- .../hummock/local_version/pinned_version.rs | 13 +- src/storage/src/hummock/observer_manager.rs | 18 +- .../src/hummock/store/hummock_storage.rs | 13 +- src/storage/src/hummock/store/version.rs | 6 +- src/storage/src/hummock/utils.rs | 3 +- .../src/compaction_test_runner.rs | 5 +- .../src/delete_range_runner.rs | 2 +- 58 files changed, 654 insertions(+), 390 deletions(-) create mode 100644 src/storage/hummock_sdk/src/version.rs diff --git a/Cargo.lock b/Cargo.lock index 26c5c012368ae..6055509d0a06a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8714,6 +8714,7 @@ dependencies = [ "risingwave_connector", "risingwave_expr", "risingwave_expr_impl", + "risingwave_hummock_sdk", "risingwave_pb", "risingwave_rpc_client", "risingwave_source", @@ -8744,6 +8745,7 @@ dependencies = [ "hex", "itertools 0.12.0", "parse-display", + "prost 0.12.1", "risingwave_common", "risingwave_pb", "tracing", @@ -8945,6 +8947,7 @@ dependencies = [ name = "risingwave_meta_model_v2" version = "1.5.0-alpha" dependencies = [ + "risingwave_hummock_sdk", "risingwave_pb", "sea-orm", "serde", @@ -8996,6 +8999,7 @@ dependencies = [ "regex", "risingwave_common", "risingwave_connector", + "risingwave_hummock_sdk", "risingwave_meta", "risingwave_meta_model_v2", "risingwave_pb", diff --git a/src/ctl/src/cmd_impl/hummock/list_version_deltas.rs b/src/ctl/src/cmd_impl/hummock/list_version_deltas.rs index 7e56078e77ed2..02a85a94953c0 100644 --- a/src/ctl/src/cmd_impl/hummock/list_version_deltas.rs +++ b/src/ctl/src/cmd_impl/hummock/list_version_deltas.rs @@ -25,6 +25,6 @@ pub async fn list_version_deltas( let resp = meta_client .list_version_deltas(start_id, num_epochs, HummockEpoch::MAX) .await?; - println!("{:#?}", resp.version_deltas); + println!("{:#?}", resp); Ok(()) } diff --git a/src/ctl/src/cmd_impl/hummock/pause_resume.rs b/src/ctl/src/cmd_impl/hummock/pause_resume.rs index d599ce2327861..bc63144547c9a 100644 --- a/src/ctl/src/cmd_impl/hummock/pause_resume.rs +++ b/src/ctl/src/cmd_impl/hummock/pause_resume.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::HummockEpoch; use crate::CtlContext; @@ -51,11 +51,13 @@ pub async fn resume_version_checkpoint(context: &CtlContext) -> anyhow::Result<( /// added/removed for what reason (row deletion/compaction/etc.). pub async fn replay_version(context: &CtlContext) -> anyhow::Result<()> { let meta_client = context.meta_client().await?; - let mut base_version = meta_client - .risectl_get_checkpoint_hummock_version() - .await? - .checkpoint_version - .unwrap(); + let mut base_version = HummockVersion::from_rpc_protobuf( + &meta_client + .risectl_get_checkpoint_hummock_version() + .await? + .checkpoint_version + .unwrap(), + ); println!("replay starts"); println!("base version {}", base_version.id); let delta_fetch_size = 100; @@ -65,10 +67,10 @@ pub async fn replay_version(context: &CtlContext) -> anyhow::Result<()> { .list_version_deltas(current_delta_id, delta_fetch_size, HummockEpoch::MAX) .await .unwrap(); - if deltas.version_deltas.is_empty() { + if deltas.is_empty() { break; } - for delta in deltas.version_deltas { + for delta in deltas { if delta.prev_id != base_version.id { eprintln!("missing delta log for version {}", base_version.id); break; diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs index 7ed529ec02834..3843dd65f4ad5 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -28,7 +28,6 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde, ValueRowDeserializer}; use risingwave_frontend::TableCatalog; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_object_store::object::{ObjectMetadata, ObjectStoreImpl}; @@ -75,7 +74,7 @@ pub async fn sst_dump(context: &CtlContext, args: SstDumpArgs) -> anyhow::Result let hummock = context .hummock_store(HummockServiceOpts::from_env(args.data_dir.clone())?) .await?; - let version = hummock.inner().get_pinned_version().version(); + let version = hummock.inner().get_pinned_version().version().clone(); let sstable_store = hummock.sstable_store(); for level in version.get_combined_levels() { for sstable_info in &level.table_infos { diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 73df1156cd035..e279ccb38ba69 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -56,6 +56,7 @@ risingwave_common = { workspace = true } risingwave_common_service = { workspace = true } risingwave_connector = { workspace = true } risingwave_expr = { workspace = true } +risingwave_hummock_sdk = { workspace = true } risingwave_pb = { workspace = true } risingwave_rpc_client = { workspace = true } risingwave_source = { workspace = true } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index 97269341d59f3..0c62a61cab738 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -16,7 +16,7 @@ use risingwave_common::catalog::RW_CATALOG_SCHEMA_NAME; use risingwave_common::error::Result; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; -use risingwave_pb::hummock::HummockVersion; +use risingwave_hummock_sdk::version::HummockVersion; use serde_json::json; use crate::catalog::system_catalog::{BuiltinTable, SysCatalogReaderImpl}; diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index ea382a03e2c5a..c7753dd20e01a 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -15,6 +15,7 @@ use std::collections::HashMap; use risingwave_common::system_param::reader::SystemParamsReader; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::Table; use risingwave_pb::common::WorkerNode; @@ -22,7 +23,7 @@ use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ BranchedObject, CompactTaskAssignment, CompactTaskProgress, CompactionGroupInfo, - HummockSnapshot, HummockVersion, HummockVersionDelta, + HummockSnapshot, }; use risingwave_pb::meta::cancel_creating_jobs_request::PbJobs; use risingwave_pb::meta::list_actor_states_response::ActorState; @@ -223,15 +224,12 @@ impl FrontendMetaClient for FrontendMetaClientImpl { self.0 .risectl_get_checkpoint_hummock_version() .await - .map(|v| v.checkpoint_version.unwrap()) + .map(|v| HummockVersion::from_rpc_protobuf(&v.checkpoint_version.unwrap())) } async fn list_version_deltas(&self) -> Result> { // FIXME #8612: there can be lots of version deltas, so better to fetch them by pages and refactor `SysRowSeqScanExecutor` to yield multiple chunks. - self.0 - .list_version_deltas(0, u32::MAX, u64::MAX) - .await - .map(|v| v.version_deltas) + self.0.list_version_deltas(0, u32::MAX, u64::MAX).await } async fn list_branched_objects(&self) -> Result> { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 2bfd561f14a0f..819542c431a38 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -31,6 +31,7 @@ use risingwave_common::catalog::{ use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::backup_service::MetaSnapshotMetadata; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ @@ -45,7 +46,7 @@ use risingwave_pb::ddl_service::{ use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ BranchedObject, CompactTaskAssignment, CompactTaskProgress, CompactionGroupInfo, - HummockSnapshot, HummockVersion, HummockVersionDelta, + HummockSnapshot, }; use risingwave_pb::meta::cancel_creating_jobs_request::PbJobs; use risingwave_pb::meta::list_actor_states_response::ActorState; diff --git a/src/jni_core/src/hummock_iterator.rs b/src/jni_core/src/hummock_iterator.rs index d7fa7533c4738..e0771ef973084 100644 --- a/src/jni_core/src/hummock_iterator.rs +++ b/src/jni_core/src/hummock_iterator.rs @@ -24,6 +24,7 @@ use risingwave_common::util::select_all; use risingwave_common::util::value_encoding::column_aware_row_encoding::ColumnAwareSerde; use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde, ValueRowDeserializer}; use risingwave_hummock_sdk::key::{prefixed_range_with_vnode, TableKeyRange}; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_object_store::object::build_remote_object_store; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; use risingwave_pb::java_binding::key_range::Bound; @@ -84,7 +85,10 @@ impl HummockJavaBindingIterator { let mut streams = Vec::with_capacity(read_plan.vnode_ids.len()); let key_range = read_plan.key_range.unwrap(); - let pin_version = PinnedVersion::new(read_plan.version.unwrap(), unbounded_channel().0); + let pin_version = PinnedVersion::new( + HummockVersion::from_rpc_protobuf(&read_plan.version.unwrap()), + unbounded_channel().0, + ); let table_id = read_plan.table_id.into(); for vnode in read_plan.vnode_ids { diff --git a/src/meta/model_v2/Cargo.toml b/src/meta/model_v2/Cargo.toml index 1d9992da8a832..f080645fc1c6a 100644 --- a/src/meta/model_v2/Cargo.toml +++ b/src/meta/model_v2/Cargo.toml @@ -14,6 +14,7 @@ ignored = ["workspace-hack"] normal = ["workspace-hack"] [dependencies] +risingwave_hummock_sdk = { workspace = true } risingwave_pb = { workspace = true } sea-orm = { version = "0.12.0", features = [ "sqlx-mysql", diff --git a/src/meta/model_v2/src/hummock_version_delta.rs b/src/meta/model_v2/src/hummock_version_delta.rs index f59f0c6f95455..ee7f5120b28a7 100644 --- a/src/meta/model_v2/src/hummock_version_delta.rs +++ b/src/meta/model_v2/src/hummock_version_delta.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_pb::hummock::HummockVersionDelta; +use risingwave_pb::hummock::PbHummockVersionDelta; use sea_orm::entity::prelude::*; use sea_orm::FromJsonQueryResult; use serde::{Deserialize, Serialize}; @@ -36,9 +36,9 @@ pub enum Relation {} impl ActiveModelBehavior for ActiveModel {} -crate::derive_from_json_struct!(FullVersionDelta, HummockVersionDelta); +crate::derive_from_json_struct!(FullVersionDelta, PbHummockVersionDelta); -impl From for HummockVersionDelta { +impl From for PbHummockVersionDelta { fn from(value: Model) -> Self { let ret = value.full_version_delta.into_inner(); assert_eq!(value.id, ret.id as i64); diff --git a/src/meta/service/Cargo.toml b/src/meta/service/Cargo.toml index b734b62106495..c9b6619565cd7 100644 --- a/src/meta/service/Cargo.toml +++ b/src/meta/service/Cargo.toml @@ -23,6 +23,7 @@ rand = "0.8" regex = "1" risingwave_common = { workspace = true } risingwave_connector = { workspace = true } +risingwave_hummock_sdk = { workspace = true } risingwave_meta = { workspace = true } risingwave_meta_model_v2 = { workspace = true } risingwave_pb = { workspace = true } diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index 835032769266f..f5a7e5b7353b1 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -18,10 +18,10 @@ use std::time::Duration; use futures::StreamExt; use itertools::Itertools; use risingwave_common::catalog::{TableId, NON_RESERVED_SYS_CATALOG_ID}; +use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_pb::hummock::get_compaction_score_response::PickerInfo; use risingwave_pb::hummock::hummock_manager_service_server::HummockManagerService; use risingwave_pb::hummock::subscribe_compaction_event_request::Event as RequestEvent; -use risingwave_pb::hummock::version_update_payload::Payload; use risingwave_pb::hummock::*; use tonic::{Request, Response, Status, Streaming}; @@ -29,6 +29,7 @@ use crate::hummock::compaction::selector::ManualCompactionOption; use crate::hummock::{HummockManagerRef, VacuumManagerRef}; use crate::manager::FragmentManagerRef; use crate::RwReceiverStream; + pub struct HummockServiceImpl { hummock_manager: HummockManagerRef, vacuum_manager: VacuumManagerRef, @@ -83,7 +84,7 @@ impl HummockManagerService for HummockServiceImpl { let current_version = self.hummock_manager.get_current_version().await; Ok(Response::new(GetCurrentVersionResponse { status: None, - current_version: Some(current_version), + current_version: Some(current_version.to_protobuf()), })) } @@ -94,10 +95,12 @@ impl HummockManagerService for HummockServiceImpl { let req = request.into_inner(); let (version, compaction_groups) = self .hummock_manager - .replay_version_delta(req.version_delta.unwrap()) + .replay_version_delta(HummockVersionDelta::from_rpc_protobuf( + &req.version_delta.unwrap(), + )) .await?; Ok(Response::new(ReplayVersionDeltaResponse { - version: Some(version), + version: Some(version.to_protobuf()), modified_compaction_groups: compaction_groups, })) } @@ -119,7 +122,7 @@ impl HummockManagerService for HummockServiceImpl { ) -> Result, Status> { let version = self.hummock_manager.disable_commit_epoch().await; Ok(Response::new(DisableCommitEpochResponse { - current_version: Some(version), + current_version: Some(version.to_protobuf()), })) } @@ -133,7 +136,12 @@ impl HummockManagerService for HummockServiceImpl { .list_version_deltas(req.start_id, req.num_limit, req.committed_epoch_limit) .await?; let resp = ListVersionDeltasResponse { - version_deltas: Some(version_deltas), + version_deltas: Some(PbHummockVersionDeltas { + version_deltas: version_deltas + .iter() + .map(HummockVersionDelta::to_protobuf) + .collect(), + }), }; Ok(Response::new(resp)) } @@ -415,15 +423,10 @@ impl HummockManagerService for HummockServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let payload = self.hummock_manager.pin_version(req.context_id).await?; - match payload { - Payload::PinnedVersion(version) => Ok(Response::new(PinVersionResponse { - pinned_version: Some(version), - })), - Payload::VersionDeltas(_) => { - unreachable!("pin_version should not return version delta") - } - } + let version = self.hummock_manager.pin_version(req.context_id).await?; + Ok(Response::new(PinVersionResponse { + pinned_version: Some(version.to_protobuf()), + })) } async fn split_compaction_group( @@ -460,7 +463,7 @@ impl HummockManagerService for HummockServiceImpl { ) -> Result, Status> { let checkpoint_version = self.hummock_manager.get_checkpoint_version().await; Ok(Response::new(RiseCtlGetCheckpointVersionResponse { - checkpoint_version: Some(checkpoint_version), + checkpoint_version: Some(checkpoint_version.to_protobuf()), })) } diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index b7b63dcc6c164..1bd0be6551604 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -183,7 +183,7 @@ impl NotificationServiceImpl { MetaSnapshot { tables, - hummock_version: Some(hummock_version), + hummock_version: Some(hummock_version.to_protobuf()), version: Some(SnapshotVersion { catalog_version, ..Default::default() diff --git a/src/meta/src/backup_restore/meta_snapshot_builder.rs b/src/meta/src/backup_restore/meta_snapshot_builder.rs index 0d3f37155a047..9d4294579d0dc 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder.rs @@ -19,11 +19,11 @@ use anyhow::anyhow; use risingwave_backup::error::{BackupError, BackupResult}; use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; use risingwave_backup::MetaSnapshotId; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::catalog::{ Connection, Database, Function, Index, Schema, Sink, Source, Table, View, }; -use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta, HummockVersionStats}; +use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::SystemParams; use risingwave_pb::user::UserInfo; @@ -46,10 +46,10 @@ impl MetaSnapshotV1Builder { } } - pub async fn build>( + pub async fn build( &mut self, id: MetaSnapshotId, - hummock_version_builder: D, + hummock_version_builder: impl Future, ) -> BackupResult<()> { self.snapshot.format_version = VERSION; self.snapshot.id = id; @@ -169,7 +169,8 @@ mod tests { use risingwave_backup::error::BackupError; use risingwave_backup::meta_snapshot_v1::MetaSnapshotV1; use risingwave_common::system_param::system_params_for_test; - use risingwave_pb::hummock::{HummockVersion, HummockVersionStats}; + use risingwave_hummock_sdk::version::HummockVersion; + use risingwave_pb::hummock::HummockVersionStats; use crate::backup_restore::meta_snapshot_builder; use crate::manager::model::SystemParamsModel; diff --git a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs index 644b581f03e30..3cb4a39b799c8 100644 --- a/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs +++ b/src/meta/src/backup_restore/meta_snapshot_builder_v2.rs @@ -20,9 +20,9 @@ use itertools::Itertools; use risingwave_backup::error::{BackupError, BackupResult}; use risingwave_backup::meta_snapshot_v2::{MetaSnapshotV2, MetadataV2}; use risingwave_backup::MetaSnapshotId; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_meta_model_v2 as model_v2; -use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta}; +use risingwave_pb::hummock::PbHummockVersionDelta; use sea_orm::{EntityTrait, QueryOrder, TransactionTrait}; use crate::controller::SqlMetaStore; @@ -42,10 +42,10 @@ impl MetaSnapshotV2Builder { } } - pub async fn build>( + pub async fn build( &mut self, id: MetaSnapshotId, - hummock_version_builder: D, + hummock_version_builder: impl Future, ) -> BackupResult<()> { self.snapshot.format_version = VERSION; self.snapshot.id = id; @@ -68,7 +68,8 @@ impl MetaSnapshotV2Builder { .await .map_err(|e| BackupError::MetaStorage(e.into()))? .into_iter() - .map_into::(); + .map_into::() + .map(|pb_delta| HummockVersionDelta::from_persisted_protobuf(&pb_delta)); let hummock_version = { let mut redo_state = hummock_version; let mut max_log_id = None; diff --git a/src/meta/src/backup_restore/restore.rs b/src/meta/src/backup_restore/restore.rs index 3e2d36eb86985..877f5d1a454a9 100644 --- a/src/meta/src/backup_restore/restore.rs +++ b/src/meta/src/backup_restore/restore.rs @@ -19,10 +19,11 @@ use risingwave_backup::meta_snapshot::Metadata; use risingwave_backup::storage::{MetaSnapshotStorage, MetaSnapshotStorageRef}; use risingwave_backup::MetaSnapshotId; use risingwave_common::config::{MetaBackend, ObjectStoreConfig}; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::version_checkpoint_path; use risingwave_object_store::object::build_remote_object_store; use risingwave_object_store::object::object_metrics::ObjectStoreMetrics; -use risingwave_pb::hummock::{HummockVersion, HummockVersionCheckpoint}; +use risingwave_pb::hummock::PbHummockVersionCheckpoint; use crate::backup_restore::restore_impl::v1::{LoaderV1, WriterModelV1ToMetaStoreV1}; use crate::backup_restore::restore_impl::v2::{LoaderV2, WriterModelV2ToMetaStoreV2}; @@ -83,8 +84,8 @@ async fn restore_hummock_version( .await, ); let checkpoint_path = version_checkpoint_path(hummock_storage_directory); - let checkpoint = HummockVersionCheckpoint { - version: Some(hummock_version.clone()), + let checkpoint = PbHummockVersionCheckpoint { + version: Some(hummock_version.to_protobuf()), // Ignore stale objects. Full GC will clear them. stale_objects: Default::default(), }; @@ -206,7 +207,8 @@ mod tests { use risingwave_backup::meta_snapshot_v1::{ClusterMetadata, MetaSnapshotV1}; use risingwave_backup::storage::MetaSnapshotStorage; use risingwave_common::config::{MetaBackend, SystemConfig}; - use risingwave_pb::hummock::{HummockVersion, HummockVersionStats}; + use risingwave_hummock_sdk::version::HummockVersion; + use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::SystemParams; use crate::backup_restore::restore::restore_impl; diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 334a2394de4fb..571caaa3dd6de 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -28,7 +28,9 @@ use risingwave_common::bail; use risingwave_common::catalog::TableId; use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY; use risingwave_common::util::tracing::TracingContext; -use risingwave_hummock_sdk::table_watermark::merge_multiple_new_table_watermarks; +use risingwave_hummock_sdk::table_watermark::{ + merge_multiple_new_table_watermarks, TableWatermarks, +}; use risingwave_hummock_sdk::{ExtendedSstableInfo, HummockSstableObjectId}; use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; @@ -1289,7 +1291,22 @@ fn collect_commit_epoch_info(resps: &mut [BarrierCompleteResponse]) -> CommitEpo } CommitEpochInfo::new( synced_ssts, - merge_multiple_new_table_watermarks(resps.iter().map(|resp| resp.table_watermarks.clone())), + merge_multiple_new_table_watermarks( + resps + .iter() + .map(|resp| { + resp.table_watermarks + .iter() + .map(|(table_id, watermarks)| { + ( + TableId::new(*table_id), + TableWatermarks::from_protobuf(watermarks), + ) + }) + .collect() + }) + .collect_vec(), + ), sst_to_worker, ) } diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index 6aa64292b9db1..523c0d35f9cd5 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::ops::Bound::{Excluded, Included}; use std::ops::{Deref, DerefMut}; use std::sync::atomic::Ordering; @@ -20,8 +21,10 @@ use function_name::named; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ object_size_map, summarize_group_deltas, }; -use risingwave_pb::hummock::hummock_version_checkpoint::StaleObjects; -use risingwave_pb::hummock::{HummockVersion, HummockVersionCheckpoint}; +use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_pb::hummock::hummock_version_checkpoint::{PbStaleObjects, StaleObjects}; +use risingwave_pb::hummock::PbHummockVersionCheckpoint; use crate::hummock::error::Result; use crate::hummock::manager::{read_lock, write_lock}; @@ -31,6 +34,32 @@ use crate::storage::{MetaStore, MetaStoreError, DEFAULT_COLUMN_FAMILY}; const HUMMOCK_INIT_FLAG_KEY: &[u8] = b"hummock_init_flag"; +#[derive(Default)] +pub struct HummockVersionCheckpoint { + pub version: HummockVersion, + pub stale_objects: HashMap, +} + +impl HummockVersionCheckpoint { + pub fn from_protobuf(checkpoint: &PbHummockVersionCheckpoint) -> Self { + Self { + version: HummockVersion::from_persisted_protobuf(checkpoint.version.as_ref().unwrap()), + stale_objects: checkpoint + .stale_objects + .iter() + .map(|(object_id, objects)| (*object_id as HummockSstableObjectId, objects.clone())) + .collect(), + } + } + + pub fn to_protobuf(&self) -> PbHummockVersionCheckpoint { + PbHummockVersionCheckpoint { + version: Some(self.version.to_protobuf()), + stale_objects: self.stale_objects.clone(), + } + } +} + /// A hummock version checkpoint compacts previous hummock version delta logs, and stores stale /// objects from those delta logs. impl HummockManager { @@ -54,8 +83,8 @@ impl HummockManager { return Err(e.into()); } }; - let ckpt = HummockVersionCheckpoint::decode(data).map_err(|e| anyhow::anyhow!(e))?; - Ok(ckpt) + let ckpt = PbHummockVersionCheckpoint::decode(data).map_err(|e| anyhow::anyhow!(e))?; + Ok(HummockVersionCheckpoint::from_protobuf(&ckpt)) } pub(super) async fn write_checkpoint( @@ -63,7 +92,7 @@ impl HummockManager { checkpoint: &HummockVersionCheckpoint, ) -> Result<()> { use prost::Message; - let buf = checkpoint.encode_to_vec(); + let buf = checkpoint.to_protobuf().encode_to_vec(); self.object_store .upload(&self.version_checkpoint_path, buf.into()) .await?; @@ -83,13 +112,13 @@ impl HummockManager { let current_version = &versioning.current_version; let old_checkpoint = &versioning.checkpoint; let new_checkpoint_id = current_version.id; - let old_checkpoint_id = old_checkpoint.version.as_ref().unwrap().id; + let old_checkpoint_id = old_checkpoint.version.id; if new_checkpoint_id < old_checkpoint_id + min_delta_log_num { return Ok(0); } let mut stale_objects = old_checkpoint.stale_objects.clone(); // `object_sizes` is used to calculate size of stale objects. - let mut object_sizes = object_size_map(old_checkpoint.version.as_ref().unwrap()); + let mut object_sizes = object_size_map(&old_checkpoint.version); for (_, version_delta) in versioning .hummock_version_deltas .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) @@ -120,7 +149,7 @@ impl HummockManager { ); } let new_checkpoint = HummockVersionCheckpoint { - version: Some(current_version.clone()), + version: current_version.clone(), stale_objects, }; drop(versioning_guard); @@ -129,11 +158,7 @@ impl HummockManager { // 3. hold write lock and update in memory state let mut versioning_guard = write_lock!(self, versioning).await; let versioning = versioning_guard.deref_mut(); - assert!( - versioning.checkpoint.version.is_none() - || new_checkpoint.version.as_ref().unwrap().id - >= versioning.checkpoint.version.as_ref().unwrap().id - ); + assert!(new_checkpoint.version.id >= versioning.checkpoint.version.id); versioning.checkpoint = new_checkpoint; versioning.mark_objects_for_deletion(); @@ -191,11 +216,6 @@ impl HummockManager { #[named] pub async fn get_checkpoint_version(&self) -> HummockVersion { let versioning_guard = read_lock!(self, versioning).await; - versioning_guard - .checkpoint - .version - .as_ref() - .unwrap() - .clone() + versioning_guard.checkpoint.version.clone() } } diff --git a/src/meta/src/hummock/manager/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction_group_manager.rs index 2389fe1538dc2..515723fcaddea 100644 --- a/src/meta/src/hummock/manager/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction_group_manager.rs @@ -18,10 +18,10 @@ use std::sync::Arc; use function_name::named; use itertools::Itertools; +use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ build_version_delta_after_version, get_compaction_group_ids, get_compaction_group_ssts, - get_member_table_ids, try_get_compaction_group_id_by_table_id, HummockVersionExt, - HummockVersionUpdateExt, TableGroupInfo, + get_member_table_ids, try_get_compaction_group_id_by_table_id, TableGroupInfo, }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::CompactionGroupId; @@ -44,7 +44,7 @@ use crate::hummock::error::{Error, Result}; use crate::hummock::manager::{drop_sst, read_lock, HummockManager}; use crate::hummock::metrics_utils::remove_compaction_group_in_sst_stat; use crate::hummock::model::CompactionGroup; -use crate::manager::{IdCategory, MetaSrvEnv, TableId}; +use crate::manager::{IdCategory, MetaSrvEnv}; use crate::model::{ BTreeMapEntryTransaction, BTreeMapTransaction, MetadataModel, TableFragments, ValTransaction, }; @@ -306,7 +306,9 @@ impl HummockManager { .len() as u64 - 1, ); - new_version_delta.removed_table_ids.push(*table_id); + new_version_delta + .removed_table_ids + .push(TableId::new(*table_id)); } // Remove empty group, GC SSTs and remove metric. @@ -462,7 +464,7 @@ impl HummockManager { table_ids: &[StateTableId], target_group_id: Option, partition_vnode_count: u32, - ) -> Result<(CompactionGroupId, BTreeMap)> { + ) -> Result<(CompactionGroupId, BTreeMap)> { let mut table_to_partition = BTreeMap::default(); if table_ids.is_empty() { return Ok((parent_group_id, table_to_partition)); diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index b069a31ce5bd3..f1b086720049a 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -18,10 +18,11 @@ use std::ops::DerefMut; use fail::fail_point; use function_name::named; use itertools::Itertools; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ ExtendedSstableInfo, HummockContextId, HummockEpoch, HummockSstableObjectId, }; -use risingwave_pb::hummock::{HummockVersion, ValidationTask}; +use risingwave_pb::hummock::ValidationTask; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::{ diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index 837564d6e6477..bfa396729aabb 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -20,7 +20,6 @@ use std::time::Duration; use function_name::named; use futures::{stream, StreamExt}; use itertools::Itertools; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_pb::common::worker_node::State::Running; use risingwave_pb::common::WorkerType; @@ -72,7 +71,7 @@ impl HummockManager { let versioning = versioning_guard.deref_mut(); let deltas_to_delete = versioning .hummock_version_deltas - .range(..=versioning.checkpoint.version.as_ref().unwrap().id) + .range(..=versioning.checkpoint.version.id) .map(|(k, _)| *k) .collect_vec(); // If there is any safe point, skip this to ensure meta backup has required delta logs to @@ -116,7 +115,7 @@ impl HummockManager { let mut tracked_object_ids = HashSet::from_iter(versioning_guard.current_version.get_object_ids()); for delta in versioning_guard.hummock_version_deltas.values() { - tracked_object_ids.extend(delta.get_gc_object_ids()); + tracked_object_ids.extend(delta.gc_object_ids.iter().cloned()); } tracked_object_ids }; diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 20cb4a4765dc1..db4c1c5d50b87 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -37,8 +37,9 @@ use risingwave_hummock_sdk::compact::{compact_task_to_string, statistics_compact use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ build_version_delta_after_version, get_compaction_group_ids, get_table_compaction_group_id_mapping, try_get_compaction_group_id_by_table_id, - BranchedSstInfo, HummockLevelsExt, HummockVersionExt, HummockVersionUpdateExt, + BranchedSstInfo, HummockLevelsExt, }; +use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_hummock_sdk::{ version_checkpoint_path, CompactionGroupId, ExtendedSstableInfo, HummockCompactionTaskId, HummockContextId, HummockEpoch, HummockSstableId, HummockSstableObjectId, HummockVersionId, @@ -54,10 +55,9 @@ use risingwave_pb::hummock::subscribe_compaction_event_response::{ Event as ResponseEvent, PullTaskAck, }; use risingwave_pb::hummock::{ - version_update_payload, CompactTask, CompactTaskAssignment, CompactionConfig, GroupDelta, - HummockPinnedSnapshot, HummockPinnedVersion, HummockSnapshot, HummockVersion, - HummockVersionCheckpoint, HummockVersionDelta, HummockVersionDeltas, HummockVersionStats, - IntraLevelDelta, SstableInfo, SubscribeCompactionEventRequest, TableOption, TableWatermarks, + CompactTask, CompactTaskAssignment, CompactionConfig, GroupDelta, HummockPinnedSnapshot, + HummockPinnedVersion, HummockSnapshot, HummockVersionStats, IntraLevelDelta, + PbCompactionGroupInfo, SstableInfo, SubscribeCompactionEventRequest, TableOption, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -99,7 +99,7 @@ mod tests; mod versioning; pub use versioning::HummockVersionSafePoint; use versioning::*; -mod checkpoint; +pub(crate) mod checkpoint; mod compaction; mod worker; @@ -211,8 +211,6 @@ use risingwave_hummock_sdk::table_stats::{ use risingwave_object_store::object::{build_remote_object_store, ObjectError, ObjectStoreRef}; use risingwave_pb::catalog::Table; use risingwave_pb::hummock::level_handler::RunningCompactTask; -use risingwave_pb::hummock::version_update_payload::Payload; -use risingwave_pb::hummock::PbCompactionGroupInfo; use risingwave_pb::meta::relation::RelationInfo; /// Acquire write lock of the lock with `lock_name`. @@ -268,14 +266,14 @@ pub enum CompactionResumeTrigger { pub struct CommitEpochInfo { pub sstables: Vec, - pub new_table_watermarks: HashMap, + pub new_table_watermarks: HashMap, pub sst_to_context: HashMap, } impl CommitEpochInfo { pub fn new( sstables: Vec, - new_table_watermarks: HashMap, + new_table_watermarks: HashMap, sst_to_context: HashMap, ) -> Self { Self { @@ -502,7 +500,7 @@ impl HummockManager { .insert(self.env.meta_store()) .await?; versioning_guard.checkpoint = HummockVersionCheckpoint { - version: Some(checkpoint_version.clone()), + version: checkpoint_version.clone(), stale_objects: Default::default(), }; self.write_checkpoint(&versioning_guard.checkpoint).await?; @@ -511,12 +509,7 @@ impl HummockManager { } else { // Read checkpoint from object store. versioning_guard.checkpoint = self.read_checkpoint().await?; - versioning_guard - .checkpoint - .version - .as_ref() - .cloned() - .unwrap() + versioning_guard.checkpoint.version.clone() }; versioning_guard.version_stats = HummockVersionStats::list(self.env.meta_store()) .await? @@ -585,10 +578,7 @@ impl HummockManager { /// Pin the current greatest hummock version. The pin belongs to `context_id` /// and will be unpinned when `context_id` is invalidated. #[named] - pub async fn pin_version( - &self, - context_id: HummockContextId, - ) -> Result { + pub async fn pin_version(&self, context_id: HummockContextId) -> Result { let mut versioning_guard = write_lock!(self, versioning).await; let _timer = start_measure_real_process_timer!(self); let versioning = versioning_guard.deref_mut(); @@ -601,7 +591,7 @@ impl HummockManager { }, ); let version_id = versioning.current_version.id; - let ret = Payload::PinnedVersion(versioning.current_version.clone()); + let ret = versioning.current_version.clone(); if context_pinned_version.min_pinned_id == INVALID_VERSION_ID || context_pinned_version.min_pinned_id > version_id { @@ -1786,7 +1776,7 @@ impl HummockManager { start_id: u64, num_limit: u32, committed_epoch_limit: HummockEpoch, - ) -> Result { + ) -> Result> { let versioning = read_lock!(self, versioning).await; let version_deltas = versioning .hummock_version_deltas @@ -1796,7 +1786,7 @@ impl HummockManager { .take(num_limit as _) .cloned() .collect(); - Ok(HummockVersionDeltas { version_deltas }) + Ok(version_deltas) } pub async fn init_metadata_for_version_replay( @@ -2047,7 +2037,7 @@ impl HummockManager { .last_key_value() .unwrap() .1 - .clone()], + .to_protobuf()], }), ); } @@ -2990,7 +2980,7 @@ impl HummockManager { rewrite_cg_ids.push(*cg_id); } - if let Some(levels) = current_version.get_levels().get(cg_id) { + if let Some(levels) = current_version.levels.get(cg_id) { if levels.member_table_ids.len() == 1 { restore_cg_to_partition_vnode.insert( *cg_id, @@ -3225,10 +3215,13 @@ fn init_selectors() -> HashMap { - unreachable!("should get full version") - } - Payload::PinnedVersion(version) => version, - }; + let version = hummock_manager.pin_version(context_id_1).await.unwrap(); assert_eq!( - version.get_id(), + version.id, init_version_id + commit_log_count + register_log_count ); assert_eq!( @@ -553,14 +548,9 @@ async fn test_hummock_manager_basic() { for _ in 0..2 { // should pin latest because deltas cannot contain INVALID_EPOCH - let version = match hummock_manager.pin_version(context_id_2).await.unwrap() { - Payload::VersionDeltas(_) => { - unreachable!("should get full version") - } - Payload::PinnedVersion(version) => version, - }; + let version = hummock_manager.pin_version(context_id_2).await.unwrap(); assert_eq!( - version.get_id(), + version.id, init_version_id + commit_log_count + register_log_count ); // pinned by context_id_1 diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 6fd7f33bb63a4..13388288f2dcd 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -20,22 +20,23 @@ use itertools::Itertools; use risingwave_common::util::epoch::INVALID_EPOCH; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ build_initial_compaction_group_levels, get_compaction_group_ids, BranchedSstInfo, - HummockVersionExt, }; use risingwave_hummock_sdk::compaction_group::{StateTableId, StaticCompactionGroupId}; use risingwave_hummock_sdk::table_stats::add_prost_table_stats_map; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockSstableObjectId, HummockVersionId, FIRST_VERSION_ID, }; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersion, - HummockVersionCheckpoint, HummockVersionDelta, HummockVersionStats, SstableInfo, TableStats, + CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionStats, + SstableInfo, TableStats, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use crate::hummock::error::Result; +use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; use crate::hummock::manager::worker::{HummockManagerEvent, HummockManagerEventSender}; use crate::hummock::manager::{commit_multi_var, read_lock, write_lock}; use crate::hummock::metrics_utils::{trigger_safepoint_stat, trigger_write_stop_stats}; @@ -399,12 +400,12 @@ mod tests { use std::collections::HashMap; use std::sync::Arc; + use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId}; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - HummockPinnedVersion, HummockVersion, HummockVersionStats, KeyRange, Level, - OverlappingLevel, SstableInfo, + HummockPinnedVersion, HummockVersionStats, KeyRange, Level, OverlappingLevel, SstableInfo, }; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; diff --git a/src/meta/src/hummock/metrics_utils.rs b/src/meta/src/hummock/metrics_utils.rs index cf32bc1e00b61..484dc86a832a9 100644 --- a/src/meta/src/hummock/metrics_utils.rs +++ b/src/meta/src/hummock/metrics_utils.rs @@ -18,22 +18,22 @@ use std::sync::Arc; use std::time::{SystemTime, UNIX_EPOCH}; use itertools::{enumerate, Itertools}; -use prost::Message; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - object_size_map, BranchedSstInfo, HummockVersionExt, + object_size_map, BranchedSstInfo, }; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, }; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ - CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersion, - HummockVersionCheckpoint, HummockVersionStats, LevelType, + CompactionConfig, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionStats, LevelType, }; use super::compaction::get_compression_algorithm; use super::compaction::selector::DynamicLevelSelectorCore; +use crate::hummock::checkpoint::HummockVersionCheckpoint; use crate::hummock::compaction::CompactStatus; use crate::rpc::metrics::MetaMetrics; @@ -47,7 +47,7 @@ pub fn trigger_version_stat( .set(current_version.max_committed_epoch as i64); metrics .version_size - .set(current_version.encoded_len() as i64); + .set(current_version.estimated_encode_len() as i64); metrics.safe_epoch.set(current_version.safe_epoch as i64); metrics.current_version_id.set(current_version.id as i64); metrics.version_stats.reset(); @@ -368,7 +368,7 @@ pub fn trigger_gc_stat( checkpoint: &HummockVersionCheckpoint, min_pinned_version_id: HummockVersionId, ) { - let current_version_object_size_map = object_size_map(checkpoint.version.as_ref().unwrap()); + let current_version_object_size_map = object_size_map(&checkpoint.version); let current_version_object_size = current_version_object_size_map.values().sum::(); let current_version_object_count = current_version_object_size_map.len(); let mut old_version_object_size = 0; diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 9b0b66fc1e03f..282d47d4c16f1 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -22,7 +22,10 @@ use async_trait::async_trait; use fail::fail_point; use futures::stream::BoxStream; use futures::{Stream, StreamExt}; +use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; +use risingwave_hummock_sdk::table_watermark::TableWatermarks; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, SstObjectIdRange, @@ -32,8 +35,8 @@ use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::subscribe_compaction_event_request::{Event, ReportTask}; use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent; use risingwave_pb::hummock::{ - compact_task, CompactTask, HummockSnapshot, HummockVersion, SubscribeCompactionEventRequest, - SubscribeCompactionEventResponse, TableWatermarks, VacuumTask, + compact_task, CompactTask, HummockSnapshot, SubscribeCompactionEventRequest, + SubscribeCompactionEventResponse, VacuumTask, }; use risingwave_rpc_client::error::{Result, RpcError}; use risingwave_rpc_client::{CompactionEventItem, HummockMetaClient}; @@ -94,7 +97,7 @@ impl MockHummockMetaClient { &self, epoch: HummockEpoch, sstables: Vec, - new_table_watermarks: HashMap, + new_table_watermarks: HashMap, ) -> Result<()> { let sst_to_worker = sstables .iter() diff --git a/src/meta/src/hummock/model/ext/hummock.rs b/src/meta/src/hummock/model/ext/hummock.rs index f61c20cf95ad5..b10d0b83da2b6 100644 --- a/src/meta/src/hummock/model/ext/hummock.rs +++ b/src/meta/src/hummock/model/ext/hummock.rs @@ -13,6 +13,7 @@ // limitations under the License. use itertools::Itertools; +use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_meta_model_v2::compaction_config::CompactionConfig; use risingwave_meta_model_v2::compaction_status::LevelHandlers; use risingwave_meta_model_v2::compaction_task::CompactionTask; @@ -22,9 +23,7 @@ use risingwave_meta_model_v2::{ hummock_pinned_version, hummock_version_delta, CompactionGroupId, CompactionTaskId, HummockVersionId, WorkerId, }; -use risingwave_pb::hummock::{ - CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion, HummockVersionDelta, -}; +use risingwave_pb::hummock::{CompactTaskAssignment, HummockPinnedSnapshot, HummockPinnedVersion}; use sea_orm::sea_query::OnConflict; use sea_orm::ActiveValue::Set; use sea_orm::EntityTrait; @@ -192,7 +191,7 @@ impl Transactional for HummockVersionDelta { max_committed_epoch: Set(self.max_committed_epoch as _), safe_epoch: Set(self.safe_epoch as _), trivial_move: Set(self.trivial_move), - full_version_delta: Set(FullVersionDelta(self.clone())), + full_version_delta: Set(FullVersionDelta(self.to_protobuf())), }; hummock_version_delta::Entity::insert(m) .on_conflict( diff --git a/src/meta/src/hummock/model/version_delta.rs b/src/meta/src/hummock/model/version_delta.rs index a7d97790ce51a..3103f428a45f8 100644 --- a/src/meta/src/hummock/model/version_delta.rs +++ b/src/meta/src/hummock/model/version_delta.rs @@ -13,8 +13,9 @@ // limitations under the License. use prost::Message; +use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_hummock_sdk::HummockVersionId; -use risingwave_pb::hummock::HummockVersionDelta; +use risingwave_pb::hummock::PbHummockVersionDelta; use crate::hummock::model::HUMMOCK_VERSION_DELTA_CF_NAME; use crate::model::{MetadataModel, MetadataModelResult}; @@ -22,22 +23,22 @@ use crate::model::{MetadataModel, MetadataModelResult}; /// `HummockVersionDelta` tracks delta of `Sstables` in given version based on previous version. impl MetadataModel for HummockVersionDelta { type KeyType = HummockVersionId; - type PbType = HummockVersionDelta; + type PbType = PbHummockVersionDelta; fn cf_name() -> String { String::from(HUMMOCK_VERSION_DELTA_CF_NAME) } fn to_protobuf(&self) -> Self::PbType { - self.clone() + self.to_protobuf() } fn to_protobuf_encoded_vec(&self) -> Vec { - self.encode_to_vec() + self.to_protobuf().encode_to_vec() } fn from_protobuf(prost: Self::PbType) -> Self { - prost + Self::from_persisted_protobuf(&prost) } fn key(&self) -> MetadataModelResult { diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 74d0698c64bea..679687e87ed96 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -18,15 +18,14 @@ use std::time::Duration; use itertools::Itertools; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::key_with_epoch; +use risingwave_hummock_sdk::version::HummockVersion; 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, HummockVersion, KeyRange, SstableInfo, -}; +use risingwave_pb::hummock::{CompactionConfig, HummockSnapshot, KeyRange, SstableInfo}; use risingwave_pb::meta::add_worker_node_request::Property; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; diff --git a/src/meta/src/model/mod.rs b/src/meta/src/model/mod.rs index 6707108974c6a..c0881813c729d 100644 --- a/src/meta/src/model/mod.rs +++ b/src/meta/src/model/mod.rs @@ -166,7 +166,6 @@ macro_rules! for_all_metadata_models { $macro! { // These items should be included in a meta snapshot. // So be sure to update meta backup/restore when adding new items. - { risingwave_pb::hummock::HummockVersion }, { risingwave_pb::hummock::HummockVersionStats }, { crate::hummock::model::CompactionGroup }, { risingwave_pb::catalog::Database }, @@ -184,7 +183,7 @@ macro_rules! for_all_metadata_models { { crate::model::cluster::Worker }, { risingwave_pb::hummock::CompactTaskAssignment }, { crate::hummock::compaction::CompactStatus }, - { risingwave_pb::hummock::HummockVersionDelta }, + { risingwave_hummock_sdk::version::HummockVersionDelta }, { risingwave_pb::hummock::HummockPinnedSnapshot }, { risingwave_pb::hummock::HummockPinnedVersion }, } diff --git a/src/rpc_client/src/hummock_meta_client.rs b/src/rpc_client/src/hummock_meta_client.rs index b02b432963ada..0244c708ab821 100644 --- a/src/rpc_client/src/hummock_meta_client.rs +++ b/src/rpc_client/src/hummock_meta_client.rs @@ -14,12 +14,12 @@ use async_trait::async_trait; use futures::stream::BoxStream; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, SstObjectIdRange, }; use risingwave_pb::hummock::{ - HummockSnapshot, HummockVersion, SubscribeCompactionEventRequest, - SubscribeCompactionEventResponse, VacuumTask, + HummockSnapshot, SubscribeCompactionEventRequest, SubscribeCompactionEventResponse, VacuumTask, }; use tokio::sync::mpsc::UnboundedSender; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index c2288dea07c0e..fa8ce12c0cf34 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -36,6 +36,7 @@ use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::compaction_group::StateTableId; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, SstObjectIdRange, @@ -946,10 +947,13 @@ impl MetaClient { version_delta: HummockVersionDelta, ) -> Result<(HummockVersion, Vec)> { let req = ReplayVersionDeltaRequest { - version_delta: Some(version_delta), + version_delta: Some(version_delta.to_protobuf()), }; let resp = self.inner.replay_version_delta(req).await?; - Ok((resp.version.unwrap(), resp.modified_compaction_groups)) + Ok(( + HummockVersion::from_rpc_protobuf(&resp.version.unwrap()), + resp.modified_compaction_groups, + )) } pub async fn list_version_deltas( @@ -957,7 +961,7 @@ impl MetaClient { start_id: u64, num_limit: u32, committed_epoch_limit: HummockEpoch, - ) -> Result { + ) -> Result> { let req = ListVersionDeltasRequest { start_id, num_limit, @@ -968,7 +972,11 @@ impl MetaClient { .list_version_deltas(req) .await? .version_deltas - .unwrap()) + .unwrap() + .version_deltas + .iter() + .map(HummockVersionDelta::from_rpc_protobuf) + .collect()) } pub async fn trigger_compaction_deterministic( @@ -986,12 +994,14 @@ impl MetaClient { pub async fn disable_commit_epoch(&self) -> Result { let req = DisableCommitEpochRequest {}; - Ok(self - .inner - .disable_commit_epoch(req) - .await? - .current_version - .unwrap()) + Ok(HummockVersion::from_rpc_protobuf( + &self + .inner + .disable_commit_epoch(req) + .await? + .current_version + .unwrap(), + )) } pub async fn pin_specific_snapshot(&self, epoch: HummockEpoch) -> Result { @@ -1284,12 +1294,14 @@ impl HummockMetaClient for MetaClient { async fn get_current_version(&self) -> Result { let req = GetCurrentVersionRequest::default(); - Ok(self - .inner - .get_current_version(req) - .await? - .current_version - .unwrap()) + Ok(HummockVersion::from_rpc_protobuf( + &self + .inner + .get_current_version(req) + .await? + .current_version + .unwrap(), + )) } async fn pin_snapshot(&self) -> Result { diff --git a/src/storage/backup/src/lib.rs b/src/storage/backup/src/lib.rs index f4b4b7e84b285..f094d16010c23 100644 --- a/src/storage/backup/src/lib.rs +++ b/src/storage/backup/src/lib.rs @@ -37,10 +37,9 @@ use std::collections::HashSet; use std::hash::Hasher; use itertools::Itertools; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockSstableObjectId, HummockVersionId}; use risingwave_pb::backup_service::{PbMetaSnapshotManifest, PbMetaSnapshotMetadata}; -use risingwave_pb::hummock::HummockVersion; use serde::{Deserialize, Serialize}; use crate::error::{BackupError, BackupResult}; diff --git a/src/storage/backup/src/meta_snapshot.rs b/src/storage/backup/src/meta_snapshot.rs index c42fcc5d5851f..e7049fbe0ae41 100644 --- a/src/storage/backup/src/meta_snapshot.rs +++ b/src/storage/backup/src/meta_snapshot.rs @@ -15,7 +15,7 @@ use std::fmt::{Display, Formatter}; use bytes::{Buf, BufMut}; -use risingwave_pb::hummock::HummockVersion; +use risingwave_hummock_sdk::version::HummockVersion; use crate::error::BackupResult; use crate::{xxhash64_checksum, xxhash64_verify, MetaSnapshotId}; diff --git a/src/storage/backup/src/meta_snapshot_v1.rs b/src/storage/backup/src/meta_snapshot_v1.rs index 731107723c2f4..76a8a548eacd7 100644 --- a/src/storage/backup/src/meta_snapshot_v1.rs +++ b/src/storage/backup/src/meta_snapshot_v1.rs @@ -18,10 +18,11 @@ use std::fmt::{Display, Formatter}; use bytes::{Buf, BufMut}; use itertools::Itertools; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_pb::catalog::{ Connection, Database, Function, Index, Schema, Sink, Source, Table, View, }; -use risingwave_pb::hummock::{CompactionGroup, HummockVersion, HummockVersionStats}; +use risingwave_pb::hummock::{CompactionGroup, HummockVersionStats}; use risingwave_pb::meta::{SystemParams, TableFragments}; use risingwave_pb::user::UserInfo; @@ -127,7 +128,7 @@ impl ClusterMetadata { let default_cf_values = self.default_cf.values().collect_vec(); Self::encode_prost_message_list(&default_cf_keys, buf); Self::encode_prost_message_list(&default_cf_values, buf); - Self::encode_prost_message(&self.hummock_version, buf); + Self::encode_prost_message(&self.hummock_version.to_protobuf(), buf); Self::encode_prost_message(&self.version_stats, buf); Self::encode_prost_message_list(&self.compaction_groups.iter().collect_vec(), buf); Self::encode_prost_message_list(&self.table_fragments.iter().collect_vec(), buf); @@ -153,7 +154,8 @@ impl ClusterMetadata { .into_iter() .zip_eq_fast(default_cf_values.into_iter()) .collect(); - let hummock_version = Self::decode_prost_message(&mut buf)?; + let hummock_version = + HummockVersion::from_persisted_protobuf(&Self::decode_prost_message(&mut buf)?); let version_stats = Self::decode_prost_message(&mut buf)?; let compaction_groups: Vec = Self::decode_prost_message_list(&mut buf)?; let table_fragments: Vec = Self::decode_prost_message_list(&mut buf)?; diff --git a/src/storage/backup/src/meta_snapshot_v2.rs b/src/storage/backup/src/meta_snapshot_v2.rs index 092cfd083783c..57bb43e24295a 100644 --- a/src/storage/backup/src/meta_snapshot_v2.rs +++ b/src/storage/backup/src/meta_snapshot_v2.rs @@ -15,8 +15,8 @@ use std::fmt::{Display, Formatter}; use bytes::{Buf, BufMut}; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_meta_model_v2 as model_v2; -use risingwave_pb::hummock::HummockVersion; use serde::{Deserialize, Serialize}; use crate::meta_snapshot::{MetaSnapshot, Metadata}; @@ -51,7 +51,7 @@ impl Display for MetadataV2 { impl Metadata for MetadataV2 { fn encode_to(&self, buf: &mut Vec) -> BackupResult<()> { put_with_len_prefix(buf, &self.cluster_id)?; - put_with_len_prefix(buf, &self.hummock_version)?; + put_with_len_prefix(buf, &self.hummock_version.to_protobuf())?; put_with_len_prefix(buf, &self.version_stats)?; put_with_len_prefix(buf, &self.compaction_configs)?; // TODO: other metadata @@ -63,13 +63,13 @@ impl Metadata for MetadataV2 { Self: Sized, { let cluster_id = get_with_len_prefix(&mut buf)?; - let hummock_version = get_with_len_prefix(&mut buf)?; + let pb_hummock_version = get_with_len_prefix(&mut buf)?; let version_stats = get_with_len_prefix(&mut buf)?; let compaction_configs = get_with_len_prefix(&mut buf)?; // TODO: other metadata Ok(Self { cluster_id, - hummock_version, + hummock_version: HummockVersion::from_persisted_protobuf(&pb_hummock_version), version_stats, compaction_configs, }) diff --git a/src/storage/hummock_sdk/Cargo.toml b/src/storage/hummock_sdk/Cargo.toml index 10752894f4cc7..e3327bdf993cc 100644 --- a/src/storage/hummock_sdk/Cargo.toml +++ b/src/storage/hummock_sdk/Cargo.toml @@ -19,6 +19,7 @@ easy-ext = "1" hex = "0.4" itertools = "0.12" parse-display = "0.8" +prost = { workspace = true } risingwave_common = { workspace = true } risingwave_pb = { workspace = true } tracing = "0.1" 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 807748cc77d6a..033f8ab54e471 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 @@ -21,10 +21,11 @@ use risingwave_common::catalog::TableId; use risingwave_pb::hummock::group_delta::DeltaType; use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; +use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; use risingwave_pb::hummock::{ CompactionConfig, CompatibilityVersion, GroupConstruct, GroupDestroy, GroupMetaChange, - GroupTableChange, HummockVersion, HummockVersionDelta, Level, LevelType, OverlappingLevel, - PbLevelType, PbTableWatermarks, SstableInfo, + GroupTableChange, Level, LevelType, OverlappingLevel, PbLevelType, PbTableWatermarks, + SstableInfo, }; use tracing::warn; @@ -32,7 +33,8 @@ use super::StateTableId; use crate::compaction_group::StaticCompactionGroupId; use crate::key_range::KeyRangeCommon; use crate::prost_key_range::KeyRangeExt; -use crate::table_watermark::{PbTableWatermarksExt, TableWatermarks, TableWatermarksIndex}; +use crate::table_watermark::{TableWatermarks, TableWatermarksIndex, VnodeWatermark}; +use crate::version::{HummockVersion, HummockVersionDelta}; use crate::{can_concat, CompactionGroupId, HummockSstableId, HummockSstableObjectId}; pub struct GroupDeltasSummary { @@ -121,7 +123,6 @@ pub struct SstDeltaInfo { pub type BranchedSstInfo = HashMap; -#[easy_ext::ext(HummockVersionExt)] impl HummockVersion { pub fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Levels { self.levels @@ -195,9 +196,8 @@ impl HummockVersion { .iter() .map(|(table_id, table_watermarks)| { ( - TableId::from(*table_id), - TableWatermarks::from_protobuf(table_watermarks) - .build_index(self.max_committed_epoch), + *table_id, + table_watermarks.build_index(self.max_committed_epoch), ) }) .collect() @@ -208,20 +208,27 @@ impl HummockVersion { existing_table_ids: &[u32], ) -> BTreeMap { fn extract_single_table_watermark( - table_watermarks: &PbTableWatermarks, + table_watermarks: &TableWatermarks, safe_epoch: u64, ) -> Option { - if let Some(first_epoch_watermark) = table_watermarks.epoch_watermarks.first() { + if let Some((first_epoch, first_epoch_watermark)) = table_watermarks.watermarks.first() + { assert!( - first_epoch_watermark.epoch >= safe_epoch, + *first_epoch >= safe_epoch, "smallest epoch {} in table watermark should be at least safe epoch {}", - first_epoch_watermark.epoch, + first_epoch, safe_epoch ); - if first_epoch_watermark.epoch == safe_epoch { + if *first_epoch == safe_epoch { Some(PbTableWatermarks { - epoch_watermarks: vec![first_epoch_watermark.clone()], - is_ascending: table_watermarks.is_ascending, + epoch_watermarks: vec![PbEpochNewWatermarks { + watermarks: first_epoch_watermark + .iter() + .map(VnodeWatermark::to_protobuf) + .collect(), + epoch: *first_epoch, + }], + is_ascending: table_watermarks.direction.is_ascending(), }) } else { None @@ -233,12 +240,12 @@ impl HummockVersion { self.table_watermarks .iter() .filter_map(|(table_id, table_watermarks)| { - let u32_table_id = *table_id as _; + let u32_table_id = table_id.table_id(); if !existing_table_ids.contains(&u32_table_id) { None } else { extract_single_table_watermark(table_watermarks, self.safe_epoch) - .map(|table_watermarks| (*table_id, table_watermarks)) + .map(|table_watermarks| (table_id.table_id, table_watermarks)) } }) .collect() @@ -256,7 +263,6 @@ pub type SstSplitInfo = ( HummockSstableId, ); -#[easy_ext::ext(HummockVersionUpdateExt)] impl HummockVersion { pub fn count_new_ssts_in_group_split( &self, @@ -1249,13 +1255,12 @@ mod tests { use risingwave_pb::hummock::hummock_version::Levels; use risingwave_pb::hummock::hummock_version_delta::GroupDeltas; use risingwave_pb::hummock::{ - CompactionConfig, GroupConstruct, GroupDelta, GroupDestroy, HummockVersion, - HummockVersionDelta, IntraLevelDelta, Level, LevelType, OverlappingLevel, SstableInfo, + CompactionConfig, GroupConstruct, GroupDelta, GroupDestroy, IntraLevelDelta, Level, + LevelType, OverlappingLevel, SstableInfo, }; - use crate::compaction_group::hummock_version_ext::{ - build_initial_compaction_group_levels, HummockVersionExt, HummockVersionUpdateExt, - }; + use crate::compaction_group::hummock_version_ext::build_initial_compaction_group_levels; + use crate::version::{HummockVersion, HummockVersionDelta}; #[test] fn test_get_sst_object_ids() { diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index be041c1588465..8ac1515de5220 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -44,6 +44,7 @@ pub mod key_range; pub mod prost_key_range; pub mod table_stats; pub mod table_watermark; +pub mod version; pub use compact::*; diff --git a/src/storage/hummock_sdk/src/table_stats.rs b/src/storage/hummock_sdk/src/table_stats.rs index 02da4999bebc3..40d799ce7256c 100644 --- a/src/storage/hummock_sdk/src/table_stats.rs +++ b/src/storage/hummock_sdk/src/table_stats.rs @@ -15,9 +15,9 @@ use std::borrow::Borrow; use std::collections::{HashMap, HashSet}; -use risingwave_pb::hummock::{HummockVersion, PbTableStats}; +use risingwave_pb::hummock::PbTableStats; -use crate::compaction_group::hummock_version_ext::HummockVersionExt; +use crate::version::HummockVersion; pub type TableStatsMap = HashMap; diff --git a/src/storage/hummock_sdk/src/table_watermark.rs b/src/storage/hummock_sdk/src/table_watermark.rs index 2f76cff6cc5e4..50ecaec0ad69e 100644 --- a/src/storage/hummock_sdk/src/table_watermark.rs +++ b/src/storage/hummock_sdk/src/table_watermark.rs @@ -15,11 +15,14 @@ use std::cmp::Ordering; use std::collections::hash_map::Entry; use std::collections::{btree_map, BTreeMap, HashMap, HashSet}; +use std::mem::size_of; use std::ops::Bound::{Excluded, Included, Unbounded}; use std::sync::Arc; use bytes::Bytes; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; +use risingwave_common::catalog::TableId; +use risingwave_common::estimate_size::EstimateSize; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; use risingwave_pb::hummock::{PbTableWatermarks, PbVnodeWatermark}; @@ -288,9 +291,16 @@ impl WatermarkDirection { WatermarkDirection::Descending => key > watermark, } } + + pub fn is_ascending(&self) -> bool { + match self { + WatermarkDirection::Ascending => true, + WatermarkDirection::Descending => false, + } + } } -#[derive(Clone, Debug, PartialEq)] +#[derive(Clone, Debug, PartialEq, EstimateSize)] pub struct VnodeWatermark { vnode_bitmap: Arc, watermark: Bytes, @@ -323,11 +333,11 @@ impl VnodeWatermark { } } -#[derive(Clone, Debug)] +#[derive(Clone, Debug, PartialEq)] pub struct TableWatermarks { // later epoch at the back - watermarks: Vec<(HummockEpoch, Vec)>, - direction: WatermarkDirection, + pub(crate) watermarks: Vec<(HummockEpoch, Vec)>, + pub(crate) direction: WatermarkDirection, } impl TableWatermarks { @@ -359,6 +369,21 @@ impl TableWatermarks { } } + pub fn estimated_encode_len(&self) -> usize { + self.watermarks.len() * size_of::() + + self + .watermarks + .iter() + .map(|(_, watermarks)| { + watermarks + .iter() + .map(|watermark| watermark.estimated_size()) + .sum::() + }) + .sum::() + + size_of::() // for direction + } + pub fn add_new_epoch_watermarks( &mut self, epoch: HummockEpoch, @@ -411,106 +436,104 @@ impl TableWatermarks { } pub fn merge_multiple_new_table_watermarks( - table_watermarks_list: impl IntoIterator>, -) -> HashMap { - let mut ret: HashMap)> = HashMap::new(); + table_watermarks_list: impl IntoIterator>, +) -> HashMap { + let mut ret: HashMap>)> = + HashMap::new(); for table_watermarks in table_watermarks_list { for (table_id, new_table_watermarks) in table_watermarks { let epoch_watermarks = match ret.entry(table_id) { Entry::Occupied(entry) => { - let (is_ascending, epoch_watermarks) = entry.into_mut(); - assert_eq!(new_table_watermarks.is_ascending, *is_ascending); + let (direction, epoch_watermarks) = entry.into_mut(); + assert_eq!(&new_table_watermarks.direction, direction); epoch_watermarks } Entry::Vacant(entry) => { let (_, epoch_watermarks) = - entry.insert((new_table_watermarks.is_ascending, BTreeMap::new())); + entry.insert((new_table_watermarks.direction, BTreeMap::new())); epoch_watermarks } }; - for new_epoch_watermarks in new_table_watermarks.epoch_watermarks { + for (new_epoch, new_epoch_watermarks) in new_table_watermarks.watermarks { epoch_watermarks - .entry(new_epoch_watermarks.epoch) - .or_insert_with(|| PbEpochNewWatermarks { - watermarks: vec![], - epoch: new_epoch_watermarks.epoch, - }) - .watermarks - .extend(new_epoch_watermarks.watermarks); + .entry(new_epoch) + .or_insert_with(Vec::new) + .extend(new_epoch_watermarks); } } } ret.into_iter() - .map(|(table_id, (is_ascending, epoch_watermarks))| { + .map(|(table_id, (direction, epoch_watermarks))| { ( table_id, - PbTableWatermarks { - is_ascending, + TableWatermarks { + direction, // ordered from earlier epoch to later epoch - epoch_watermarks: epoch_watermarks.into_values().collect(), + watermarks: epoch_watermarks.into_iter().collect(), }, ) }) .collect() } -#[easy_ext::ext(PbTableWatermarksExt)] -impl PbTableWatermarks { - pub fn apply_new_table_watermarks(&mut self, newly_added_watermarks: &PbTableWatermarks) { - assert_eq!(self.is_ascending, newly_added_watermarks.is_ascending); - assert!(self.epoch_watermarks.iter().map(|w| w.epoch).is_sorted()); +impl TableWatermarks { + pub fn apply_new_table_watermarks(&mut self, newly_added_watermarks: &TableWatermarks) { + assert_eq!(self.direction, newly_added_watermarks.direction); + assert!(self.watermarks.iter().map(|(epoch, _)| epoch).is_sorted()); assert!(newly_added_watermarks - .epoch_watermarks + .watermarks .iter() - .map(|w| w.epoch) + .map(|(epoch, _)| epoch) .is_sorted()); // ensure that the newly added watermarks have a later epoch than the previous latest epoch. - if let Some(prev_last_epoch_watermarks) = self.epoch_watermarks.last() && let Some(new_first_epoch_watermarks) = newly_added_watermarks.epoch_watermarks.first() { - assert!(prev_last_epoch_watermarks.epoch < new_first_epoch_watermarks.epoch); + if let Some((prev_last_epoch, _)) = self.watermarks.last() + && let Some((new_first_epoch, _)) = newly_added_watermarks.watermarks.first() { + assert!(prev_last_epoch < new_first_epoch); } - self.epoch_watermarks - .extend(newly_added_watermarks.epoch_watermarks.clone()); + self.watermarks.extend( + newly_added_watermarks + .watermarks + .iter() + .map(|(epoch, new_watermarks)| (*epoch, new_watermarks.clone())), + ); } pub fn clear_stale_epoch_watermark(&mut self, safe_epoch: u64) { - match self.epoch_watermarks.first() { + match self.watermarks.first() { None => { // return on empty watermark return; } - Some(earliest_epoch_watermark) => { - if earliest_epoch_watermark.epoch >= safe_epoch { + Some((earliest_epoch, _)) => { + if *earliest_epoch >= safe_epoch { // No stale epoch watermark needs to be cleared. return; } } } debug!("clear stale table watermark below epoch {}", safe_epoch); - let mut result_epoch_watermark = Vec::with_capacity(self.epoch_watermarks.len()); + let mut result_epoch_watermark = Vec::with_capacity(self.watermarks.len()); let mut unset_vnode: HashSet = (0..VirtualNode::COUNT) .map(VirtualNode::from_index) .collect(); - while let Some(epoch_watermark) = self.epoch_watermarks.last() { - if epoch_watermark.epoch >= safe_epoch { - let epoch_watermark = self.epoch_watermarks.pop().expect("have check Some"); - for watermark in &epoch_watermark.watermarks { - for vnode in - Bitmap::from(watermark.vnode_bitmap.as_ref().expect("should not be None")) - .iter_vnodes() - { + while let Some((epoch, _)) = self.watermarks.last() { + if *epoch >= safe_epoch { + let (epoch, watermarks) = self.watermarks.pop().expect("have check Some"); + for watermark in &watermarks { + for vnode in watermark.vnode_bitmap.iter_vnodes() { unset_vnode.remove(&vnode); } } - result_epoch_watermark.push(epoch_watermark); + result_epoch_watermark.push((epoch, watermarks)); } else { break; } } - while !unset_vnode.is_empty() && let Some(epoch_watermark) = self.epoch_watermarks.pop() { + while !unset_vnode.is_empty() && let Some((_, watermarks)) = self.watermarks.pop() { let mut new_vnode_watermarks = Vec::new(); - for vnode_watermark in &epoch_watermark.watermarks { + for vnode_watermark in watermarks { let mut set_vnode = Vec::new(); - for vnode in Bitmap::from(vnode_watermark.vnode_bitmap.as_ref().expect("should not be None")).iter_vnodes() { + for vnode in vnode_watermark.vnode_bitmap.iter_vnodes() { if unset_vnode.remove(&vnode) { set_vnode.push(vnode); } @@ -520,36 +543,34 @@ impl PbTableWatermarks { for vnode in set_vnode { builder.set(vnode.to_index(), true); } - let bitmap = builder.finish(); - new_vnode_watermarks.push(PbVnodeWatermark { - vnode_bitmap: Some(bitmap.to_protobuf()), - watermark: vnode_watermark.watermark.clone(), + let bitmap = Arc::new(builder.finish()); + new_vnode_watermarks.push(VnodeWatermark { + vnode_bitmap: bitmap, + watermark: vnode_watermark.watermark, }) } } if !new_vnode_watermarks.is_empty() { - if let Some(last_epoch_watermark) = result_epoch_watermark.last_mut() && last_epoch_watermark.epoch == safe_epoch { - last_epoch_watermark.watermarks.extend(new_vnode_watermarks); + if let Some((last_epoch, last_watermarks)) = result_epoch_watermark.last_mut() && *last_epoch == safe_epoch { + last_watermarks.extend(new_vnode_watermarks); } else { - result_epoch_watermark.push(PbEpochNewWatermarks { - watermarks: new_vnode_watermarks, - // set epoch as safe epoch - epoch: safe_epoch, - }) + result_epoch_watermark.push((safe_epoch, new_vnode_watermarks)); } } } // epoch watermark are added from later epoch to earlier epoch. // reverse to ensure that earlier epochs are at the front result_epoch_watermark.reverse(); - assert!(result_epoch_watermark.is_sorted_by(|first, second| { - let ret = first.epoch.cmp(&second.epoch); - assert_ne!(ret, Ordering::Equal); - Some(ret) - })); - *self = PbTableWatermarks { - epoch_watermarks: result_epoch_watermark, - is_ascending: self.is_ascending, + assert!( + result_epoch_watermark.is_sorted_by(|(first_epoch, _), (second_epoch, _)| { + let ret = first_epoch.cmp(second_epoch); + assert_ne!(ret, Ordering::Equal); + Some(ret) + }) + ); + *self = TableWatermarks { + watermarks: result_epoch_watermark, + direction: self.direction, } } } @@ -566,18 +587,16 @@ mod tests { use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; - use risingwave_pb::hummock::table_watermarks::PbEpochNewWatermarks; - use risingwave_pb::hummock::{HummockVersion, PbTableWatermarks, PbVnodeWatermark}; - use crate::compaction_group::hummock_version_ext::HummockVersionExt; use crate::key::{ is_empty_key_range, map_table_key_range, prefix_slice_with_vnode, prefixed_range_with_vnode, TableKeyRange, }; use crate::table_watermark::{ - merge_multiple_new_table_watermarks, PbTableWatermarksExt, TableWatermarks, - TableWatermarksIndex, VnodeWatermark, WatermarkDirection, + merge_multiple_new_table_watermarks, TableWatermarks, TableWatermarksIndex, VnodeWatermark, + WatermarkDirection, }; + use crate::version::HummockVersion; fn build_bitmap(vnodes: impl IntoIterator) -> Arc { let mut builder = BitmapBuilder::zeroed(VirtualNode::COUNT); @@ -613,7 +632,7 @@ mod tests { direction, ); - let mut pb_table_watermark = table_watermarks.to_protobuf(); + let mut table_watermark_checkpoint = table_watermarks.clone(); let epoch3 = epoch2 + 1; let mut second_table_watermark = TableWatermarks::single_epoch( @@ -651,8 +670,8 @@ mod tests { direction, ); - pb_table_watermark.apply_new_table_watermarks(&second_table_watermark.to_protobuf()); - assert_eq!(table_watermarks.to_protobuf(), pb_table_watermark); + table_watermark_checkpoint.apply_new_table_watermarks(&second_table_watermark); + assert_eq!(table_watermarks, table_watermark_checkpoint); } #[test] @@ -700,13 +719,13 @@ mod tests { direction, ); - let mut pb_table_watermarks = table_watermarks.to_protobuf(); - pb_table_watermarks.clear_stale_epoch_watermark(epoch1); - assert_eq!(pb_table_watermarks, table_watermarks.to_protobuf()); + let mut table_watermarks_checkpoint = table_watermarks.clone(); + table_watermarks_checkpoint.clear_stale_epoch_watermark(epoch1); + assert_eq!(table_watermarks_checkpoint, table_watermarks); - pb_table_watermarks.clear_stale_epoch_watermark(epoch2); + table_watermarks_checkpoint.clear_stale_epoch_watermark(epoch2); assert_eq!( - pb_table_watermarks, + table_watermarks_checkpoint, TableWatermarks { watermarks: vec![ ( @@ -733,12 +752,11 @@ mod tests { ], direction, } - .to_protobuf() ); - pb_table_watermarks.clear_stale_epoch_watermark(epoch3); + table_watermarks_checkpoint.clear_stale_epoch_watermark(epoch3); assert_eq!( - pb_table_watermarks, + table_watermarks_checkpoint, TableWatermarks { watermarks: vec![ ( @@ -758,12 +776,11 @@ mod tests { ], direction, } - .to_protobuf() ); - pb_table_watermarks.clear_stale_epoch_watermark(epoch4); + table_watermarks_checkpoint.clear_stale_epoch_watermark(epoch4); assert_eq!( - pb_table_watermarks, + table_watermarks_checkpoint, TableWatermarks { watermarks: vec![ ( @@ -783,12 +800,11 @@ mod tests { ], direction, } - .to_protobuf() ); - pb_table_watermarks.clear_stale_epoch_watermark(epoch5); + table_watermarks_checkpoint.clear_stale_epoch_watermark(epoch5); assert_eq!( - pb_table_watermarks, + table_watermarks_checkpoint, TableWatermarks { watermarks: vec![( epoch5, @@ -802,35 +818,34 @@ mod tests { )], direction, } - .to_protobuf() ); } #[test] fn test_merge_multiple_new_table_watermarks() { - fn epoch_new_watermark(epoch: u64, bitmaps: Vec<&Bitmap>) -> PbEpochNewWatermarks { - PbEpochNewWatermarks { - watermarks: bitmaps + fn epoch_new_watermark(epoch: u64, bitmaps: Vec<&Bitmap>) -> (u64, Vec) { + ( + epoch, + bitmaps .into_iter() - .map(|bitmap| PbVnodeWatermark { - watermark: vec![1, 2, epoch as _], - vnode_bitmap: Some(bitmap.to_protobuf()), + .map(|bitmap| VnodeWatermark { + watermark: Bytes::from(vec![1, 2, epoch as _]), + vnode_bitmap: Arc::new(bitmap.clone()), }) .collect(), - epoch: epoch as _, - } + ) } fn build_table_watermark( vnodes: impl IntoIterator, epochs: impl IntoIterator, - ) -> PbTableWatermarks { + ) -> TableWatermarks { let bitmap = build_bitmap(vnodes); - PbTableWatermarks { - epoch_watermarks: epochs + TableWatermarks { + watermarks: epochs .into_iter() .map(|epoch: u64| epoch_new_watermark(epoch, vec![&bitmap])) .collect(), - is_ascending: true, + direction: WatermarkDirection::Ascending, } } let table1_watermark1 = build_table_watermark(0..3, vec![1, 2, 4]); @@ -838,27 +853,27 @@ mod tests { let table2_watermark = build_table_watermark(0..4, 1..3); let table3_watermark = build_table_watermark(0..4, 3..5); let mut first = HashMap::new(); - first.insert(1, table1_watermark1); - first.insert(2, table2_watermark.clone()); + first.insert(TableId::new(1), table1_watermark1); + first.insert(TableId::new(2), table2_watermark.clone()); let mut second = HashMap::new(); - second.insert(1, table1_watermark2); - second.insert(3, table3_watermark.clone()); + second.insert(TableId::new(1), table1_watermark2); + second.insert(TableId::new(3), table3_watermark.clone()); let result = merge_multiple_new_table_watermarks(vec![first, second]); let mut expected = HashMap::new(); expected.insert( - 1, - PbTableWatermarks { - epoch_watermarks: vec![ + TableId::new(1), + TableWatermarks { + watermarks: vec![ epoch_new_watermark(1, vec![&build_bitmap(0..3), &build_bitmap(4..6)]), epoch_new_watermark(2, vec![&build_bitmap(0..3), &build_bitmap(4..6)]), epoch_new_watermark(4, vec![&build_bitmap(0..3)]), epoch_new_watermark(5, vec![&build_bitmap(4..6)]), ], - is_ascending: true, + direction: WatermarkDirection::Ascending, }, ); - expected.insert(2, table2_watermark); - expected.insert(3, table3_watermark); + expected.insert(TableId::new(2), table2_watermark); + expected.insert(TableId::new(3), table3_watermark); assert_eq!(result, expected); } @@ -1053,16 +1068,16 @@ mod tests { }; let test_table_id = TableId::from(233); version.table_watermarks.insert( - test_table_id.table_id, - PbTableWatermarks { - epoch_watermarks: vec![PbEpochNewWatermarks { - watermarks: vec![PbVnodeWatermark { - watermark: watermark1.to_vec(), - vnode_bitmap: Some(build_bitmap(0..VirtualNode::COUNT).to_protobuf()), + test_table_id, + TableWatermarks { + watermarks: vec![( + EPOCH1, + vec![VnodeWatermark { + watermark: watermark1.clone(), + vnode_bitmap: build_bitmap(0..VirtualNode::COUNT), }], - epoch: EPOCH1, - }], - is_ascending: true, + )], + direction: WatermarkDirection::Ascending, }, ); let committed_index = version diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs new file mode 100644 index 0000000000000..758d374e21321 --- /dev/null +++ b/src/storage/hummock_sdk/src/version.rs @@ -0,0 +1,192 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::mem::size_of; + +use prost::Message; +use risingwave_common::catalog::TableId; +use risingwave_pb::hummock::hummock_version::PbLevels; +use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; +use risingwave_pb::hummock::{PbHummockVersion, PbHummockVersionDelta}; + +use crate::table_watermark::TableWatermarks; +use crate::{CompactionGroupId, HummockSstableObjectId}; + +#[derive(Debug, Clone, PartialEq)] +pub struct HummockVersion { + pub id: u64, + pub levels: HashMap, + pub max_committed_epoch: u64, + pub safe_epoch: u64, + pub table_watermarks: HashMap, +} + +impl Default for HummockVersion { + fn default() -> Self { + HummockVersion::from_protobuf_inner(&PbHummockVersion::default()) + } +} + +impl HummockVersion { + /// Convert the `PbHummockVersion` received from rpc to `HummockVersion`. No need to + /// maintain backward compatibility. + pub fn from_rpc_protobuf(pb_version: &PbHummockVersion) -> Self { + Self::from_protobuf_inner(pb_version) + } + + /// Convert the `PbHummockVersion` deserialized from persisted state to `HummockVersion`. + /// We should maintain backward compatibility. + pub fn from_persisted_protobuf(pb_version: &PbHummockVersion) -> Self { + Self::from_protobuf_inner(pb_version) + } + + fn from_protobuf_inner(pb_version: &PbHummockVersion) -> Self { + Self { + id: pb_version.id, + levels: pb_version + .levels + .iter() + .map(|(group_id, levels)| (*group_id as CompactionGroupId, levels.clone())) + .collect(), + max_committed_epoch: pb_version.max_committed_epoch, + safe_epoch: pb_version.safe_epoch, + table_watermarks: pb_version + .table_watermarks + .iter() + .map(|(table_id, table_watermark)| { + ( + TableId::new(*table_id), + TableWatermarks::from_protobuf(table_watermark), + ) + }) + .collect(), + } + } + + pub fn to_protobuf(&self) -> PbHummockVersion { + PbHummockVersion { + id: self.id, + levels: self + .levels + .iter() + .map(|(group_id, levels)| (*group_id as _, levels.clone())) + .collect(), + max_committed_epoch: self.max_committed_epoch, + safe_epoch: self.safe_epoch, + table_watermarks: self + .table_watermarks + .iter() + .map(|(table_id, watermark)| (table_id.table_id, watermark.to_protobuf())) + .collect(), + } + } + + pub fn estimated_encode_len(&self) -> usize { + self.levels.len() * size_of::() + + self + .levels + .values() + .map(|level| level.encoded_len()) + .sum::() + + self.table_watermarks.len() * size_of::() + + self + .table_watermarks + .values() + .map(|table_watermark| table_watermark.estimated_encode_len()) + .sum::() + } +} + +#[derive(Debug, PartialEq, Clone)] +pub struct HummockVersionDelta { + pub id: u64, + pub prev_id: u64, + pub group_deltas: HashMap, + pub max_committed_epoch: u64, + pub safe_epoch: u64, + pub trivial_move: bool, + pub gc_object_ids: Vec, + pub new_table_watermarks: HashMap, + pub removed_table_ids: Vec, +} + +impl Default for HummockVersionDelta { + fn default() -> Self { + HummockVersionDelta::from_protobuf_inner(&PbHummockVersionDelta::default()) + } +} + +impl HummockVersionDelta { + /// Convert the `PbHummockVersionDelta` deserialized from persisted state to `HummockVersionDelta`. + /// We should maintain backward compatibility. + pub fn from_persisted_protobuf(delta: &PbHummockVersionDelta) -> Self { + Self::from_protobuf_inner(delta) + } + + /// Convert the `PbHummockVersionDelta` received from rpc to `HummockVersionDelta`. No need to + /// maintain backward compatibility. + pub fn from_rpc_protobuf(delta: &PbHummockVersionDelta) -> Self { + Self::from_protobuf_inner(delta) + } + + fn from_protobuf_inner(delta: &PbHummockVersionDelta) -> Self { + Self { + id: delta.id, + prev_id: delta.prev_id, + group_deltas: delta.group_deltas.clone(), + max_committed_epoch: delta.max_committed_epoch, + safe_epoch: delta.safe_epoch, + trivial_move: delta.trivial_move, + gc_object_ids: delta.gc_object_ids.clone(), + new_table_watermarks: delta + .new_table_watermarks + .iter() + .map(|(table_id, watermarks)| { + ( + TableId::new(*table_id), + TableWatermarks::from_protobuf(watermarks), + ) + }) + .collect(), + removed_table_ids: delta + .removed_table_ids + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + } + } + + pub fn to_protobuf(&self) -> PbHummockVersionDelta { + PbHummockVersionDelta { + id: self.id, + prev_id: self.prev_id, + group_deltas: self.group_deltas.clone(), + max_committed_epoch: self.max_committed_epoch, + safe_epoch: self.safe_epoch, + trivial_move: self.trivial_move, + gc_object_ids: self.gc_object_ids.clone(), + new_table_watermarks: self + .new_table_watermarks + .iter() + .map(|(table_id, watermarks)| (table_id.table_id, watermarks.to_protobuf())) + .collect(), + removed_table_ids: self + .removed_table_ids + .iter() + .map(|table_id| table_id.table_id) + .collect(), + } + } +} diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index 4edd044d5838c..be0ebe204d745 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -29,11 +29,11 @@ pub(crate) mod tests { use risingwave_common::util::epoch::Epoch; use risingwave_common_service::observer_manager::NotificationClient; use risingwave_hummock_sdk::can_concat; - use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{next_key, FullKey, TableKey, TABLE_PREFIX_LEN}; use risingwave_hummock_sdk::prost_key_range::KeyRangeExt; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; + use risingwave_hummock_sdk::version::HummockVersion; use risingwave_meta::hummock::compaction::compaction_config::CompactionConfigBuilder; use risingwave_meta::hummock::compaction::selector::{ default_compaction_selector, ManualCompactionOption, @@ -44,7 +44,7 @@ pub(crate) mod tests { }; use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_pb::common::{HostAddress, WorkerType}; - use risingwave_pb::hummock::{CompactTask, HummockVersion, InputLevel, KeyRange, TableOption}; + use risingwave_pb::hummock::{CompactTask, InputLevel, KeyRange, TableOption}; use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::filter_key_extractor::{ diff --git a/src/storage/hummock_test/src/mock_notification_client.rs b/src/storage/hummock_test/src/mock_notification_client.rs index 991a5a9d5bf84..998fcf39b010c 100644 --- a/src/storage/hummock_test/src/mock_notification_client.rs +++ b/src/storage/hummock_test/src/mock_notification_client.rs @@ -62,7 +62,7 @@ impl NotificationClient for MockNotificationClient { let hummock_version = self.hummock_manager.get_current_version().await; let meta_snapshot = MetaSnapshot { - hummock_version: Some(hummock_version), + hummock_version: Some(hummock_version.to_protobuf()), version: Some(Default::default()), meta_backup_manifest_id: Some(MetaBackupManifestId { id: 0 }), hummock_write_limits: Some(WriteLimits { diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index aa862d80085f7..37a7db52e7e1a 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -22,7 +22,6 @@ use risingwave_common::cache::CachePriority; use risingwave_common::catalog::hummock::CompactionFilterFlag; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionExt; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{next_key, user_key}; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; diff --git a/src/storage/hummock_test/src/test_utils.rs b/src/storage/hummock_test/src/test_utils.rs index e06f798fc76a9..9ef7350e28228 100644 --- a/src/storage/hummock_test/src/test_utils.rs +++ b/src/storage/hummock_test/src/test_utils.rs @@ -29,14 +29,13 @@ use risingwave_meta::hummock::{HummockManagerRef, MockHummockMetaClient}; use risingwave_meta::manager::MetaSrvEnv; use risingwave_pb::catalog::{PbTable, Table}; use risingwave_pb::common::WorkerNode; -use risingwave_pb::hummock::version_update_payload; use risingwave_storage::error::StorageResult; use risingwave_storage::filter_key_extractor::{ FilterKeyExtractorImpl, FilterKeyExtractorManager, FullKeyFilterKeyExtractor, RpcFilterKeyExtractorManager, }; use risingwave_storage::hummock::backup_reader::BackupReader; -use risingwave_storage::hummock::event_handler::HummockEvent; +use risingwave_storage::hummock::event_handler::{HummockEvent, HummockVersionUpdate}; use risingwave_storage::hummock::iterator::test_utils::mock_sstable_store; use risingwave_storage::hummock::local_version::pinned_version::PinnedVersion; use risingwave_storage::hummock::observer_manager::HummockObserverNode; @@ -75,9 +74,7 @@ pub async fn prepare_first_valid_version( .await; observer_manager.start().await; let hummock_version = match rx.recv().await { - Some(HummockEvent::VersionUpdate(version_update_payload::Payload::PinnedVersion( - version, - ))) => version, + Some(HummockEvent::VersionUpdate(HummockVersionUpdate::PinnedVersion(version))) => version, _ => unreachable!("should be full version"), }; @@ -257,14 +254,7 @@ impl HummockTestEnv { pub async fn commit_epoch(&self, epoch: u64) { let res = self.storage.seal_and_sync_epoch(epoch).await.unwrap(); self.meta_client - .commit_epoch_with_watermark( - epoch, - res.uncommitted_ssts, - res.table_watermarks - .into_iter() - .map(|(table_id, watermark)| (table_id.table_id, watermark.to_protobuf())) - .collect(), - ) + .commit_epoch_with_watermark(epoch, res.uncommitted_ssts, res.table_watermarks) .await .unwrap(); diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 498c41c6e635d..5d043d42806c6 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -21,9 +21,7 @@ use arc_swap::ArcSwap; use await_tree::InstrumentAwait; use parking_lot::RwLock; use prometheus::core::{AtomicU64, GenericGauge}; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockVersionUpdateExt; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; -use risingwave_pb::hummock::version_update_payload::Payload; use tokio::spawn; use tokio::sync::{mpsc, oneshot}; use tracing::{debug, error, info, trace, warn}; @@ -37,7 +35,7 @@ use crate::hummock::event_handler::refiller::CacheRefillerEvent; use crate::hummock::event_handler::uploader::{ HummockUploader, SyncedData, UploadTaskInfo, UploadTaskPayload, UploaderEvent, }; -use crate::hummock::event_handler::HummockEvent; +use crate::hummock::event_handler::{HummockEvent, HummockVersionUpdate}; use crate::hummock::local_version::pinned_version::PinnedVersion; use crate::hummock::store::version::{ HummockReadVersion, StagingData, StagingSstableInfo, VersionUpdate, @@ -403,7 +401,7 @@ impl HummockEventHandler { }); } - fn handle_version_update(&mut self, version_payload: Payload) { + fn handle_version_update(&mut self, version_payload: HummockVersionUpdate) { let pinned_version = self .refiller .last_new_pinned_version() @@ -413,9 +411,9 @@ impl HummockEventHandler { let mut sst_delta_infos = vec![]; let newly_pinned_version = match version_payload { - Payload::VersionDeltas(version_deltas) => { - let mut version_to_apply = pinned_version.version(); - for version_delta in &version_deltas.version_deltas { + HummockVersionUpdate::VersionDeltas(version_deltas) => { + let mut version_to_apply = pinned_version.version().clone(); + for version_delta in &version_deltas { assert_eq!(version_to_apply.id, version_delta.prev_id); if version_to_apply.max_committed_epoch == version_delta.max_committed_epoch { sst_delta_infos = version_to_apply.build_sst_delta_infos(version_delta); @@ -425,7 +423,7 @@ impl HummockEventHandler { version_to_apply } - Payload::PinnedVersion(version) => version, + HummockVersionUpdate::PinnedVersion(version) => version, }; validate_table_key_range(&newly_pinned_version); diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index a6722b0d77116..b39b5fca708b4 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -18,7 +18,6 @@ use std::sync::Arc; use parking_lot::RwLock; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::HummockEpoch; -use risingwave_pb::hummock::version_update_payload; use tokio::sync::{mpsc, oneshot}; use crate::hummock::shared_buffer::shared_buffer_batch::SharedBufferBatch; @@ -31,6 +30,7 @@ pub mod refiller; pub mod uploader; pub use hummock_event_handler::HummockEventHandler; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use super::store::version::HummockReadVersion; @@ -41,6 +41,12 @@ pub struct BufferWriteRequest { pub grant_sender: oneshot::Sender<()>, } +#[derive(Debug)] +pub enum HummockVersionUpdate { + VersionDeltas(Vec), + PinnedVersion(HummockVersion), +} + pub enum HummockEvent { /// Notify that we may flush the shared buffer. BufferMayFlush, @@ -58,7 +64,7 @@ pub enum HummockEvent { Shutdown, - VersionUpdate(version_update_payload::Payload), + VersionUpdate(HummockVersionUpdate), ImmToUploader(ImmutableMemtable), diff --git a/src/storage/src/hummock/event_handler/uploader.rs b/src/storage/src/hummock/event_handler/uploader.rs index 66357753fd038..a23bacb940ae0 100644 --- a/src/storage/src/hummock/event_handler/uploader.rs +++ b/src/storage/src/hummock/event_handler/uploader.rs @@ -1146,8 +1146,9 @@ mod tests { use prometheus::core::GenericGauge; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; + use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; - use risingwave_pb::hummock::{HummockVersion, KeyRange, SstableInfo}; + use risingwave_pb::hummock::{KeyRange, SstableInfo}; use spin::Mutex; use tokio::spawn; use tokio::sync::mpsc::unbounded_channel; diff --git a/src/storage/src/hummock/hummock_meta_client.rs b/src/storage/src/hummock/hummock_meta_client.rs index d56334b802355..94af950f42936 100644 --- a/src/storage/src/hummock/hummock_meta_client.rs +++ b/src/storage/src/hummock/hummock_meta_client.rs @@ -16,10 +16,9 @@ use std::sync::Arc; use async_trait::async_trait; use futures::stream::BoxStream; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo, SstObjectIdRange}; -use risingwave_pb::hummock::{ - HummockSnapshot, HummockVersion, SubscribeCompactionEventRequest, VacuumTask, -}; +use risingwave_pb::hummock::{HummockSnapshot, SubscribeCompactionEventRequest, VacuumTask}; use risingwave_rpc_client::error::Result; use risingwave_rpc_client::{CompactionEventItem, HummockMetaClient, MetaClient}; use tokio::sync::mpsc::UnboundedSender; diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index 563d63f5c4705..14749c1fa1fc6 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -14,19 +14,16 @@ use std::collections::{BTreeMap, HashMap}; use std::iter::empty; -use std::ops::Deref; use std::sync::Arc; use std::time::{Duration, Instant}; use auto_enums::auto_enum; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - HummockVersionExt, HummockVersionUpdateExt, -}; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId, INVALID_VERSION_ID}; use risingwave_pb::hummock::hummock_version::Levels; -use risingwave_pb::hummock::{HummockVersion, Level}; +use risingwave_pb::hummock::PbLevel; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::error::TryRecvError; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; @@ -145,7 +142,7 @@ impl PinnedVersion { self.version.levels.get(&compaction_group_id).unwrap() } - pub fn levels(&self, table_id: TableId) -> impl Iterator { + pub fn levels(&self, table_id: TableId) -> impl Iterator { #[auto_enum(Iterator)] match self.compaction_group_index.get(&table_id) { Some(compaction_group_id) => { @@ -172,8 +169,8 @@ impl PinnedVersion { } /// ret value can't be used as `HummockVersion`. it must be modified with delta - pub fn version(&self) -> HummockVersion { - self.version.deref().clone() + pub fn version(&self) -> &HummockVersion { + &self.version } } diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index 3a0715e2417ad..9c455e33782aa 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -16,9 +16,9 @@ use std::collections::HashMap; use std::sync::Arc; use risingwave_common_service::observer_manager::{ObserverState, SubscribeHummock}; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_trace::TraceSpan; use risingwave_pb::catalog::Table; -use risingwave_pb::hummock::version_update_payload; use risingwave_pb::meta::relation::RelationInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::SubscribeResponse; @@ -26,7 +26,7 @@ use tokio::sync::mpsc::UnboundedSender; use crate::filter_key_extractor::{FilterKeyExtractorImpl, FilterKeyExtractorManagerRef}; use crate::hummock::backup_reader::BackupReaderRef; -use crate::hummock::event_handler::HummockEvent; +use crate::hummock::event_handler::{HummockEvent, HummockVersionUpdate}; use crate::hummock::write_limiter::WriteLimiterRef; pub struct HummockObserverNode { @@ -72,7 +72,13 @@ impl ObserverState for HummockObserverNode { let _ = self .version_update_sender .send(HummockEvent::VersionUpdate( - version_update_payload::Payload::VersionDeltas(hummock_version_deltas), + HummockVersionUpdate::VersionDeltas( + hummock_version_deltas + .version_deltas + .iter() + .map(HummockVersionDelta::from_rpc_protobuf) + .collect(), + ), )) .inspect_err(|e| { tracing::error!("unable to send version delta: {:?}", e); @@ -118,11 +124,11 @@ impl ObserverState for HummockObserverNode { let _ = self .version_update_sender .send(HummockEvent::VersionUpdate( - version_update_payload::Payload::PinnedVersion( - snapshot + HummockVersionUpdate::PinnedVersion(HummockVersion::from_rpc_protobuf( + &snapshot .hummock_version .expect("should get hummock version"), - ), + )), )) .inspect_err(|e| { tracing::error!("unable to send full version: {:?}", e); diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index bbdeec1ed67b3..3f1f34016d436 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -27,10 +27,9 @@ use risingwave_common::util::epoch::is_max_epoch; use risingwave_common_service::observer_manager::{NotificationClient, ObserverManager}; use risingwave_hummock_sdk::key::{is_empty_key_range, TableKey, TableKeyRange}; use risingwave_hummock_sdk::table_watermark::ReadTableWatermark; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::HummockReadEpoch; -#[cfg(any(test, feature = "test"))] -use risingwave_pb::hummock::HummockVersion; -use risingwave_pb::hummock::{version_update_payload, SstableInfo}; +use risingwave_pb::hummock::SstableInfo; use risingwave_rpc_client::HummockMetaClient; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; use tokio::sync::oneshot; @@ -44,7 +43,9 @@ use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; use crate::hummock::compactor::CompactorContext; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::refiller::CacheRefillConfig; -use crate::hummock::event_handler::{HummockEvent, HummockEventHandler, ReadVersionMappingType}; +use crate::hummock::event_handler::{ + HummockEvent, HummockEventHandler, HummockVersionUpdate, ReadVersionMappingType, +}; use crate::hummock::local_version::pinned_version::{start_pinned_version_worker, PinnedVersion}; use crate::hummock::observer_manager::HummockObserverNode; use crate::hummock::store::version::read_filter_for_batch; @@ -168,7 +169,7 @@ impl HummockStorage { observer_manager.start().await; let hummock_version = match event_rx.recv().await { - Some(HummockEvent::VersionUpdate(version_update_payload::Payload::PinnedVersion(version))) => version, + Some(HummockEvent::VersionUpdate(HummockVersionUpdate::PinnedVersion(version))) => version, _ => unreachable!("the hummock observer manager is the first one to take the event tx. Should be full hummock version") }; @@ -542,7 +543,7 @@ impl HummockStorage { let version_id = version.id; self.hummock_event_sender .send(HummockEvent::VersionUpdate( - version_update_payload::Payload::PinnedVersion(version), + HummockVersionUpdate::PinnedVersion(version), )) .unwrap(); loop { diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 8fcd81d6b75de..dc1c4afda73f8 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -30,8 +30,9 @@ use risingwave_hummock_sdk::key_range::KeyRangeCommon; use risingwave_hummock_sdk::table_watermark::{ ReadTableWatermark, TableWatermarksIndex, VnodeWatermark, WatermarkDirection, }; +use risingwave_hummock_sdk::version::HummockVersionDelta; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; -use risingwave_pb::hummock::{HummockVersionDelta, LevelType, SstableInfo}; +use risingwave_pb::hummock::{LevelType, SstableInfo}; use sync_point::sync_point; use tracing::Instrument; @@ -59,9 +60,6 @@ use crate::monitor::{ }; use crate::store::{gen_min_epoch, ReadOptions, StateStoreIterExt, StreamTypeOfIter}; -// TODO: use a custom data structure to allow in-place update instead of proto -// pub type CommittedVersion = HummockVersion; - pub type CommittedVersion = PinnedVersion; /// Data not committed to Hummock. There are two types of staging data: diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 6404d80bb265f..9edfa8431f2e8 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -26,8 +26,9 @@ use risingwave_common::catalog::{TableId, TableOption}; use risingwave_hummock_sdk::key::{ bound_table_key_range, EmptySliceRef, FullKey, TableKey, UserKey, }; +use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{can_concat, HummockEpoch}; -use risingwave_pb::hummock::{HummockVersion, SstableInfo}; +use risingwave_pb::hummock::SstableInfo; use tokio::sync::watch::Sender; use tokio::sync::Notify; diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index cf3e35b48c692..813e773fd17bf 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -32,9 +32,9 @@ use risingwave_common::config::{ use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_hummock_sdk::key::TableKey; +use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{CompactionGroupId, HummockEpoch, FIRST_VERSION_ID}; use risingwave_pb::common::WorkerType; -use risingwave_pb::hummock::{HummockVersion, HummockVersionDelta}; use risingwave_rpc_client::{HummockMetaClient, MetaClient}; use risingwave_storage::hummock::hummock_meta_client::MonitoredHummockMetaClient; use risingwave_storage::hummock::{CachePolicy, HummockStorage}; @@ -296,8 +296,7 @@ async fn pull_version_deltas( let res = meta_client .list_version_deltas(0, u32::MAX, u64::MAX) .await - .unwrap() - .version_deltas; + .unwrap(); if let Err(err) = shutdown_tx.send(()) { tracing::warn!("Failed to send shutdown to heartbeat task: {:?}", err); diff --git a/src/tests/compaction_test/src/delete_range_runner.rs b/src/tests/compaction_test/src/delete_range_runner.rs index d2acd7c754c74..2aab4679bb634 100644 --- a/src/tests/compaction_test/src/delete_range_runner.rs +++ b/src/tests/compaction_test/src/delete_range_runner.rs @@ -269,7 +269,7 @@ async fn compaction_test( ) .await .unwrap(); - let version = store.get_pinned_version().version(); + let version = store.get_pinned_version().version().clone(); let remote_version = meta_client.get_current_version().await.unwrap(); println!( "version-{}, remote version-{}", From d25c5da46c51581c1dd2dc4c627ebbda96573a3d Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Dec 2023 15:25:07 +0800 Subject: [PATCH 06/15] feat: show thread name in logs for playground and standalone (#14114) Signed-off-by: Bugen Zhao --- .../com/risingwave/tracing/TracingSlf4jImpl.java | 2 +- .../java/com/risingwave/java/binding/Binding.java | 3 ++- src/batch/src/task/task_manager.rs | 2 +- src/cmd_all/src/bin/risingwave.rs | 6 ++++-- src/connector/src/source/data_gen_util.rs | 2 +- src/frontend/src/session.rs | 2 +- src/jni_core/src/macros.rs | 4 ++-- src/jni_core/src/tracing_slf4j.rs | 7 ++++++- .../src/hummock/compactor/compaction_executor.rs | 2 +- src/stream/src/task/stream_manager.rs | 2 +- src/utils/runtime/src/lib.rs | 2 +- src/utils/runtime/src/logger.rs | 12 +++++++++++- 12 files changed, 32 insertions(+), 14 deletions(-) diff --git a/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jImpl.java b/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jImpl.java index 0f87988ab57dc..9354a6043a56a 100644 --- a/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jImpl.java +++ b/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jImpl.java @@ -27,6 +27,6 @@ public class TracingSlf4jImpl { public static final int TRACE = 4; public static void event(String name, int level, String message) { - Binding.tracingSlf4jEvent(name, level, message); + Binding.tracingSlf4jEvent(Thread.currentThread().getName(), name, level, message); } } diff --git a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java index 3001a180a15de..ff490982ccbd0 100644 --- a/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java +++ b/java/java-binding/src/main/java/com/risingwave/java/binding/Binding.java @@ -26,7 +26,8 @@ public class Binding { } } - public static native void tracingSlf4jEvent(String name, int level, String message); + public static native void tracingSlf4jEvent( + String threadName, String name, int level, String message); public static native int vnodeCount(); diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index b2f20ea6412a0..e24c9ee50b288 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -70,7 +70,7 @@ impl BatchManager { builder.worker_threads(worker_threads_num); } builder - .thread_name("risingwave-batch-tasks") + .thread_name("rw-batch") .enable_all() .build() .unwrap() diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index b7693c6fa06a2..27d24096487bf 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -193,14 +193,16 @@ fn main() -> Result<()> { fn playground(opts: PlaygroundOpts) { let settings = risingwave_rt::LoggerSettings::new("playground") - .with_target("risingwave_storage", Level::WARN); + .with_target("risingwave_storage", Level::WARN) + .with_thread_name(true); risingwave_rt::init_risingwave_logger(settings); risingwave_rt::main_okk(risingwave_cmd_all::playground(opts)).unwrap(); } fn standalone(opts: StandaloneOpts) { let settings = risingwave_rt::LoggerSettings::new("standalone") - .with_target("risingwave_storage", Level::WARN); + .with_target("risingwave_storage", Level::WARN) + .with_thread_name(true); risingwave_rt::init_risingwave_logger(settings); risingwave_rt::main_okk(risingwave_cmd_all::standalone(opts)).unwrap(); } diff --git a/src/connector/src/source/data_gen_util.rs b/src/connector/src/source/data_gen_util.rs index 7d990cf0ff8c9..001a726f93018 100644 --- a/src/connector/src/source/data_gen_util.rs +++ b/src/connector/src/source/data_gen_util.rs @@ -31,7 +31,7 @@ pub fn spawn_data_generation_stream( ) -> impl Stream + Send + 'static { static RUNTIME: LazyLock = LazyLock::new(|| { tokio::runtime::Builder::new_multi_thread() - .thread_name("risingwave-data-generation") + .thread_name("rw-datagen") .enable_all() .build() .expect("failed to build data-generation runtime") diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index e68cb3ff80d28..ba786e2bb34a3 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -436,7 +436,7 @@ impl FrontendEnv { Arc::new(BackgroundShutdownRuntime::from( Builder::new_multi_thread() .worker_threads(4) - .thread_name("frontend-compute-threads") + .thread_name("rw-batch-local") .enable_all() .build() .unwrap(), diff --git a/src/jni_core/src/macros.rs b/src/jni_core/src/macros.rs index 89b10b98b9d4c..97a9e60492844 100644 --- a/src/jni_core/src/macros.rs +++ b/src/jni_core/src/macros.rs @@ -442,7 +442,7 @@ macro_rules! for_all_plain_native_methods { ($macro:path $(,$args:tt)*) => { $macro! { { - public static native void tracingSlf4jEvent(String name, int level, String string); + public static native void tracingSlf4jEvent(String threadName, String name, int level, String string); public static native int vnodeCount(); @@ -882,7 +882,7 @@ mod tests { // This test shows the signature of all native methods let expected = expect_test::expect![[r#" [ - tracingSlf4jEvent (Ljava/lang/String;ILjava/lang/String;)V, + tracingSlf4jEvent (Ljava/lang/String;Ljava/lang/String;ILjava/lang/String;)V, vnodeCount ()I, iteratorNewHummock ([B)J, iteratorNewStreamChunk (J)J, diff --git a/src/jni_core/src/tracing_slf4j.rs b/src/jni_core/src/tracing_slf4j.rs index ce410b9bcb001..8f7222d11c647 100644 --- a/src/jni_core/src/tracing_slf4j.rs +++ b/src/jni_core/src/tracing_slf4j.rs @@ -23,11 +23,15 @@ use crate::{execute_and_catch, EnvParam}; #[no_mangle] pub(crate) extern "system" fn Java_com_risingwave_java_binding_Binding_tracingSlf4jEvent( env: EnvParam<'_>, + thread_name: JString<'_>, class_name: JString<'_>, level: jint, message: JString<'_>, ) { execute_and_catch(env, move |env| { + let thread_name = env.get_string(&thread_name)?; + let thread_name: Cow<'_, str> = (&thread_name).into(); + let class_name = env.get_string(&class_name)?; let class_name: Cow<'_, str> = (&class_name).into(); @@ -39,7 +43,8 @@ pub(crate) extern "system" fn Java_com_risingwave_java_binding_Binding_tracingSl tracing::event!( target: "risingwave_connector_node", $lvl, - class = class_name.as_ref(), + thread = &*thread_name, + class = &*class_name, "{message}", ) }; diff --git a/src/storage/src/hummock/compactor/compaction_executor.rs b/src/storage/src/hummock/compactor/compaction_executor.rs index d3086709a33de..426245a89a392 100644 --- a/src/storage/src/hummock/compactor/compaction_executor.rs +++ b/src/storage/src/hummock/compactor/compaction_executor.rs @@ -30,7 +30,7 @@ impl CompactionExecutor { let mut worker_num = resource_util::cpu::total_cpu_available() as usize; let runtime = { let mut builder = tokio::runtime::Builder::new_multi_thread(); - builder.thread_name("risingwave-compaction"); + builder.thread_name("rw-compaction"); if let Some(worker_threads_num) = worker_threads_num { builder.worker_threads(worker_threads_num); worker_num = worker_threads_num; diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index 013873f17697e..579cb7810c49f 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -427,7 +427,7 @@ impl LocalStreamManagerCore { builder.worker_threads(worker_threads_num); } builder - .thread_name("risingwave-streaming-actor") + .thread_name("rw-streaming") .enable_all() .build() .unwrap() diff --git a/src/utils/runtime/src/lib.rs b/src/utils/runtime/src/lib.rs index 343550246e243..50577c8df1565 100644 --- a/src/utils/runtime/src/lib.rs +++ b/src/utils/runtime/src/lib.rs @@ -74,7 +74,7 @@ where } tokio::runtime::Builder::new_multi_thread() - .thread_name("risingwave-main") + .thread_name("rw-main") .enable_all() .build() .unwrap() diff --git a/src/utils/runtime/src/logger.rs b/src/utils/runtime/src/logger.rs index 18794b257f79f..e8abb0aef65eb 100644 --- a/src/utils/runtime/src/logger.rs +++ b/src/utils/runtime/src/logger.rs @@ -38,6 +38,8 @@ pub struct LoggerSettings { colorful: bool, /// Output to `stderr` instead of `stdout`. stderr: bool, + /// Whether to include thread name in the log. + with_thread_name: bool, /// Override target settings. targets: Vec<(String, tracing::metadata::LevelFilter)>, /// Override the default level. @@ -57,6 +59,7 @@ impl LoggerSettings { enable_tokio_console: false, colorful: console::colors_enabled_stderr() && console::colors_enabled(), stderr: false, + with_thread_name: false, targets: vec![], default_level: None, } @@ -74,6 +77,12 @@ impl LoggerSettings { self } + /// Whether to include thread name in the log. + pub fn with_thread_name(mut self, enabled: bool) -> Self { + self.with_thread_name = enabled; + self + } + /// Overrides the default target settings. pub fn with_target( mut self, @@ -210,6 +219,7 @@ pub fn init_risingwave_logger(settings: LoggerSettings) { // fmt layer (formatting and logging to `stdout` or `stderr`) { let fmt_layer = tracing_subscriber::fmt::layer() + .with_thread_names(settings.with_thread_name) .with_timer(default_timer.clone()) .with_ansi(settings.colorful) .with_writer(move || { @@ -359,7 +369,7 @@ pub fn init_risingwave_logger(settings: LoggerSettings) { let otel_tracer = { let runtime = tokio::runtime::Builder::new_multi_thread() .enable_all() - .thread_name("risingwave-otel") + .thread_name("rw-otel") .worker_threads(2) .build() .unwrap(); From b1e261a925bd607ae0c9ad472de8082efa7752f2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 26 Dec 2023 15:53:38 +0800 Subject: [PATCH 07/15] chore(deps): Bump sysinfo from 0.29.10 to 0.30.0 (#14138) Signed-off-by: dependabot[bot] Signed-off-by: Richard Chien Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Richard Chien --- Cargo.lock | 29 +++++++++++++++++++++++----- src/common/Cargo.toml | 2 +- src/common/src/telemetry/mod.rs | 15 +++++--------- src/common/src/util/resource_util.rs | 2 +- 4 files changed, 31 insertions(+), 17 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 6055509d0a06a..b6b6ba743eed8 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -4189,7 +4189,7 @@ dependencies = [ "libc", "log", "rustversion", - "windows", + "windows 0.48.0", ] [[package]] @@ -4678,7 +4678,7 @@ dependencies = [ "iana-time-zone-haiku", "js-sys", "wasm-bindgen", - "windows", + "windows 0.48.0", ] [[package]] @@ -11023,16 +11023,16 @@ checksum = "2047c6ded9c721764247e62cd3b03c09ffc529b2ba5b10ec482ae507a4a70160" [[package]] name = "sysinfo" -version = "0.29.10" +version = "0.30.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0a18d114d420ada3a891e6bc8e96a2023402203296a47cdd65083377dad18ba5" +checksum = "c68492e7268037de59ae153d7efb79546cf94a18a9548235420d3d8d2436b4b1" dependencies = [ "cfg-if", "core-foundation-sys", "libc", "ntapi", "once_cell", - "winapi", + "windows 0.51.1", ] [[package]] @@ -12162,6 +12162,25 @@ dependencies = [ "windows-targets 0.48.5", ] +[[package]] +name = "windows" +version = "0.51.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ca229916c5ee38c2f2bc1e9d8f04df975b4bd93f9955dc69fabb5d91270045c9" +dependencies = [ + "windows-core", + "windows-targets 0.48.5", +] + +[[package]] +name = "windows-core" +version = "0.51.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f1f8cf84f35d2db49a46868f947758c7a1138116f7fac3bc844f43ade1292e64" +dependencies = [ + "windows-targets 0.48.5", +] + [[package]] name = "windows-sys" version = "0.45.0" diff --git a/src/common/Cargo.toml b/src/common/Cargo.toml index f47d32862cc42..02f0a9bdda539 100644 --- a/src/common/Cargo.toml +++ b/src/common/Cargo.toml @@ -93,7 +93,7 @@ speedate = "0.13.0" static_assertions = "1" strum = "0.25" strum_macros = "0.25" -sysinfo = { version = "0.29", default-features = false } +sysinfo = { version = "0.30", default-features = false } thiserror = "1" thiserror-ext = { workspace = true } tinyvec = { version = "1", features = ["rustc_1_55", "grab_spare_slice"] } diff --git a/src/common/src/telemetry/mod.rs b/src/common/src/telemetry/mod.rs index 42d6ef1eaafcf..092baa465c7f2 100644 --- a/src/common/src/telemetry/mod.rs +++ b/src/common/src/telemetry/mod.rs @@ -18,7 +18,7 @@ pub mod report; use std::time::SystemTime; use serde::{Deserialize, Serialize}; -use sysinfo::{System, SystemExt}; +use sysinfo::System; use thiserror_ext::AsReport; use crate::util::env_var::env_var_is_true_or; @@ -95,21 +95,16 @@ struct Cpu { impl SystemData { pub fn new() -> Self { - let mut sys = System::new(); - let memory = { let total = system_memory_available_bytes(); let used = total_memory_used_bytes(); Memory { used, total } }; - let os = { - sys.refresh_system(); - Os { - name: sys.name().unwrap_or_default(), - kernel_version: sys.kernel_version().unwrap_or_default(), - version: sys.os_version().unwrap_or_default(), - } + let os = Os { + name: System::name().unwrap_or_default(), + kernel_version: System::kernel_version().unwrap_or_default(), + version: System::os_version().unwrap_or_default(), }; let cpu = Cpu { diff --git a/src/common/src/util/resource_util.rs b/src/common/src/util/resource_util.rs index 3a3825a14e95e..d97182e062666 100644 --- a/src/common/src/util/resource_util.rs +++ b/src/common/src/util/resource_util.rs @@ -132,7 +132,7 @@ mod runtime { } pub mod memory { - use sysinfo::{System, SystemExt}; + use sysinfo::System; use super::runtime::get_resource; From b1c61b7a9c1089b29546fd3aa3823f6c2a12a2a1 Mon Sep 17 00:00:00 2001 From: Richard Chien Date: Tue, 26 Dec 2023 15:57:09 +0800 Subject: [PATCH 08/15] test: run over window e2e test in all cache policy modes (#13951) Signed-off-by: Richard Chien --- .../over_window/generated/batch/main.slt.part | 32 ++++++++++++++----- .../generated/batch/run_all.slt.part | 8 +++++ .../generated/streaming/main.slt.part | 32 ++++++++++++++----- .../generated/streaming/run_all.slt.part | 8 +++++ e2e_test/over_window/templates/main.slt.part | 32 ++++++++++++++----- .../over_window/templates/run_all.slt.part | 6 ++++ 6 files changed, 94 insertions(+), 24 deletions(-) create mode 100644 e2e_test/over_window/generated/batch/run_all.slt.part create mode 100644 e2e_test/over_window/generated/streaming/run_all.slt.part create mode 100644 e2e_test/over_window/templates/run_all.slt.part diff --git a/e2e_test/over_window/generated/batch/main.slt.part b/e2e_test/over_window/generated/batch/main.slt.part index 9ef787964fafe..2f9c16987df49 100644 --- a/e2e_test/over_window/generated/batch/main.slt.part +++ b/e2e_test/over_window/generated/batch/main.slt.part @@ -1,11 +1,27 @@ # This file is generated by `gen.py`. Do not edit it manually! statement ok -SET RW_IMPLICIT_FLUSH TO true; - -include ./basic/mod.slt.part -include ./rank_func/mod.slt.part -include ./expr_in_win_func/mod.slt.part -include ./agg_in_win_func/mod.slt.part -include ./opt_agg_then_join/mod.slt.part -include ./with_filter/mod.slt.part +set rw_implicit_flush = true; + +statement ok +set rw_streaming_over_window_cache_policy = full; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_first_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_last_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = default; diff --git a/e2e_test/over_window/generated/batch/run_all.slt.part b/e2e_test/over_window/generated/batch/run_all.slt.part new file mode 100644 index 0000000000000..07a7f538a7ba4 --- /dev/null +++ b/e2e_test/over_window/generated/batch/run_all.slt.part @@ -0,0 +1,8 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part +include ./with_filter/mod.slt.part diff --git a/e2e_test/over_window/generated/streaming/main.slt.part b/e2e_test/over_window/generated/streaming/main.slt.part index 9ef787964fafe..2f9c16987df49 100644 --- a/e2e_test/over_window/generated/streaming/main.slt.part +++ b/e2e_test/over_window/generated/streaming/main.slt.part @@ -1,11 +1,27 @@ # This file is generated by `gen.py`. Do not edit it manually! statement ok -SET RW_IMPLICIT_FLUSH TO true; - -include ./basic/mod.slt.part -include ./rank_func/mod.slt.part -include ./expr_in_win_func/mod.slt.part -include ./agg_in_win_func/mod.slt.part -include ./opt_agg_then_join/mod.slt.part -include ./with_filter/mod.slt.part +set rw_implicit_flush = true; + +statement ok +set rw_streaming_over_window_cache_policy = full; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_first_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_last_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = default; diff --git a/e2e_test/over_window/generated/streaming/run_all.slt.part b/e2e_test/over_window/generated/streaming/run_all.slt.part new file mode 100644 index 0000000000000..07a7f538a7ba4 --- /dev/null +++ b/e2e_test/over_window/generated/streaming/run_all.slt.part @@ -0,0 +1,8 @@ +# This file is generated by `gen.py`. Do not edit it manually! + +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part +include ./with_filter/mod.slt.part diff --git a/e2e_test/over_window/templates/main.slt.part b/e2e_test/over_window/templates/main.slt.part index 5cd945b123ea1..26513fab2ff49 100644 --- a/e2e_test/over_window/templates/main.slt.part +++ b/e2e_test/over_window/templates/main.slt.part @@ -1,9 +1,25 @@ statement ok -SET RW_IMPLICIT_FLUSH TO true; - -include ./basic/mod.slt.part -include ./rank_func/mod.slt.part -include ./expr_in_win_func/mod.slt.part -include ./agg_in_win_func/mod.slt.part -include ./opt_agg_then_join/mod.slt.part -include ./with_filter/mod.slt.part +set rw_implicit_flush = true; + +statement ok +set rw_streaming_over_window_cache_policy = full; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_first_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = recent_last_n; + +include ./run_all.slt.part + +statement ok +set rw_streaming_over_window_cache_policy = default; diff --git a/e2e_test/over_window/templates/run_all.slt.part b/e2e_test/over_window/templates/run_all.slt.part new file mode 100644 index 0000000000000..d939acb58fc08 --- /dev/null +++ b/e2e_test/over_window/templates/run_all.slt.part @@ -0,0 +1,6 @@ +include ./basic/mod.slt.part +include ./rank_func/mod.slt.part +include ./expr_in_win_func/mod.slt.part +include ./agg_in_win_func/mod.slt.part +include ./opt_agg_then_join/mod.slt.part +include ./with_filter/mod.slt.part From 590fd7edf79c9fa06a91d0401ca3c34670506aa1 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Dec 2023 16:01:06 +0800 Subject: [PATCH 09/15] feat(storage): compare deserialized row value in sanity check (#14178) --- .../hummock_test/src/state_store_tests.rs | 2 +- src/storage/hummock_trace/src/opts.rs | 10 +- .../hummock/store/local_hummock_storage.rs | 29 +++--- src/storage/src/hummock/utils.rs | 52 ++++++---- src/storage/src/mem_table.rs | 70 +++++++------ src/storage/src/store.rs | 48 ++++++--- .../common/log_store_impl/kv_log_store/mod.rs | 4 +- src/stream/src/common/table/state_table.rs | 99 ++++++++++++++----- src/stream/src/executor/mview/materialize.rs | 11 +-- 9 files changed, 211 insertions(+), 114 deletions(-) diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 9babe12b9054b..c59b027b44bc4 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -1381,7 +1381,7 @@ async fn test_replicated_local_hummock_storage() { let mut local_hummock_storage = hummock_storage .new_local(NewLocalOptions::new_replicated( TEST_TABLE_ID, - false, + OpConsistencyLevel::Inconsistent, TableOption { retention_seconds: None, }, diff --git a/src/storage/hummock_trace/src/opts.rs b/src/storage/hummock_trace/src/opts.rs index dfa3ba46ac4e7..046d90a7e18d3 100644 --- a/src/storage/hummock_trace/src/opts.rs +++ b/src/storage/hummock_trace/src/opts.rs @@ -135,10 +135,16 @@ impl From for TableOption { } } +#[derive(Encode, Decode, PartialEq, Eq, Debug, Clone, Copy, Hash)] +pub enum TracedOpConsistencyLevel { + Inconsistent, + ConsistentOldValue, +} + #[derive(Encode, Decode, PartialEq, Eq, Debug, Clone, Copy, Hash)] pub struct TracedNewLocalOptions { pub table_id: TracedTableId, - pub is_consistent_op: bool, + pub op_consistency_level: TracedOpConsistencyLevel, pub table_option: TracedTableOption, pub is_replicated: bool, } @@ -148,7 +154,7 @@ impl TracedNewLocalOptions { pub(crate) fn for_test(table_id: u32) -> Self { Self { table_id: TracedTableId { table_id }, - is_consistent_op: true, + op_consistency_level: TracedOpConsistencyLevel::Inconsistent, table_option: TracedTableOption { retention_seconds: None, }, diff --git a/src/storage/src/hummock/store/local_hummock_storage.rs b/src/storage/src/hummock/store/local_hummock_storage.rs index 72c24cc5b1cb7..8e489a5ebd975 100644 --- a/src/storage/src/hummock/store/local_hummock_storage.rs +++ b/src/storage/src/hummock/store/local_hummock_storage.rs @@ -58,7 +58,7 @@ pub struct LocalHummockStorage { epoch: Option, table_id: TableId, - is_consistent_op: bool, + op_consistency_level: OpConsistencyLevel, table_option: TableOption, instance_guard: LocalInstanceGuard, @@ -300,43 +300,46 @@ impl LocalStateStore for LocalHummockStorage { // a workaround you may call disable the check by initializing the // state store with `is_consistent_op=false`. KeyOp::Insert(value) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_insert_sanity_check( - key.clone(), - value.clone(), + &key, + &value, self, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } kv_pairs.push((key, StorageValue::new_put(value))); } KeyOp::Delete(old_value) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_delete_sanity_check( - key.clone(), - old_value, + &key, + &old_value, self, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } kv_pairs.push((key, StorageValue::new_delete())); } KeyOp::Update((old_value, new_value)) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_update_sanity_check( - key.clone(), - old_value, - new_value.clone(), + &key, + &old_value, + &new_value, self, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } @@ -530,11 +533,11 @@ impl LocalHummockStorage { ) -> Self { let stats = hummock_version_reader.stats().clone(); Self { - mem_table: MemTable::new(option.is_consistent_op), + mem_table: MemTable::new(option.op_consistency_level.clone()), spill_offset: 0, epoch: None, table_id: option.table_id, - is_consistent_op: option.is_consistent_op, + op_consistency_level: option.op_consistency_level, table_option: option.table_option, is_replicated: option.is_replicated, instance_guard, diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 9edfa8431f2e8..c32c0a02128ce 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -36,7 +36,7 @@ use super::{HummockError, HummockResult}; use crate::error::StorageResult; use crate::hummock::CachePolicy; use crate::mem_table::{KeyOp, MemTableError}; -use crate::store::{ReadOptions, StateStoreRead}; +use crate::store::{OpConsistencyLevel, ReadOptions, StateStoreRead}; pub fn range_overlap( search_key_range: &R, @@ -373,13 +373,17 @@ pub(crate) const ENABLE_SANITY_CHECK: bool = cfg!(debug_assertions); /// Make sure the key to insert should not exist in storage. pub(crate) async fn do_insert_sanity_check( - key: TableKey, - value: Bytes, + key: &TableKey, + value: &Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, table_option: TableOption, + op_consistency_level: &OpConsistencyLevel, ) -> StorageResult<()> { + if let OpConsistencyLevel::Inconsistent = op_consistency_level { + return Ok(()); + } let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, @@ -390,9 +394,9 @@ pub(crate) async fn do_insert_sanity_check( if let Some(stored_value) = stored_value { return Err(Box::new(MemTableError::InconsistentOperation { - key, + key: key.clone(), prev: KeyOp::Insert(stored_value), - new: KeyOp::Insert(value), + new: KeyOp::Insert(value.clone()), }) .into()); } @@ -401,13 +405,17 @@ pub(crate) async fn do_insert_sanity_check( /// Make sure that the key to delete should exist in storage and the value should be matched. pub(crate) async fn do_delete_sanity_check( - key: TableKey, - old_value: Bytes, + key: &TableKey, + old_value: &Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, table_option: TableOption, + op_consistency_level: &OpConsistencyLevel, ) -> StorageResult<()> { + let OpConsistencyLevel::ConsistentOldValue(old_value_checker) = op_consistency_level else { + return Ok(()); + }; let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, @@ -416,17 +424,17 @@ pub(crate) async fn do_delete_sanity_check( }; match inner.get(key.clone(), epoch, read_options).await? { None => Err(Box::new(MemTableError::InconsistentOperation { - key, + key: key.clone(), prev: KeyOp::Delete(Bytes::default()), - new: KeyOp::Delete(old_value), + new: KeyOp::Delete(old_value.clone()), }) .into()), Some(stored_value) => { - if stored_value != old_value { + if !old_value_checker(&stored_value, old_value) { Err(Box::new(MemTableError::InconsistentOperation { - key, + key: key.clone(), prev: KeyOp::Insert(stored_value), - new: KeyOp::Delete(old_value), + new: KeyOp::Delete(old_value.clone()), }) .into()) } else { @@ -438,14 +446,18 @@ pub(crate) async fn do_delete_sanity_check( /// Make sure that the key to update should exist in storage and the value should be matched pub(crate) async fn do_update_sanity_check( - key: TableKey, - old_value: Bytes, - new_value: Bytes, + key: &TableKey, + old_value: &Bytes, + new_value: &Bytes, inner: &impl StateStoreRead, epoch: u64, table_id: TableId, table_option: TableOption, + op_consistency_level: &OpConsistencyLevel, ) -> StorageResult<()> { + let OpConsistencyLevel::ConsistentOldValue(old_value_checker) = op_consistency_level else { + return Ok(()); + }; let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, @@ -455,17 +467,17 @@ pub(crate) async fn do_update_sanity_check( match inner.get(key.clone(), epoch, read_options).await? { None => Err(Box::new(MemTableError::InconsistentOperation { - key, + key: key.clone(), prev: KeyOp::Delete(Bytes::default()), - new: KeyOp::Update((old_value, new_value)), + new: KeyOp::Update((old_value.clone(), new_value.clone())), }) .into()), Some(stored_value) => { - if stored_value != old_value { + if !old_value_checker(&stored_value, old_value) { Err(Box::new(MemTableError::InconsistentOperation { - key, + key: key.clone(), prev: KeyOp::Insert(stored_value), - new: KeyOp::Update((old_value, new_value)), + new: KeyOp::Update((old_value.clone(), new_value.clone())), }) .into()) } else { diff --git a/src/storage/src/mem_table.rs b/src/storage/src/mem_table.rs index eca66bfba2f74..6adf9a7ef5d7e 100644 --- a/src/storage/src/mem_table.rs +++ b/src/storage/src/mem_table.rs @@ -55,7 +55,7 @@ pub enum KeyOp { #[derive(Clone)] pub struct MemTable { pub(crate) buffer: MemTableStore, - pub(crate) is_consistent_op: bool, + pub(crate) op_consistency_level: OpConsistencyLevel, pub(crate) kv_size: KvSize, } @@ -108,17 +108,17 @@ impl RustIteratorBuilder for MemTableIteratorBuilder { pub type MemTableHummockIterator<'a> = FromRustIterator<'a, MemTableIteratorBuilder>; impl MemTable { - pub fn new(is_consistent_op: bool) -> Self { + pub fn new(op_consistency_level: OpConsistencyLevel) -> Self { Self { buffer: BTreeMap::new(), - is_consistent_op, + op_consistency_level, kv_size: KvSize::new(), } } pub fn drain(&mut self) -> Self { self.kv_size.set(0); - std::mem::replace(self, Self::new(self.is_consistent_op)) + std::mem::replace(self, Self::new(self.op_consistency_level.clone())) } pub fn is_dirty(&self) -> bool { @@ -127,7 +127,7 @@ impl MemTable { /// write methods pub fn insert(&mut self, pk: TableKey, value: Bytes) -> Result<()> { - if !self.is_consistent_op { + if let OpConsistencyLevel::Inconsistent = &self.op_consistency_level { let key_len = std::mem::size_of::() + pk.len(); let insert_value = KeyOp::Insert(value); self.kv_size.add(&pk, &insert_value); @@ -135,7 +135,7 @@ impl MemTable { self.sub_origin_size(origin_value, key_len); return Ok(()); - } + }; let entry = self.buffer.entry(pk); match entry { Entry::Vacant(e) => { @@ -170,13 +170,14 @@ impl MemTable { pub fn delete(&mut self, pk: TableKey, old_value: Bytes) -> Result<()> { let key_len = std::mem::size_of::() + pk.len(); - if !self.is_consistent_op { + let OpConsistencyLevel::ConsistentOldValue(value_checker) = &self.op_consistency_level + else { let delete_value = KeyOp::Delete(old_value); self.kv_size.add(&pk, &delete_value); let origin_value = self.buffer.insert(pk, delete_value); self.sub_origin_size(origin_value, key_len); return Ok(()); - } + }; let entry = self.buffer.entry(pk); match entry { Entry::Vacant(e) => { @@ -190,7 +191,7 @@ impl MemTable { self.kv_size.sub_val(origin_value); match origin_value { KeyOp::Insert(original_value) => { - if ENABLE_SANITY_CHECK && original_value != &old_value { + if ENABLE_SANITY_CHECK && !value_checker(original_value, &old_value) { return Err(Box::new(MemTableError::InconsistentOperation { key: e.key().clone(), prev: e.get().clone(), @@ -234,7 +235,8 @@ impl MemTable { old_value: Bytes, new_value: Bytes, ) -> Result<()> { - if !self.is_consistent_op { + let OpConsistencyLevel::ConsistentOldValue(value_checker) = &self.op_consistency_level + else { let key_len = std::mem::size_of::() + pk.len(); let update_value = KeyOp::Update((old_value, new_value)); @@ -242,7 +244,7 @@ impl MemTable { let origin_value = self.buffer.insert(pk, update_value); self.sub_origin_size(origin_value, key_len); return Ok(()); - } + }; let entry = self.buffer.entry(pk); match entry { Entry::Vacant(e) => { @@ -256,7 +258,7 @@ impl MemTable { self.kv_size.sub_val(origin_value); match origin_value { KeyOp::Insert(original_new_value) => { - if ENABLE_SANITY_CHECK && original_new_value != &old_value { + if ENABLE_SANITY_CHECK && !value_checker(original_new_value, &old_value) { return Err(Box::new(MemTableError::InconsistentOperation { key: e.key().clone(), prev: e.get().clone(), @@ -269,7 +271,7 @@ impl MemTable { Ok(()) } KeyOp::Update((origin_old_value, original_new_value)) => { - if ENABLE_SANITY_CHECK && original_new_value != &old_value { + if ENABLE_SANITY_CHECK && !value_checker(original_new_value, &old_value) { return Err(Box::new(MemTableError::InconsistentOperation { key: e.key().clone(), prev: e.get().clone(), @@ -427,7 +429,7 @@ pub struct MemtableLocalStateStore { epoch: Option, table_id: TableId, - is_consistent_op: bool, + op_consistency_level: OpConsistencyLevel, table_option: TableOption, } @@ -435,10 +437,10 @@ impl MemtableLocalStateStore { pub fn new(inner: S, option: NewLocalOptions) -> Self { Self { inner, - mem_table: MemTable::new(option.is_consistent_op), + mem_table: MemTable::new(option.op_consistency_level.clone()), epoch: None, table_id: option.table_id, - is_consistent_op: option.is_consistent_op, + op_consistency_level: option.op_consistency_level, table_option: option.table_option, } } @@ -525,45 +527,48 @@ impl LocalStateStore for MemtableLocalState match key_op { // Currently, some executors do not strictly comply with these semantics. As // a workaround you may call disable the check by initializing the - // state store with `is_consistent_op=false`. + // state store with `op_consistency_level=Inconsistent`. KeyOp::Insert(value) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_insert_sanity_check( - key.clone(), - value.clone(), + &key, + &value, &self.inner, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } kv_pairs.push((key, StorageValue::new_put(value))); } KeyOp::Delete(old_value) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_delete_sanity_check( - key.clone(), - old_value, + &key, + &old_value, &self.inner, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } kv_pairs.push((key, StorageValue::new_delete())); } KeyOp::Update((old_value, new_value)) => { - if ENABLE_SANITY_CHECK && self.is_consistent_op { + if ENABLE_SANITY_CHECK { do_update_sanity_check( - key.clone(), - old_value, - new_value.clone(), + &key, + &old_value, + &new_value, &self.inner, self.epoch(), self.table_id, self.table_option, + &self.op_consistency_level, ) .await?; } @@ -637,10 +642,13 @@ mod tests { use crate::hummock::iterator::HummockIterator; use crate::hummock::value::HummockValue; use crate::mem_table::{KeyOp, MemTable, MemTableHummockIterator}; + use crate::store::{OpConsistencyLevel, CHECK_BYTES_EQUAL}; #[tokio::test] async fn test_mem_table_memory_size() { - let mut mem_table = MemTable::new(true); + let mut mem_table = MemTable::new(OpConsistencyLevel::ConsistentOldValue( + CHECK_BYTES_EQUAL.clone(), + )); assert_eq!(mem_table.kv_size.size(), 0); mem_table @@ -750,7 +758,7 @@ mod tests { #[tokio::test] async fn test_mem_table_memory_size_not_consistent_op() { - let mut mem_table = MemTable::new(false); + let mut mem_table = MemTable::new(OpConsistencyLevel::Inconsistent); assert_eq!(mem_table.kv_size.size(), 0); mem_table @@ -833,7 +841,9 @@ mod tests { let mut test_data = ordered_test_data.clone(); test_data.shuffle(&mut rng); - let mut mem_table = MemTable::new(true); + let mut mem_table = MemTable::new(OpConsistencyLevel::ConsistentOldValue( + CHECK_BYTES_EQUAL.clone(), + )); for (key, op) in test_data { match op { KeyOp::Insert(value) => { diff --git a/src/storage/src/store.rs b/src/storage/src/store.rs index cf5211d7069e5..cbdd719e486d3 100644 --- a/src/storage/src/store.rs +++ b/src/storage/src/store.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use std::default::Default; use std::future::Future; use std::ops::Bound; -use std::sync::Arc; +use std::sync::{Arc, LazyLock}; use bytes::Bytes; use futures::{Stream, StreamExt, TryStreamExt}; @@ -30,8 +30,8 @@ use risingwave_hummock_sdk::table_watermark::{ }; use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo}; use risingwave_hummock_trace::{ - TracedInitOptions, TracedNewLocalOptions, TracedPrefetchOptions, TracedReadOptions, - TracedSealCurrentEpochOptions, TracedWriteOptions, + TracedInitOptions, TracedNewLocalOptions, TracedOpConsistencyLevel, TracedPrefetchOptions, + TracedReadOptions, TracedSealCurrentEpochOptions, TracedWriteOptions, }; use crate::error::{StorageError, StorageResult}; @@ -397,6 +397,18 @@ impl From for WriteOptions { } } +pub trait CheckOldValueEquality = Fn(&Bytes, &Bytes) -> bool + Send + Sync; + +pub static CHECK_BYTES_EQUAL: LazyLock> = + LazyLock::new(|| Arc::new(|first: &Bytes, second: &Bytes| first == second)); + +#[derive(Default, Clone)] +pub enum OpConsistencyLevel { + #[default] + Inconsistent, + ConsistentOldValue(Arc), +} + #[derive(Clone, Default)] pub struct NewLocalOptions { pub table_id: TableId, @@ -407,7 +419,7 @@ pub struct NewLocalOptions { /// /// 2. The old value passed from /// `update` and `delete` should match the original stored value. - pub is_consistent_op: bool, + pub op_consistency_level: OpConsistencyLevel, pub table_option: TableOption, /// Indicate if this is replicated. If it is, we should not @@ -419,7 +431,12 @@ impl From for NewLocalOptions { fn from(value: TracedNewLocalOptions) -> Self { Self { table_id: value.table_id.into(), - is_consistent_op: value.is_consistent_op, + op_consistency_level: match value.op_consistency_level { + TracedOpConsistencyLevel::Inconsistent => OpConsistencyLevel::Inconsistent, + TracedOpConsistencyLevel::ConsistentOldValue => { + OpConsistencyLevel::ConsistentOldValue(CHECK_BYTES_EQUAL.clone()) + } + }, table_option: value.table_option.into(), is_replicated: value.is_replicated, } @@ -430,7 +447,12 @@ impl From for TracedNewLocalOptions { fn from(value: NewLocalOptions) -> Self { Self { table_id: value.table_id.into(), - is_consistent_op: value.is_consistent_op, + op_consistency_level: match value.op_consistency_level { + OpConsistencyLevel::Inconsistent => TracedOpConsistencyLevel::Inconsistent, + OpConsistencyLevel::ConsistentOldValue(_) => { + TracedOpConsistencyLevel::ConsistentOldValue + } + }, table_option: value.table_option.into(), is_replicated: value.is_replicated, } @@ -438,10 +460,14 @@ impl From for TracedNewLocalOptions { } impl NewLocalOptions { - pub fn new(table_id: TableId, is_consistent_op: bool, table_option: TableOption) -> Self { + pub fn new( + table_id: TableId, + op_consistency_level: OpConsistencyLevel, + table_option: TableOption, + ) -> Self { NewLocalOptions { table_id, - is_consistent_op, + op_consistency_level, table_option, is_replicated: false, } @@ -449,12 +475,12 @@ impl NewLocalOptions { pub fn new_replicated( table_id: TableId, - is_consistent_op: bool, + op_consistency_level: OpConsistencyLevel, table_option: TableOption, ) -> Self { NewLocalOptions { table_id, - is_consistent_op, + op_consistency_level, table_option, is_replicated: true, } @@ -463,7 +489,7 @@ impl NewLocalOptions { pub fn for_test(table_id: TableId) -> Self { Self { table_id, - is_consistent_op: false, + op_consistency_level: OpConsistencyLevel::Inconsistent, table_option: TableOption { retention_seconds: None, }, diff --git a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs index 74358f7944c3b..ca6137db47a70 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/mod.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/mod.rs @@ -20,7 +20,7 @@ use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntCounter}; use risingwave_connector::sink::log_store::LogStoreFactory; use risingwave_connector::sink::{SinkParam, SinkWriterParam}; use risingwave_pb::catalog::Table; -use risingwave_storage::store::NewLocalOptions; +use risingwave_storage::store::{NewLocalOptions, OpConsistencyLevel}; use risingwave_storage::StateStore; use tokio::sync::watch; @@ -237,7 +237,7 @@ impl LogStoreFactory for KvLogStoreFactory { table_id: TableId { table_id: self.table_catalog.id, }, - is_consistent_op: false, + op_consistency_level: OpConsistencyLevel::Inconsistent, table_option: TableOption { retention_seconds: None, }, diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index d887684686977..7126b67147ed8 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -54,8 +54,8 @@ use risingwave_storage::row_serde::row_serde_util::{ }; use risingwave_storage::row_serde::value_serde::ValueRowSerde; use risingwave_storage::store::{ - InitOptions, LocalStateStore, NewLocalOptions, PrefetchOptions, ReadOptions, - SealCurrentEpochOptions, StateStoreIterItemStream, + InitOptions, LocalStateStore, NewLocalOptions, OpConsistencyLevel, PrefetchOptions, + ReadOptions, SealCurrentEpochOptions, StateStoreIterItemStream, }; use risingwave_storage::table::merge_sort::merge_sort; use risingwave_storage::table::{compute_chunk_vnode, compute_vnode, Distribution, KeyedRow}; @@ -212,6 +212,34 @@ where } } +fn consistent_old_value_op(row_serde: impl ValueRowSerde) -> OpConsistencyLevel { + OpConsistencyLevel::ConsistentOldValue(Arc::new(move |first: &Bytes, second: &Bytes| { + if first == second { + return true; + } + let first = match row_serde.deserialize(first) { + Ok(rows) => rows, + Err(e) => { + error!(err = %e, value = ?first, "fail to deserialize serialized value"); + return false; + } + }; + let second = match row_serde.deserialize(second) { + Ok(rows) => rows, + Err(e) => { + error!(err = %e, value = ?second, "fail to deserialize serialized value"); + return false; + } + }; + if first != second { + error!(first = ?first, second = ?second, "sanity check fail"); + false + } else { + true + } + })) +} + // initialize // FIXME(kwannoel): Enforce that none of the constructors here // should be used by replicated state table. @@ -294,14 +322,6 @@ where .collect() }; - let table_option = TableOption::build_table_option(table_catalog.get_properties()); - let new_local_options = if IS_REPLICATED { - NewLocalOptions::new_replicated(table_id, is_consistent_op, table_option) - } else { - NewLocalOptions::new(table_id, is_consistent_op, table_option) - }; - let local_state_store = store.new_local(new_local_options).await; - let pk_data_types = pk_indices .iter() .map(|i| table_columns[*i].data_type.clone()) @@ -334,10 +354,29 @@ where }; let prefix_hint_len = table_catalog.read_prefix_len_hint as usize; - let row_serde = SD::new( - Arc::from_iter(table_catalog.value_indices.iter().map(|val| *val as usize)), - Arc::from(table_columns.into_boxed_slice()), - ); + let make_row_serde = || { + SD::new( + Arc::from_iter(table_catalog.value_indices.iter().map(|val| *val as usize)), + Arc::from(table_columns.clone().into_boxed_slice()), + ) + }; + + let op_consistency_level = if is_consistent_op { + let row_serde = make_row_serde(); + consistent_old_value_op(row_serde) + } else { + OpConsistencyLevel::Inconsistent + }; + + let table_option = TableOption::build_table_option(table_catalog.get_properties()); + let new_local_options = if IS_REPLICATED { + NewLocalOptions::new_replicated(table_id, op_consistency_level, table_option) + } else { + NewLocalOptions::new(table_id, op_consistency_level, table_option) + }; + let local_state_store = store.new_local(new_local_options).await; + + let row_serde = make_row_serde(); // If state table has versioning, that means it supports // Schema change. In that case, the row encoding should be column aware as well. @@ -527,13 +566,31 @@ where value_indices: Option>, is_consistent_op: bool, ) -> Self { + let make_row_serde = || { + SD::new( + Arc::from( + value_indices + .clone() + .unwrap_or_else(|| (0..table_columns.len()).collect_vec()) + .into_boxed_slice(), + ), + Arc::from(table_columns.clone().into_boxed_slice()), + ) + }; + let op_consistency_level = if is_consistent_op { + let row_serde = make_row_serde(); + consistent_old_value_op(row_serde) + } else { + OpConsistencyLevel::Inconsistent + }; let local_state_store = store .new_local(NewLocalOptions::new( table_id, - is_consistent_op, + op_consistency_level, TableOption::default(), )) .await; + let row_serde = make_row_serde(); let data_types: Vec = table_columns .iter() .map(|col| col.data_type.clone()) @@ -553,15 +610,7 @@ where table_id, local_store: local_state_store, pk_serde, - row_serde: SD::new( - Arc::from( - value_indices - .clone() - .unwrap_or_else(|| (0..table_columns.len()).collect_vec()) - .into_boxed_slice(), - ), - Arc::from(table_columns.into_boxed_slice()), - ), + row_serde, pk_indices, dist_key_in_pk_indices, prefix_hint_len: 0, @@ -909,7 +958,7 @@ where /// Update a row without giving old value. /// - /// `is_consistent_op` should be set to false. + /// `op_consistency_level` should be set to `Inconsistent`. pub fn update_without_old_value(&mut self, new_value: impl Row) { let new_pk = (&new_value).project(self.pk_indices()); let new_key_bytes = diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index a3ec5c36a5eb1..2db6047f716b1 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -85,16 +85,7 @@ impl MaterializeExecutor { ) -> Self { let arrange_key_indices: Vec = arrange_key.iter().map(|k| k.column_index).collect(); - let state_table = if table_catalog.version.is_some() { - // TODO: If we do some `Delete` after schema change, we cannot ensure the encoded value - // with the new version of serializer is the same as the old one, even if they can be - // decoded into the same value. The table is now performing consistency check on the raw - // bytes, so we need to turn off the check here. We may turn it on if we can compare the - // decoded row. - StateTableInner::from_table_catalog_inconsistent_op(table_catalog, store, vnodes).await - } else { - StateTableInner::from_table_catalog(table_catalog, store, vnodes).await - }; + let state_table = StateTableInner::from_table_catalog(table_catalog, store, vnodes).await; let metrics_info = MetricsInfo::new(metrics, table_catalog.id, actor_context.id, "Materialize"); From f1f3f7b725cad0b18692a7f3ac50fa894d19750b Mon Sep 17 00:00:00 2001 From: Runji Wang Date: Tue, 26 Dec 2023 16:17:16 +0800 Subject: [PATCH 10/15] refactor(expr): move cast signatures to frontend (#14203) Signed-off-by: Runji Wang --- src/expr/core/src/sig/cast.rs | 143 ------------------- src/expr/core/src/sig/mod.rs | 2 - src/frontend/src/expr/type_inference/cast.rs | 86 ++++++++++- src/tests/sqlsmith/src/sql_gen/expr.rs | 2 +- src/tests/sqlsmith/src/sql_gen/types.rs | 3 +- 5 files changed, 87 insertions(+), 149 deletions(-) delete mode 100644 src/expr/core/src/sig/cast.rs diff --git a/src/expr/core/src/sig/cast.rs b/src/expr/core/src/sig/cast.rs deleted file mode 100644 index 73841a85e303e..0000000000000 --- a/src/expr/core/src/sig/cast.rs +++ /dev/null @@ -1,143 +0,0 @@ -// Copyright 2023 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::collections::BTreeMap; -use std::sync::LazyLock; - -use parse_display::Display; -use risingwave_common::types::DataTypeName; - -#[derive(Clone, Debug)] -pub struct CastSig { - pub from_type: DataTypeName, - pub to_type: DataTypeName, - pub context: CastContext, -} - -/// The context a cast operation is invoked in. An implicit cast operation is allowed in a context -/// that allows explicit casts, but not vice versa. See details in -/// [PG](https://www.postgresql.org/docs/current/catalog-pg-cast.html). -#[derive(Clone, Copy, Debug, Display, Eq, Ord, PartialEq, PartialOrd)] -pub enum CastContext { - #[display("i")] - Implicit, - #[display("a")] - Assign, - #[display("e")] - Explicit, -} - -pub type CastMap = BTreeMap<(DataTypeName, DataTypeName), CastContext>; - -pub fn cast_sigs() -> impl Iterator { - CAST_MAP - .iter() - .map(|((from_type, to_type), context)| CastSig { - from_type: *from_type, - to_type: *to_type, - context: *context, - }) -} - -pub static CAST_MAP: LazyLock = LazyLock::new(|| { - use DataTypeName as T; - - // Implicit cast operations in PG are organized in 3 sequences, with the reverse direction being - // assign cast operations. - // https://github.com/postgres/postgres/blob/e0064f0ff6dfada2695330c6bc1945fa7ae813be/src/include/catalog/pg_cast.dat#L18-L20 - let mut m = BTreeMap::new(); - insert_cast_seq( - &mut m, - &[ - T::Int16, - T::Int32, - T::Int64, - T::Decimal, - T::Float32, - T::Float64, - ], - ); - insert_cast_seq(&mut m, &[T::Date, T::Timestamp, T::Timestamptz]); - insert_cast_seq(&mut m, &[T::Time, T::Interval]); - - // Casting to and from string type. - for t in [ - T::Boolean, - T::Int16, - T::Int32, - T::Int64, - T::Int256, - T::Decimal, - T::Float32, - T::Float64, - T::Date, - T::Timestamp, - T::Timestamptz, - T::Time, - T::Interval, - T::Jsonb, - T::Bytea, - ] { - m.insert((t, T::Varchar), CastContext::Assign); - m.insert((T::Varchar, t), CastContext::Explicit); - } - - // Casting between `decimal`, `int256`, and `float` is not allowed. - m.insert((T::Int16, T::Int256), CastContext::Implicit); - m.insert((T::Int32, T::Int256), CastContext::Implicit); - m.insert((T::Int64, T::Int256), CastContext::Implicit); - - m.insert((T::Int256, T::Float64), CastContext::Explicit); - - // Misc casts allowed by PG that are neither in implicit cast sequences nor from/to string. - m.insert((T::Timestamp, T::Time), CastContext::Assign); - m.insert((T::Timestamptz, T::Time), CastContext::Assign); - m.insert((T::Boolean, T::Int32), CastContext::Explicit); - m.insert((T::Int32, T::Boolean), CastContext::Explicit); - - // Casting from jsonb to bool / number. - for t in [ - T::Boolean, - T::Int16, - T::Int32, - T::Int64, - T::Decimal, - T::Float32, - T::Float64, - ] { - m.insert((T::Jsonb, t), CastContext::Explicit); - } - - m -}); - -fn insert_cast_seq( - m: &mut BTreeMap<(DataTypeName, DataTypeName), CastContext>, - types: &[DataTypeName], -) { - for (source_index, source_type) in types.iter().enumerate() { - for (target_index, target_type) in types.iter().enumerate() { - let cast_context = match source_index.cmp(&target_index) { - std::cmp::Ordering::Less => CastContext::Implicit, - // Unnecessary cast between the same type should have been removed. - // Note that sizing cast between `NUMERIC(18, 3)` and `NUMERIC(20, 4)` or between - // `int` and `int not null` may still be necessary. But we do not have such types - // yet. - std::cmp::Ordering::Equal => continue, - std::cmp::Ordering::Greater => CastContext::Assign, - }; - m.insert((*source_type, *target_type), cast_context); - } - } -} diff --git a/src/expr/core/src/sig/mod.rs b/src/expr/core/src/sig/mod.rs index 738b4f6b9eaf9..32593607381c5 100644 --- a/src/expr/core/src/sig/mod.rs +++ b/src/expr/core/src/sig/mod.rs @@ -29,8 +29,6 @@ use crate::expr::BoxedExpression; use crate::table_function::BoxedTableFunction; use crate::ExprError; -pub mod cast; - /// The global registry of all function signatures. pub static FUNCTION_REGISTRY: LazyLock = LazyLock::new(|| unsafe { // SAFETY: this function is called after all `#[ctor]` functions are called. diff --git a/src/frontend/src/expr/type_inference/cast.rs b/src/frontend/src/expr/type_inference/cast.rs index b941732a2a720..ccd99048557ee 100644 --- a/src/frontend/src/expr/type_inference/cast.rs +++ b/src/frontend/src/expr/type_inference/cast.rs @@ -12,11 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::BTreeMap; +use std::sync::LazyLock; + use itertools::Itertools as _; +use parse_display::Display; use risingwave_common::error::ErrorCode; use risingwave_common::types::{DataType, DataTypeName}; use risingwave_common::util::iter_util::ZipEqFast; -pub use risingwave_expr::sig::cast::*; use crate::expr::{Expr as _, ExprImpl, InputRef, Literal}; @@ -165,6 +168,87 @@ pub fn cast_map_array() -> Vec<(DataTypeName, DataTypeName, CastContext)> { .collect_vec() } +#[derive(Clone, Debug)] +pub struct CastSig { + pub from_type: DataTypeName, + pub to_type: DataTypeName, + pub context: CastContext, +} + +/// The context a cast operation is invoked in. An implicit cast operation is allowed in a context +/// that allows explicit casts, but not vice versa. See details in +/// [PG](https://www.postgresql.org/docs/current/catalog-pg-cast.html). +#[derive(Clone, Copy, Debug, Display, Eq, Ord, PartialEq, PartialOrd)] +pub enum CastContext { + #[display("i")] + Implicit, + #[display("a")] + Assign, + #[display("e")] + Explicit, +} + +pub type CastMap = BTreeMap<(DataTypeName, DataTypeName), CastContext>; + +pub fn cast_sigs() -> impl Iterator { + CAST_MAP + .iter() + .map(|((from_type, to_type), context)| CastSig { + from_type: *from_type, + to_type: *to_type, + context: *context, + }) +} + +pub static CAST_MAP: LazyLock = LazyLock::new(|| { + // cast rules: + // 1. implicit cast operations in PG are organized in 3 sequences, + // with the reverse direction being assign cast operations. + // https://github.com/postgres/postgres/blob/e0064f0ff6dfada2695330c6bc1945fa7ae813be/src/include/catalog/pg_cast.dat#L18-L20 + // 1. int2 -> int4 -> int8 -> numeric -> float4 -> float8 + // 2. date -> timestamp -> timestamptz + // 3. time -> interval + // 2. any -> varchar is assign and varchar -> any is explicit + // 3. jsonb -> bool/number is explicit + // 4. int32 <-> bool is explicit + // 5. timestamp/timestamptz -> time is assign + // 6. int2/int4/int8 -> int256 is implicit and int256 -> float8 is explicit + use DataTypeName::*; + const CAST_TABLE: &[(&str, DataTypeName)] = &[ + // 123456789ABCDEF + (". e a", Boolean), // 0 + (" .iiiiii a", Int16), // 1 + ("ea.iiiii a", Int32), // 2 + (" aa.iiii a", Int64), // 3 + (" aaa.ii a", Decimal), // 4 + (" aaaa.i a", Float32), // 5 + (" aaaaa. a", Float64), // 6 + (" e. a", Int256), // 7 + (" .ii a", Date), // 8 + (" a.ia a", Timestamp), // 9 + (" aa.a a", Timestamptz), // A + (" .i a", Time), // B + (" a. a", Interval), // C + ("eeeeeee . a", Jsonb), // D + (" .a", Bytea), // E + ("eeeeeeeeeeeeeee.", Varchar), // F + ]; + let mut map = BTreeMap::new(); + for (row, source) in CAST_TABLE { + for ((_, target), c) in CAST_TABLE.iter().zip_eq_fast(row.bytes()) { + let context = match c { + b' ' | b'.' => continue, + b'i' => CastContext::Implicit, + b'a' => CastContext::Assign, + b'e' => CastContext::Explicit, + _ => unreachable!("invalid cast table char"), + }; + map.insert((*source, *target), context); + } + } + map +}); + #[cfg(test)] mod tests { use super::*; diff --git a/src/tests/sqlsmith/src/sql_gen/expr.rs b/src/tests/sqlsmith/src/sql_gen/expr.rs index 9999dcd9ea641..b1b895e492b63 100644 --- a/src/tests/sqlsmith/src/sql_gen/expr.rs +++ b/src/tests/sqlsmith/src/sql_gen/expr.rs @@ -16,8 +16,8 @@ use itertools::Itertools; use rand::seq::SliceRandom; use rand::Rng; use risingwave_common::types::{DataType, DataTypeName, StructType}; -use risingwave_expr::sig::cast::cast_sigs; use risingwave_expr::sig::FUNCTION_REGISTRY; +use risingwave_frontend::expr::cast_sigs; use risingwave_sqlparser::ast::{Expr, Ident, OrderByExpr, Value}; use crate::sql_gen::types::data_type_to_ast_data_type; diff --git a/src/tests/sqlsmith/src/sql_gen/types.rs b/src/tests/sqlsmith/src/sql_gen/types.rs index 5ef762f9951ad..ca462bb9f3601 100644 --- a/src/tests/sqlsmith/src/sql_gen/types.rs +++ b/src/tests/sqlsmith/src/sql_gen/types.rs @@ -20,9 +20,8 @@ use std::sync::LazyLock; use itertools::Itertools; use risingwave_common::types::{DataType, DataTypeName}; use risingwave_expr::aggregate::AggKind; -use risingwave_expr::sig::cast::{cast_sigs, CastContext, CastSig as RwCastSig}; use risingwave_expr::sig::{FuncSign, FUNCTION_REGISTRY}; -use risingwave_frontend::expr::ExprType; +use risingwave_frontend::expr::{cast_sigs, CastContext, CastSig as RwCastSig, ExprType}; use risingwave_sqlparser::ast::{BinaryOperator, DataType as AstDataType, StructField}; pub(super) fn data_type_to_ast_data_type(data_type: &DataType) -> AstDataType { From 070bd0cdc1314e4b3f36b626fca010b392db7614 Mon Sep 17 00:00:00 2001 From: Dylan Date: Tue, 26 Dec 2023 18:17:57 +0800 Subject: [PATCH 11/15] fix(meta): add index_columns_len to index catalog (#14210) Co-authored-by: August --- e2e_test/batch/catalog/pg_attribute.slt.part | 4 +--- e2e_test/batch/catalog/pg_index.slt.part | 6 +++--- proto/catalog.proto | 5 ++++- .../tests/testdata/output/subquery.yaml | 2 +- src/frontend/src/catalog/index_catalog.rs | 14 +++----------- .../system_catalog/pg_catalog/pg_index.rs | 4 ++-- .../system_catalog/rw_catalog/rw_indexes.rs | 17 +++++++++++------ src/frontend/src/handler/create_index.rs | 10 ++-------- .../migration/src/m20230908_072257_init.rs | 4 ++-- src/meta/model_v2/src/index.rs | 6 +++--- src/meta/src/controller/mod.rs | 2 +- 11 files changed, 33 insertions(+), 41 deletions(-) diff --git a/e2e_test/batch/catalog/pg_attribute.slt.part b/e2e_test/batch/catalog/pg_attribute.slt.part index 8bd43485c3ebe..7f96653af83a0 100644 --- a/e2e_test/batch/catalog/pg_attribute.slt.part +++ b/e2e_test/batch/catalog/pg_attribute.slt.part @@ -38,9 +38,7 @@ select i.relname, a.attname, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_attribute a on t.oid = a.attrelid and a.attnum = ANY(ix.indkey) where t.relname = 'tmp' order by a.attnum; ---- -tmp_idx id2 {2,3,4,5} -tmp_idx id3 {2,3,4,5} -tmp_idx id4 {2,3,4,5} +tmp_idx id2 {2} statement ok drop table tmp; diff --git a/e2e_test/batch/catalog/pg_index.slt.part b/e2e_test/batch/catalog/pg_index.slt.part index 3ebace06f207c..c42e74d60ab49 100644 --- a/e2e_test/batch/catalog/pg_index.slt.part +++ b/e2e_test/batch/catalog/pg_index.slt.part @@ -10,7 +10,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id2_idx'; ---- -2 {2,3} +1 {2} statement ok create index tmp_id2_idx_include_id1 on tmp(id2) include(id1); @@ -21,7 +21,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id2_idx_include_id1'; ---- -3 {2,3,4} +1 {2} statement ok create index tmp_id1_id2_idx on tmp(id1, id2); @@ -32,7 +32,7 @@ select ix.indnatts, ix.indkey from pg_catalog.pg_class t join pg_catalog.pg_class i on i.oid = ix.indexrelid where t.relname = 'tmp' and i.relname = 'tmp_id1_id2_idx'; ---- -3 {2,3,4} +2 {1,2} statement ok drop table tmp; diff --git a/proto/catalog.proto b/proto/catalog.proto index 33d56224976ea..4ebf9d0af6340 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -186,11 +186,14 @@ 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 int32 original_columns = 9; + reserved 9; // Deprecated repeated int32 original_columns = 9; optional uint64 initialized_at_epoch = 10; optional uint64 created_at_epoch = 11; StreamJobStatus stream_job_status = 12; + + // Use to record the prefix len of the index_item to reconstruct index columns provided by users. + uint32 index_columns_len = 13; } message Function { diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 7dd59eed432c6..ebdfed20bd32b 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -253,7 +253,7 @@ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } - │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.original_column_ids, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } + │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.indkey, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } │ │ │ │ └─LogicalSysScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.connector, rw_sinks.sink_type, rw_sinks.connection_id, rw_sinks.definition, rw_sinks.acl, rw_sinks.initialized_at, rw_sinks.created_at] } │ │ │ └─LogicalProject { exprs: [rw_materialized_views.id, rw_materialized_views.name, 'materialized view':Varchar, rw_materialized_views.schema_id, rw_materialized_views.owner, rw_materialized_views.definition, rw_materialized_views.acl] } diff --git a/src/frontend/src/catalog/index_catalog.rs b/src/frontend/src/catalog/index_catalog.rs index e5537b06717bd..4e66cea8e73ff 100644 --- a/src/frontend/src/catalog/index_catalog.rs +++ b/src/frontend/src/catalog/index_catalog.rs @@ -23,7 +23,6 @@ use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::{PbIndex, PbStreamJobStatus}; -use super::ColumnId; use crate::catalog::{DatabaseId, OwnedByUserCatalog, SchemaId, TableCatalog}; use crate::expr::{Expr, ExprDisplay, ExprImpl, FunctionCall}; use crate::user::UserId; @@ -58,7 +57,7 @@ pub struct IndexCatalog { #[educe(Hash(ignore))] pub function_mapping: HashMap, - pub original_columns: Vec, + pub index_columns_len: u32, pub created_at_epoch: Option, @@ -105,13 +104,6 @@ impl IndexCatalog { }) .collect(); - let original_columns = index_prost - .original_columns - .clone() - .into_iter() - .map(Into::into) - .collect(); - IndexCatalog { id: index_prost.id.into(), name: index_prost.name.clone(), @@ -121,7 +113,7 @@ impl IndexCatalog { primary_to_secondary_mapping, secondary_to_primary_mapping, function_mapping, - original_columns, + index_columns_len: index_prost.index_columns_len, created_at_epoch: index_prost.created_at_epoch.map(Epoch::from), initialized_at_epoch: index_prost.initialized_at_epoch.map(Epoch::from), } @@ -181,7 +173,7 @@ impl IndexCatalog { .iter() .map(|expr| expr.to_expr_proto()) .collect_vec(), - original_columns: self.original_columns.iter().map(Into::into).collect_vec(), + 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), stream_job_status: PbStreamJobStatus::Creating.into(), 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 c6ac7da458e2c..f98aacc680f6f 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 @@ -47,9 +47,9 @@ pub static PG_INDEX: LazyLock = LazyLock::new(|| BuiltinView { columns: &PG_INDEX_COLUMNS, sql: "SELECT id AS indexrelid, \ primary_table_id AS indrelid, \ - ARRAY_LENGTH(original_column_ids)::smallint AS indnatts, \ + ARRAY_LENGTH(indkey)::smallint AS indnatts, \ false AS indisunique, \ - original_column_ids AS indkey, \ + indkey, \ ARRAY[]::smallint[] as indoption, \ NULL AS indexprs, \ NULL AS indpred, \ diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs index ab9f7b3f2eb22..f8991b7a229c8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_indexes.rs @@ -28,10 +28,7 @@ pub static RW_INDEXES_COLUMNS: LazyLock>> = Lazy (DataType::Int32, "id"), (DataType::Varchar, "name"), (DataType::Int32, "primary_table_id"), - ( - DataType::List(Box::new(DataType::Int16)), - "original_column_ids", - ), + (DataType::List(Box::new(DataType::Int16)), "indkey"), (DataType::Int32, "schema_id"), (DataType::Int32, "owner"), (DataType::Varchar, "definition"), @@ -62,9 +59,17 @@ impl SysCatalogReaderImpl { Some(ScalarImpl::Int32(index.primary_table.id().table_id as i32)), Some(ScalarImpl::List(ListValue::from_iter( index - .original_columns + .index_item .iter() - .map(|index| index.get_id() as i16 + 1), + .take(index.index_columns_len as usize) + .map(|index| { + let ind = if let Some(input_ref) = index.as_input_ref() { + input_ref.index() + 1 + } else { + 0 + }; + ind as i16 + }), ))), Some(ScalarImpl::Int32(schema.id() as i32)), Some(ScalarImpl::Int32(index.index_table.owner as i32)), diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index 1553a84c1a09b..f2fa8bca2de4e 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -219,13 +219,7 @@ pub(crate) fn gen_create_index_plan( index_table_prost.owner = session.user_id(); index_table_prost.dependent_relations = vec![table.id.table_id]; - // FIXME: why sqlalchemy need these information? - let original_columns = index_table - .columns - .iter() - .map(|x| x.column_desc.column_id.get_id()) - .collect(); - + let index_columns_len = index_columns_ordered_expr.len() as u32; let index_item = build_index_item( index_table.table_desc().into(), table.name(), @@ -242,7 +236,7 @@ pub(crate) fn gen_create_index_plan( index_table_id: TableId::placeholder().table_id, primary_table_id: table.id.table_id, index_item, - original_columns, + index_columns_len, initialized_at_epoch: None, created_at_epoch: None, stream_job_status: PbStreamJobStatus::Creating.into(), diff --git a/src/meta/model_v2/migration/src/m20230908_072257_init.rs b/src/meta/model_v2/migration/src/m20230908_072257_init.rs index ee853876bc0f9..fbee1aba46b94 100644 --- a/src/meta/model_v2/migration/src/m20230908_072257_init.rs +++ b/src/meta/model_v2/migration/src/m20230908_072257_init.rs @@ -669,7 +669,7 @@ impl MigrationTrait for Migration { .col(ColumnDef::new(Index::IndexTableId).integer().not_null()) .col(ColumnDef::new(Index::PrimaryTableId).integer().not_null()) .col(ColumnDef::new(Index::IndexItems).json().not_null()) - .col(ColumnDef::new(Index::OriginalColumns).json().not_null()) + .col(ColumnDef::new(Index::IndexColumnsLen).integer().not_null()) .foreign_key( &mut ForeignKey::create() .name("FK_index_object_id") @@ -1084,7 +1084,7 @@ enum Index { IndexTableId, PrimaryTableId, IndexItems, - OriginalColumns, + IndexColumnsLen, } #[derive(DeriveIden)] diff --git a/src/meta/model_v2/src/index.rs b/src/meta/model_v2/src/index.rs index 8b10c58e04674..0146fe34f4050 100644 --- a/src/meta/model_v2/src/index.rs +++ b/src/meta/model_v2/src/index.rs @@ -16,7 +16,7 @@ use risingwave_pb::catalog::PbIndex; use sea_orm::entity::prelude::*; use sea_orm::ActiveValue::Set; -use crate::{ExprNodeArray, I32Array, IndexId, TableId}; +use crate::{ExprNodeArray, IndexId, TableId}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq)] #[sea_orm(table_name = "index")] @@ -27,7 +27,7 @@ pub struct Model { pub index_table_id: TableId, pub primary_table_id: TableId, pub index_items: ExprNodeArray, - pub original_columns: I32Array, + pub index_columns_len: i32, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -74,7 +74,7 @@ impl From for ActiveModel { index_table_id: Set(pb_index.index_table_id as _), primary_table_id: Set(pb_index.primary_table_id as _), index_items: Set(pb_index.index_item.into()), - original_columns: Set(pb_index.original_columns.into()), + index_columns_len: Set(pb_index.index_columns_len as _), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 48c6b6d43172b..137ed6af69c76 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -217,7 +217,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.0, - original_columns: value.0.original_columns.0, + 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, ), From 5a312a206def916902ec901e106da0293db3dc30 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Tue, 26 Dec 2023 19:22:43 +0800 Subject: [PATCH 12/15] doc(delta-lake): add demo for delta lake sink (#11087) Co-authored-by: Xinhao Xu <84456268+xxhZs@users.noreply.github.com> Co-authored-by: xxhZs <1060434431@qq.com> --- ci/scripts/gen-integration-test-yaml.py | 1 + ci/scripts/notify.py | 1 + integration_tests/deltalake-sink/README.md | 25 +++++++++++ .../deltalake-sink/create_sink.sql | 10 +++++ .../deltalake-sink/create_source.sql | 3 ++ .../deltalake-sink/docker-compose.yml | 43 +++++++++++++++++++ integration_tests/deltalake-sink/prepare.sh | 7 +++ .../deltalake-sink/sink_check.py | 30 +++++++++++++ .../deltalake-sink/spark-script/.gitignore | 3 ++ .../spark-script/create-table.sql | 1 + .../spark-script/query-table.sql | 1 + .../spark-script/run-sql-file.sh | 11 +++++ 12 files changed, 136 insertions(+) create mode 100644 integration_tests/deltalake-sink/README.md create mode 100644 integration_tests/deltalake-sink/create_sink.sql create mode 100644 integration_tests/deltalake-sink/create_source.sql create mode 100644 integration_tests/deltalake-sink/docker-compose.yml create mode 100644 integration_tests/deltalake-sink/prepare.sh create mode 100644 integration_tests/deltalake-sink/sink_check.py create mode 100644 integration_tests/deltalake-sink/spark-script/.gitignore create mode 100644 integration_tests/deltalake-sink/spark-script/create-table.sql create mode 100644 integration_tests/deltalake-sink/spark-script/query-table.sql create mode 100644 integration_tests/deltalake-sink/spark-script/run-sql-file.sh diff --git a/ci/scripts/gen-integration-test-yaml.py b/ci/scripts/gen-integration-test-yaml.py index 9ca065f856e89..13022a06d5ebe 100644 --- a/ci/scripts/gen-integration-test-yaml.py +++ b/ci/scripts/gen-integration-test-yaml.py @@ -35,6 +35,7 @@ 'nats': ['json'], 'doris-sink': ['json'], 'starrocks-sink': ['json'], + 'deltalake-sink': ['json'], } def gen_pipeline_steps(): diff --git a/ci/scripts/notify.py b/ci/scripts/notify.py index 1e8e8580e9082..2f68733b26022 100755 --- a/ci/scripts/notify.py +++ b/ci/scripts/notify.py @@ -58,6 +58,7 @@ "vector-json": ["tao"], "doris-sink": ["xinhao"], "starrocks-sink": ["xinhao"], + "deltalake-sink": ["xinhao"], } def get_failed_tests(get_test_status, test_map): diff --git a/integration_tests/deltalake-sink/README.md b/integration_tests/deltalake-sink/README.md new file mode 100644 index 0000000000000..0dd34cab2ff70 --- /dev/null +++ b/integration_tests/deltalake-sink/README.md @@ -0,0 +1,25 @@ +# Demo: Sinking to Delta Lake + +In this demo, we will create an append-only source via our datagen source, +and sink the data generated from source to the downstream delta lake table +stored on minio. + +1. Launch the cluster via docker compose +``` +docker compose up -d +``` + +2. Create a delta lake table on minio +``` +docker compose exec minio-0 mkdir /data/deltalake +docker compose exec spark bash /spark-script/run-sql-file.sh create-table +``` + +3. Execute the SQL queries in sequence: + - create_source.sql + - create_sink.sql + +4. Query delta lake table. The following command will query the total count of records. +``` +docker compose exec spark bash /spark-script/run-sql-file.sh query-table +``` \ No newline at end of file diff --git a/integration_tests/deltalake-sink/create_sink.sql b/integration_tests/deltalake-sink/create_sink.sql new file mode 100644 index 0000000000000..c7dab6ef5dd9e --- /dev/null +++ b/integration_tests/deltalake-sink/create_sink.sql @@ -0,0 +1,10 @@ +create sink delta_lake_sink from source +with ( + connector = 'deltalake', + type = 'append-only', + force_append_only='true', + location = 's3a://deltalake/delta', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin', + s3.endpoint = 'http://minio-0:9301' +); \ No newline at end of file diff --git a/integration_tests/deltalake-sink/create_source.sql b/integration_tests/deltalake-sink/create_source.sql new file mode 100644 index 0000000000000..d78b460512b8b --- /dev/null +++ b/integration_tests/deltalake-sink/create_source.sql @@ -0,0 +1,3 @@ +CREATE table source (id int, name varchar); + +INSERT into source values (1, 'a'), (2, 'b'), (3, 'c'); \ No newline at end of file diff --git a/integration_tests/deltalake-sink/docker-compose.yml b/integration_tests/deltalake-sink/docker-compose.yml new file mode 100644 index 0000000000000..8e9a20533f25e --- /dev/null +++ b/integration_tests/deltalake-sink/docker-compose.yml @@ -0,0 +1,43 @@ +--- +version: "3" +services: + spark: + image: apache/spark:3.3.1 + command: tail -f /dev/null + depends_on: + - minio-0 + volumes: + - "./spark-script:/spark-script" + container_name: spark + risingwave-standalone: + extends: + file: ../../docker/docker-compose.yml + service: risingwave-standalone + etcd-0: + extends: + file: ../../docker/docker-compose.yml + service: etcd-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: + compute-node-0: + external: false + etcd-0: + external: false + grafana-0: + external: false + minio-0: + external: false + prometheus-0: + external: false +name: risingwave-compose \ No newline at end of file diff --git a/integration_tests/deltalake-sink/prepare.sh b/integration_tests/deltalake-sink/prepare.sh new file mode 100644 index 0000000000000..419edbb9b794f --- /dev/null +++ b/integration_tests/deltalake-sink/prepare.sh @@ -0,0 +1,7 @@ +#!/bin/bash + +set -euo pipefail + +# build minio dir and create table +docker compose exec minio-0 mkdir /data/deltalake +docker compose exec spark bash /spark-script/run-sql-file.sh create-table \ No newline at end of file diff --git a/integration_tests/deltalake-sink/sink_check.py b/integration_tests/deltalake-sink/sink_check.py new file mode 100644 index 0000000000000..b3796d2ca56ee --- /dev/null +++ b/integration_tests/deltalake-sink/sink_check.py @@ -0,0 +1,30 @@ +import subprocess +from time import sleep + +sleep(60) + +query_sql = open("spark-script/query-table.sql").read() + +print("querying deltalake with sql: %s" % query_sql) + +query_output_file_name = "query_output.txt" + +query_output_file = open(query_output_file_name, "wb") + +subprocess.run( + ["docker", "compose", "exec", "spark", "bash", "/spark-script/run-sql-file.sh", "query-table"], + check=True, stdout=query_output_file) +query_output_file.close() + +with open(query_output_file_name, 'r') as file: + all_lines = file.readlines() + +last_three_lines = all_lines[-3:] + +print("result", last_three_lines) + +line1, line2, line3 = last_three_lines + +assert line1.strip() == '1\ta' +assert line2.strip() == '2\tb' +assert line3.strip() == '3\tc' \ No newline at end of file diff --git a/integration_tests/deltalake-sink/spark-script/.gitignore b/integration_tests/deltalake-sink/spark-script/.gitignore new file mode 100644 index 0000000000000..2af1a65665298 --- /dev/null +++ b/integration_tests/deltalake-sink/spark-script/.gitignore @@ -0,0 +1,3 @@ +derby.log +metastore_db +.ivy2 \ No newline at end of file diff --git a/integration_tests/deltalake-sink/spark-script/create-table.sql b/integration_tests/deltalake-sink/spark-script/create-table.sql new file mode 100644 index 0000000000000..edd31d08f3ebc --- /dev/null +++ b/integration_tests/deltalake-sink/spark-script/create-table.sql @@ -0,0 +1 @@ +create table delta.`s3a://deltalake/delta`(id int, name string) using delta \ No newline at end of file diff --git a/integration_tests/deltalake-sink/spark-script/query-table.sql b/integration_tests/deltalake-sink/spark-script/query-table.sql new file mode 100644 index 0000000000000..bdd1dea729836 --- /dev/null +++ b/integration_tests/deltalake-sink/spark-script/query-table.sql @@ -0,0 +1 @@ +SELECT * from delta.`s3a://deltalake/delta` order by id; \ No newline at end of file diff --git a/integration_tests/deltalake-sink/spark-script/run-sql-file.sh b/integration_tests/deltalake-sink/spark-script/run-sql-file.sh new file mode 100644 index 0000000000000..58132bcfafa0a --- /dev/null +++ b/integration_tests/deltalake-sink/spark-script/run-sql-file.sh @@ -0,0 +1,11 @@ +set -ex + +/opt/spark/bin/spark-sql --packages io.delta:delta-core_2.12:2.2.0,org.apache.hadoop:hadoop-aws:3.3.2\ + --conf "spark.driver.extraJavaOptions=-Divy.cache.dir=/tmp -Divy.home=/tmp" \ + --conf 'spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension' \ + --conf 'spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog' \ + --conf 'spark.hadoop.fs.s3a.access.key=hummockadmin' \ + --conf 'spark.hadoop.fs.s3a.secret.key=hummockadmin' \ + --conf 'spark.hadoop.fs.s3a.endpoint=http://minio-0:9301' \ + --conf 'spark.hadoop.fs.s3a.path.style.access=true' \ + -f /spark-script/$1.sql \ No newline at end of file From a8d0541b08b29822c342f3e096b376376e71a305 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Dec 2023 22:24:29 +0800 Subject: [PATCH 13/15] fix(connector): use log4j's string interpolation (#14206) Signed-off-by: Bugen Zhao --- java/connector-node/tracing/pom.xml | 4 + .../tracing/TracingSlf4jAdapter.java | 156 ++++++++++++++---- src/utils/runtime/src/logger.rs | 1 + 3 files changed, 130 insertions(+), 31 deletions(-) diff --git a/java/connector-node/tracing/pom.xml b/java/connector-node/tracing/pom.xml index 29ed2a08fa50a..01ba0b6bbe181 100644 --- a/java/connector-node/tracing/pom.xml +++ b/java/connector-node/tracing/pom.xml @@ -27,6 +27,10 @@ org.slf4j slf4j-api + + org.apache.logging.log4j + log4j-core + diff --git a/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jAdapter.java b/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jAdapter.java index 25527a6c7e695..a3a1e752cd3ee 100644 --- a/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jAdapter.java +++ b/java/connector-node/tracing/src/main/java/com/risingwave/tracing/TracingSlf4jAdapter.java @@ -17,6 +17,9 @@ package com.risingwave.tracing; +// Import log4j's ParameterizedMessage, so that we can format the messages +// with the same interpolation as log4j (i.e. "{}" instead of "%s"). +import org.apache.logging.log4j.message.ParameterizedMessage; import org.slf4j.Logger; import org.slf4j.Marker; @@ -45,17 +48,27 @@ public void trace(String msg) { @Override public void trace(String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void trace(String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, arg1, arg2)); + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage(); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void trace(String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -76,17 +89,26 @@ public void trace(Marker marker, String msg) { @Override public void trace(Marker marker, String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void trace(Marker marker, String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override - public void trace(Marker marker, String format, Object... argArray) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.TRACE, String.format(format, argArray)); + public void trace(Marker marker, String format, Object... arguments) { + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.TRACE, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -107,17 +129,26 @@ public void debug(String msg) { @Override public void debug(String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void debug(String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void debug(String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -138,17 +169,26 @@ public void debug(Marker marker, String msg) { @Override public void debug(Marker marker, String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void debug(Marker marker, String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void debug(Marker marker, String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.DEBUG, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.DEBUG, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -169,17 +209,26 @@ public void info(String msg) { @Override public void info(String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void info(String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void info(String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -200,17 +249,26 @@ public void info(Marker marker, String msg) { @Override public void info(Marker marker, String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void info(Marker marker, String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void info(Marker marker, String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.INFO, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.INFO, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -231,17 +289,26 @@ public void warn(String msg) { @Override public void warn(String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void warn(String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override public void warn(String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override @@ -262,17 +329,26 @@ public void warn(Marker marker, String msg) { @Override public void warn(Marker marker, String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void warn(Marker marker, String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void warn(Marker marker, String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.WARN, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.WARN, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -293,17 +369,26 @@ public void error(String msg) { @Override public void error(String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void error(String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void error(String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override @@ -324,17 +409,26 @@ public void error(Marker marker, String msg) { @Override public void error(Marker marker, String format, Object arg) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arg)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arg).getFormattedMessage()); } @Override public void error(Marker marker, String format, Object arg1, Object arg2) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arg1, arg2)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arg1, arg2).getFormattedMessage()); } @Override public void error(Marker marker, String format, Object... arguments) { - TracingSlf4jImpl.event(name, TracingSlf4jImpl.ERROR, String.format(format, arguments)); + TracingSlf4jImpl.event( + name, + TracingSlf4jImpl.ERROR, + new ParameterizedMessage(format, arguments).getFormattedMessage()); } @Override diff --git a/src/utils/runtime/src/logger.rs b/src/utils/runtime/src/logger.rs index e8abb0aef65eb..8dde42ce783f0 100644 --- a/src/utils/runtime/src/logger.rs +++ b/src/utils/runtime/src/logger.rs @@ -158,6 +158,7 @@ pub fn init_risingwave_logger(settings: LoggerSettings) { // Other RisingWave crates like `stream` and `storage` will follow the default level. filter = filter .with_target("risingwave_sqlparser", Level::INFO) + .with_target("risingwave_connector_node", Level::INFO) .with_target("pgwire", Level::INFO) .with_target(PGWIRE_QUERY_LOG, Level::OFF) // debug-purposed events are disabled unless `RUST_LOG` overrides From 9e4c68e4c8c781885fac9ca9f3fb1724e98b9d97 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 26 Dec 2023 23:19:35 +0800 Subject: [PATCH 14/15] fix(cdc): bump debezium dependence to fix postgres cdc data loss (#14151) --- .../source/common/DbzSourceUtils.java | 2 +- .../connector/source/core/DbzCdcEngine.java | 5 +- .../source/core/DbzCdcEngineRunner.java | 4 +- .../src/main/resources/debezium.properties | 6 +- .../src/main/resources/mysql.properties | 3 - .../src/main/resources/postgres.properties | 8 +- .../risingwave-connector-test/pom.xml | 16 +-- .../src/test/resources/log4j2.properties | 18 ++- .../ConfigurableOffsetBackingStore.java | 20 +++- .../risingwave-source-test/pom.xml | 108 ++++++++++++++++++ .../connector/source/MySQLSourceTest.java | 0 .../connector/source/PostgresSourceTest.java | 0 .../connector/source/SourceTestClient.java | 2 +- .../src/test/resources/log4j2.properties | 14 +++ .../src/test/resources/my.cnf | 0 .../src/test/resources/orders.tbl | 0 .../test/resources/stored_queries.properties | 0 java/pom.xml | 28 ++++- 18 files changed, 185 insertions(+), 49 deletions(-) create mode 100644 java/connector-node/risingwave-source-test/pom.xml rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/java/com/risingwave/connector/source/MySQLSourceTest.java (100%) rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/java/com/risingwave/connector/source/PostgresSourceTest.java (100%) rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/java/com/risingwave/connector/source/SourceTestClient.java (99%) create mode 100644 java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/resources/my.cnf (100%) rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/resources/orders.tbl (100%) rename java/connector-node/{risingwave-connector-test => risingwave-source-test}/src/test/resources/stored_queries.properties (100%) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java index d5e09b03825c9..ce9c746d9b9bd 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzSourceUtils.java @@ -41,7 +41,7 @@ public static boolean waitForStreamingRunning(SourceTypeE sourceType, String dbS } private static boolean waitForStreamingRunningInner(String connector, String dbServerName) { - int maxPollCount = 10; + int maxPollCount = 11; // max poll 10 seconds while (!isStreamingRunning(connector, dbServerName, "streaming")) { maxPollCount--; if (maxPollCount == 0) { diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java index 4f4a13f3027a8..65f961dbf523a 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngine.java @@ -14,11 +14,12 @@ package com.risingwave.connector.source.core; +import static io.debezium.schema.AbstractTopicNamingStrategy.TOPIC_HEARTBEAT_PREFIX; + import com.risingwave.connector.api.source.CdcEngine; import com.risingwave.proto.ConnectorServiceProto; import io.debezium.embedded.Connect; import io.debezium.engine.DebeziumEngine; -import io.debezium.heartbeat.Heartbeat; import java.util.Properties; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; @@ -35,7 +36,7 @@ public DbzCdcEngine( long sourceId, Properties config, DebeziumEngine.CompletionCallback completionCallback) { - var dbzHeartbeatPrefix = config.getProperty(Heartbeat.HEARTBEAT_TOPICS_PREFIX.name()); + var dbzHeartbeatPrefix = config.getProperty(TOPIC_HEARTBEAT_PREFIX.name()); var consumer = new DbzCdcEventConsumer( sourceId, diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java index caf8ed4b87285..d4728c333b9f0 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java @@ -18,6 +18,7 @@ import com.risingwave.connector.source.common.DbzConnectorConfig; import com.risingwave.connector.source.common.DbzSourceUtils; import com.risingwave.proto.ConnectorServiceProto.GetEventStreamResponse; +import io.debezium.config.CommonConnectorConfig; import io.grpc.stub.StreamObserver; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -113,7 +114,8 @@ public boolean start() throws InterruptedException { // For backfill source, we need to wait for the streaming source to start before proceeding if (config.isBackfillSource()) { var databaseServerName = - config.getResolvedDebeziumProps().getProperty("database.server.name"); + config.getResolvedDebeziumProps() + .getProperty(CommonConnectorConfig.TOPIC_PREFIX.name()); startOk = DbzSourceUtils.waitForStreamingRunning( config.getSourceType(), databaseServerName); diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties b/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties index d98e5b710ecb2..3337aebf92c6c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/debezium.properties @@ -1,5 +1,8 @@ # Store common debezium configs shared by all connectors -database.server.name=RW_CDC_${source.id} +topic.prefix=RW_CDC_${source.id} +topic.heartbeat.prefix=${debezium.topic.heartbeat.prefix:-RW_CDC_HeartBeat_} +schema.history.internal=io.debezium.relational.history.MemorySchemaHistory +offset.storage=com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore converters=datetime datetime.type=com.risingwave.connector.cdc.debezium.converters.DatetimeTypeConverter # use string to preserve the precision of decimal, since currently we cannot @@ -8,5 +11,4 @@ decimal.handling.mode=${debezium.decimal.handling.mode:-string} interval.handling.mode=string max.batch.size=${debezium.max.batch.size:-1024} max.queue.size=${debezium.max.queue.size:-8192} - time.precision.mode=adaptive_time_microseconds diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties index 04ee9a2dad821..9c8f4f6e0d306 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties @@ -1,7 +1,5 @@ # configs for mysql connector connector.class=io.debezium.connector.mysql.MySqlConnector -offset.storage=com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore -database.history=io.debezium.relational.history.MemoryDatabaseHistory # default snapshot mode to initial snapshot.mode=${debezium.snapshot.mode:-initial} database.hostname=${hostname} @@ -17,7 +15,6 @@ database.server.id=${server.id} database.connectionTimeZone=+00:00 # default heartbeat interval 60 seconds heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-60000} -heartbeat.topics.prefix=${debezium.heartbeat.topics.prefix:-RW_CDC_HeartBeat_} # In sharing cdc mode, we will subscribe to multiple tables in the given database, # so here we set ${table.name} to a default value `RW_CDC_Sharing` just for display. name=${hostname}:${port}:${database.name}.${table.name:-RW_CDC_Sharing} diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties b/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties index 62ef2e0e713d7..4d71480dc3536 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/postgres.properties @@ -1,7 +1,5 @@ # configs for postgres conneoctor connector.class=io.debezium.connector.postgresql.PostgresConnector -offset.storage=com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore -database.history=io.debezium.relational.history.MemoryDatabaseHistory # default snapshot mode to initial snapshot.mode=${debezium.snapshot.mode:-initial} database.hostname=${hostname} @@ -17,9 +15,7 @@ plugin.name=${debezium.plugin.name:-pgoutput} # allow to auto create publication for given tables publication.autocreate.mode=${debezium.publication.autocreate.mode:-filtered} publication.name=${publication.name:-rw_publication} -# default heartbeat interval 5 mins -heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} -heartbeat.topics.prefix=${debezium.heartbeat.topics.prefix:-RW_CDC_HeartBeat_} +# default heartbeat interval 60 seconds +heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-60000} name=${hostname}:${port}:${database.name}.${schema.name}.${table.name} - provide.transaction.metadata=${transactional:-false} diff --git a/java/connector-node/risingwave-connector-test/pom.xml b/java/connector-node/risingwave-connector-test/pom.xml index bd9195e099f41..14b1c7bd65fc0 100644 --- a/java/connector-node/risingwave-connector-test/pom.xml +++ b/java/connector-node/risingwave-connector-test/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> risingwave-java-root com.risingwave @@ -14,7 +14,6 @@ risingwave-connector-test - 1.17.6 1.0.0 @@ -111,20 +110,14 @@ org.testcontainers testcontainers - ${testcontainers.version} - test org.testcontainers mysql - ${testcontainers.version} - test org.testcontainers postgresql - ${testcontainers.version} - test org.testcontainers @@ -145,11 +138,6 @@ test - - com.risingwave - risingwave-source-cdc - test - com.risingwave risingwave-connector-service diff --git a/java/connector-node/risingwave-connector-test/src/test/resources/log4j2.properties b/java/connector-node/risingwave-connector-test/src/test/resources/log4j2.properties index 12c71029bcab6..a380d72fc3c2a 100644 --- a/java/connector-node/risingwave-connector-test/src/test/resources/log4j2.properties +++ b/java/connector-node/risingwave-connector-test/src/test/resources/log4j2.properties @@ -1,12 +1,10 @@ -rootLogger.level = ERROR +rootLogger.level=ERROR # declare the appender to use -appenders = console - +appenders=console # appender properties -appender.console.type = Console -appender.console.name = stdout -appender.console.layout.type = PatternLayout -appender.console.layout.pattern = %d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%t] %c{2}:%L - %m%n - -rootLogger.appenderRefs = console -rootLogger.appenderRef.console.ref = stdout +appender.console.type=Console +appender.console.name=stdout +appender.console.layout.type=PatternLayout +appender.console.layout.pattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%t] %c{2}:%L - %m%n +rootLogger.appenderRefs=console +rootLogger.appenderRef.console.ref=stdout diff --git a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java index 0ceb4ba16e11e..cb7c71ec08546 100644 --- a/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java +++ b/java/connector-node/risingwave-source-cdc/src/main/java/com/risingwave/connector/cdc/debezium/internal/ConfigurableOffsetBackingStore.java @@ -18,18 +18,22 @@ package com.risingwave.connector.cdc.debezium.internal; +import io.debezium.config.Instantiator; import io.debezium.embedded.EmbeddedEngine; import io.debezium.engine.DebeziumEngine; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.*; import org.apache.kafka.common.utils.ThreadUtils; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.storage.Converter; import org.apache.kafka.connect.storage.OffsetBackingStore; @@ -81,12 +85,13 @@ public void configure(WorkerConfig config) { } String engineName = (String) conf.get(EmbeddedEngine.ENGINE_NAME.name()); - Converter keyConverter = new JsonConverter(); + Map converterConfig = + Collections.singletonMap(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, "false"); + + Converter keyConverter = Instantiator.getInstance(JsonConverter.class.getName()); + keyConverter.configure(converterConfig, true); Converter valueConverter = new JsonConverter(); - keyConverter.configure(config.originals(), true); - Map valueConfigs = new HashMap<>(conf); - valueConfigs.put("schemas.enable", false); - valueConverter.configure(valueConfigs, true); + valueConverter.configure(converterConfig, true); OffsetStorageWriter offsetWriter = new OffsetStorageWriter( this, @@ -195,4 +200,9 @@ public Future set( return null; }); } + + @Override + public Set> connectorPartitions(String connectorName) { + return null; + } } diff --git a/java/connector-node/risingwave-source-test/pom.xml b/java/connector-node/risingwave-source-test/pom.xml new file mode 100644 index 0000000000000..46956b642f15e --- /dev/null +++ b/java/connector-node/risingwave-source-test/pom.xml @@ -0,0 +1,108 @@ + + + + risingwave-java-root + com.risingwave + 0.1.0-SNAPSHOT + ../../pom.xml + + 4.0.0 + risingwave-source-test + jar + risingwave-source-test + + + + com.risingwave + connector-api + test + + + com.risingwave + s3-common + test + + + org.slf4j + slf4j-log4j12 + + + org.slf4j + slf4j-reload4j + + + + + io.grpc + grpc-protobuf + + + org.apache.logging.log4j + log4j-api + + + org.apache.logging.log4j + log4j-core + + + junit + junit + test + + + org.assertj + assertj-core + 3.24.2 + test + + + com.zaxxer + HikariCP + 5.0.1 + test + + + org.testcontainers + testcontainers + + + org.testcontainers + mysql + + + org.testcontainers + postgresql + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + test + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + test + + + + com.risingwave + risingwave-source-cdc + test + + + com.risingwave + risingwave-connector-service + test + + + com.risingwave + tracing + + + + + diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/MySQLSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MySQLSourceTest.java similarity index 100% rename from java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/MySQLSourceTest.java rename to java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MySQLSourceTest.java diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/PostgresSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/PostgresSourceTest.java similarity index 100% rename from java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/PostgresSourceTest.java rename to java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/PostgresSourceTest.java diff --git a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java similarity index 99% rename from java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java rename to java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java index 359746bc90f77..e0e6613df662d 100644 --- a/java/connector-node/risingwave-connector-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java @@ -130,7 +130,7 @@ protected Iterator getEventStreamS String port = String.valueOf(URI.create(container.getJdbcUrl().substring(5)).getPort()); ConnectorServiceProto.GetEventStreamRequest req = ConnectorServiceProto.GetEventStreamRequest.newBuilder() - .setSourceId(0) + .setSourceId(1005) .setSourceType(sourceType) .setStartOffset("") .putProperties("hostname", container.getHost()) diff --git a/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties b/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties new file mode 100644 index 0000000000000..d46bd5f609266 --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties @@ -0,0 +1,14 @@ +rootLogger.level=ERROR +# declare the appender to use +appenders=console +# appender properties +appender.console.type=Console +appender.console.name=stdout +appender.console.layout.type=PatternLayout +appender.console.layout.pattern=%d{yyyy-MM-dd HH:mm:ss,SSS} %-5p [%t] %c{2}:%L - %m%n +rootLogger.appenderRefs=console +rootLogger.appenderRef.console.ref=stdout +logger.connector.name=com.risingwave.connector.source +logger.connector.level=INFO +logger.connector.appenderRefs=console +logger.connector.appenderRef.console.ref=stdout diff --git a/java/connector-node/risingwave-connector-test/src/test/resources/my.cnf b/java/connector-node/risingwave-source-test/src/test/resources/my.cnf similarity index 100% rename from java/connector-node/risingwave-connector-test/src/test/resources/my.cnf rename to java/connector-node/risingwave-source-test/src/test/resources/my.cnf diff --git a/java/connector-node/risingwave-connector-test/src/test/resources/orders.tbl b/java/connector-node/risingwave-source-test/src/test/resources/orders.tbl similarity index 100% rename from java/connector-node/risingwave-connector-test/src/test/resources/orders.tbl rename to java/connector-node/risingwave-source-test/src/test/resources/orders.tbl diff --git a/java/connector-node/risingwave-connector-test/src/test/resources/stored_queries.properties b/java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties similarity index 100% rename from java/connector-node/risingwave-connector-test/src/test/resources/stored_queries.properties rename to java/connector-node/risingwave-source-test/src/test/resources/stored_queries.properties diff --git a/java/pom.xml b/java/pom.xml index fca52806e878e..7e7e554abc0d4 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -1,7 +1,7 @@ + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"> 4.0.0 com.risingwave @@ -52,6 +52,7 @@ connector-node/risingwave-source-cdc connector-node/risingwave-connector-test connector-node/risingwave-connector-service + connector-node/risingwave-source-test connector-node/assembly connector-node/s3-common connector-node/risingwave-sink-mock-flink @@ -74,13 +75,14 @@ 1.5.0 2.11.0 1.10.0 - 1.9.7.Final + 2.4.2.Final 2.13.5 3.3.1 3.3.3 7.17.14 4.15.0 1.18.0 + 1.17.6 @@ -312,6 +314,24 @@ ${spark_sql.version} test + + org.testcontainers + testcontainers + ${testcontainers.version} + test + + + org.testcontainers + mysql + ${testcontainers.version} + test + + + org.testcontainers + postgresql + ${testcontainers.version} + test + @@ -359,7 +379,7 @@ - + From fcaa3184e052f0094a86ec74acd8333e496a0381 Mon Sep 17 00:00:00 2001 From: Bohan Zhang Date: Wed, 27 Dec 2023 11:09:54 +0800 Subject: [PATCH 15/15] fix: make json/csv right (#14222) Signed-off-by: tabVersion --- src/connector/src/source/filesystem/nd_streaming.rs | 6 ++++++ .../src/source/filesystem/opendal_source/opendal_reader.rs | 6 ++---- src/connector/src/source/filesystem/s3/source/reader.rs | 7 +++---- 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs index 428514d2d6adf..bce96fc0976d7 100644 --- a/src/connector/src/source/filesystem/nd_streaming.rs +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -18,8 +18,14 @@ use futures::io::Cursor; use futures::AsyncBufReadExt; use futures_async_stream::try_stream; +use crate::parser::EncodingProperties; use crate::source::{BoxSourceStream, SourceMessage}; +pub fn need_nd_streaming(encode_config: &EncodingProperties) -> bool { + matches!(encode_config, &EncodingProperties::Json(_)) + || matches!(encode_config, EncodingProperties::Csv(_)) +} + #[try_stream(boxed, ok = Vec, error = anyhow::Error)] /// This function splits a byte stream by the newline separator "(\r)\n" into a message stream. /// It can be difficult to split and compute offsets correctly when the bytes are received in diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index d2758ba9bb0ef..7efb13f409478 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -24,6 +24,7 @@ use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; +use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, SplitMetaData, @@ -81,10 +82,7 @@ impl OpendalReader { let parser = ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx).await?; - let msg_stream = if matches!( - parser, - ByteStreamSourceParserImpl::Json(_) | ByteStreamSourceParserImpl::Csv(_) - ) { + let msg_stream = if need_nd_streaming(&self.parser_config.specific.encoding_config) { parser.into_stream(nd_streaming::split_stream(data_stream)) } else { parser.into_stream(data_stream) diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 4d51dbc4d2b44..6cf4387ae3e92 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -36,11 +36,13 @@ use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::base::{SplitMetaData, SplitReader}; use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::nd_streaming; +use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::s3::S3Properties; use crate::source::{ BoxSourceWithStateStream, Column, SourceContextRef, SourceMessage, SourceMeta, StreamChunkWithState, }; + const MAX_CHANNEL_BUFFER_SIZE: usize = 2048; const STREAM_READER_CAPACITY: usize = 4096; @@ -221,10 +223,7 @@ impl S3FileReader { let parser = ByteStreamSourceParserImpl::create(self.parser_config.clone(), source_ctx).await?; - let msg_stream = if matches!( - parser, - ByteStreamSourceParserImpl::Json(_) | ByteStreamSourceParserImpl::Csv(_) - ) { + let msg_stream = if need_nd_streaming(&self.parser_config.specific.encoding_config) { parser.into_stream(nd_streaming::split_stream(data_stream)) } else { parser.into_stream(data_stream)