diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index 50612830c8125..28c1383e668f3 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -105,7 +105,7 @@ public MetaClient(String metaAddr, ScheduledExecutorService scheduler) { .build(); AddWorkerNodeResponse resp = clusterStub.addWorkerNode(req); - this.workerId = resp.getNode().getId(); + this.workerId = resp.getNodeId(); } public HummockVersion pinVersion() { diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 4b5e31877f437..db910930b5bee 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -16,8 +16,7 @@ message CreateDatabaseRequest { message CreateDatabaseResponse { common.Status status = 1; - uint32 database_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropDatabaseRequest { @@ -35,8 +34,7 @@ message CreateSchemaRequest { message CreateSchemaResponse { common.Status status = 1; - uint32 schema_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropSchemaRequest { @@ -55,8 +53,7 @@ message CreateSourceRequest { message CreateSourceResponse { common.Status status = 1; - uint32 source_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropSourceRequest { @@ -87,8 +84,7 @@ message CreateSinkRequest { message CreateSinkResponse { common.Status status = 1; - uint32 sink_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropSinkRequest { @@ -109,8 +105,7 @@ message CreateMaterializedViewRequest { message CreateMaterializedViewResponse { common.Status status = 1; - uint32 table_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropMaterializedViewRequest { @@ -129,8 +124,7 @@ message CreateViewRequest { message CreateViewResponse { common.Status status = 1; - uint32 view_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropViewRequest { @@ -166,8 +160,7 @@ message CreateTableRequest { message CreateTableResponse { common.Status status = 1; - uint32 table_id = 2; - uint64 version = 3; + uint64 version = 2; } message AlterNameRequest { @@ -235,8 +228,7 @@ message CreateFunctionRequest { message CreateFunctionResponse { common.Status status = 1; - uint32 function_id = 2; - uint64 version = 3; + uint64 version = 2; } message DropFunctionRequest { @@ -277,8 +269,7 @@ message CreateIndexRequest { message CreateIndexResponse { common.Status status = 1; - uint32 index_id = 2; - uint64 version = 4; + uint64 version = 2; } message DropIndexRequest { @@ -352,9 +343,8 @@ message CreateConnectionRequest { } message CreateConnectionResponse { - uint32 connection_id = 1; // global catalog version - uint64 version = 2; + uint64 version = 1; } message ListConnectionsRequest {} diff --git a/proto/meta.proto b/proto/meta.proto index e8483a86f1327..a54375be80827 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -288,11 +288,12 @@ message AddWorkerNodeResponse { reserved 3; reserved "system_params"; common.Status status = 1; - common.WorkerNode node = 2; + optional uint32 node_id = 2; } message ActivateWorkerNodeRequest { common.HostAddress host = 1; + uint32 node_id = 2; } message ActivateWorkerNodeResponse { diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index aa4c29eceaa4c..ca1a9ec0d5d26 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -195,7 +195,7 @@ pub struct CatalogWriterImpl { #[async_trait::async_trait] impl CatalogWriter for CatalogWriterImpl { async fn create_database(&self, db_name: &str, owner: UserId) -> Result<()> { - let (_, version) = self + let version = self .meta_client .create_database(PbDatabase { name: db_name.to_string(), @@ -212,7 +212,7 @@ impl CatalogWriter for CatalogWriterImpl { schema_name: &str, owner: UserId, ) -> Result<()> { - let (_, version) = self + let version = self .meta_client .create_schema(PbSchema { id: 0, @@ -231,7 +231,7 @@ impl CatalogWriter for CatalogWriterImpl { graph: StreamFragmentGraph, ) -> Result<()> { let create_type = table.get_create_type().unwrap_or(PbCreateType::Foreground); - let (_, version) = self + let version = self .meta_client .create_materialized_view(table, graph) .await?; @@ -242,7 +242,7 @@ impl CatalogWriter for CatalogWriterImpl { } async fn create_view(&self, view: PbView) -> Result<()> { - let (_, version) = self.meta_client.create_view(view).await?; + let version = self.meta_client.create_view(view).await?; self.wait_version(version).await } @@ -252,7 +252,7 @@ impl CatalogWriter for CatalogWriterImpl { table: PbTable, graph: StreamFragmentGraph, ) -> Result<()> { - let (_, version) = self.meta_client.create_index(index, table, graph).await?; + let version = self.meta_client.create_index(index, table, graph).await?; self.wait_version(version).await } @@ -263,7 +263,7 @@ impl CatalogWriter for CatalogWriterImpl { graph: StreamFragmentGraph, job_type: PbTableJobType, ) -> Result<()> { - let (_, version) = self + let version = self .meta_client .create_table(source, table, graph, job_type) .await?; @@ -290,7 +290,7 @@ impl CatalogWriter for CatalogWriterImpl { } async fn create_source(&self, source: PbSource) -> Result<()> { - let (_id, version) = self.meta_client.create_source(source).await?; + let version = self.meta_client.create_source(source).await?; self.wait_version(version).await } @@ -299,7 +299,7 @@ impl CatalogWriter for CatalogWriterImpl { source: PbSource, graph: StreamFragmentGraph, ) -> Result<()> { - let (_id, version) = self + let version = self .meta_client .create_source_with_graph(source, graph) .await?; @@ -312,7 +312,7 @@ impl CatalogWriter for CatalogWriterImpl { graph: StreamFragmentGraph, affected_table_change: Option, ) -> Result<()> { - let (_id, version) = self + let version = self .meta_client .create_sink(sink, graph, affected_table_change) .await?; @@ -320,7 +320,7 @@ impl CatalogWriter for CatalogWriterImpl { } async fn create_function(&self, function: PbFunction) -> Result<()> { - let (_, version) = self.meta_client.create_function(function).await?; + let version = self.meta_client.create_function(function).await?; self.wait_version(version).await } @@ -332,7 +332,7 @@ impl CatalogWriter for CatalogWriterImpl { owner_id: u32, connection: create_connection_request::Payload, ) -> Result<()> { - let (_, version) = self + let version = self .meta_client .create_connection( connection_name, diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index f2b4fa9fc4cbd..67bd1e8f6ba26 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -26,6 +26,9 @@ use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common_service::metrics_manager::MetricsManager; use risingwave_common_service::tracing::TracingExtractLayer; +use risingwave_meta::controller::catalog::CatalogController; +use risingwave_meta::controller::cluster::ClusterController; +use risingwave_meta::manager::MetadataManager; use risingwave_meta::rpc::intercept::MetricsMiddlewareLayer; use risingwave_meta::rpc::ElectionClientRef; use risingwave_meta_model_migration::{Migrator, MigratorTrait}; @@ -391,7 +394,6 @@ pub async fn start_service_as_election_leader( meta_store_sql.clone(), ) .await?; - let fragment_manager = Arc::new(FragmentManager::new(env.clone()).await.unwrap()); let system_params_manager = env.system_params_manager_ref(); let mut system_params_reader = system_params_manager.get_params().await; @@ -413,20 +415,33 @@ pub async fn start_service_as_election_leader( ))); } - let cluster_manager = Arc::new( - ClusterManager::new(env.clone(), max_cluster_heartbeat_interval) - .await - .unwrap(), - ); + let metadata_manager = if meta_store_sql.is_some() { + let cluster_controller = Arc::new( + ClusterController::new(env.clone(), max_cluster_heartbeat_interval) + .await + .unwrap(), + ); + let catalog_controller = Arc::new(CatalogController::new(env.clone()).unwrap()); + MetadataManager::new_v2(cluster_controller, catalog_controller) + } else { + MetadataManager::new_v1( + Arc::new( + ClusterManager::new(env.clone(), max_cluster_heartbeat_interval) + .await + .unwrap(), + ), + Arc::new(CatalogManager::new(env.clone()).await.unwrap()), + Arc::new(FragmentManager::new(env.clone()).await.unwrap()), + ) + }; + let serving_vnode_mapping = Arc::new(ServingVnodeMapping::default()); serving::on_meta_start( env.notification_manager_ref(), - cluster_manager.clone(), - fragment_manager.clone(), + &metadata_manager, serving_vnode_mapping.clone(), ) .await; - let heartbeat_srv = HeartbeatServiceImpl::new(cluster_manager.clone()); let compactor_manager = Arc::new( hummock::CompactorManager::with_meta(env.clone()) @@ -434,7 +449,8 @@ pub async fn start_service_as_election_leader( .unwrap(), ); - let catalog_manager = Arc::new(CatalogManager::new(env.clone()).await.unwrap()); + let heartbeat_srv = HeartbeatServiceImpl::new(metadata_manager.clone()); + let (compactor_streams_change_tx, compactor_streams_change_rx) = tokio::sync::mpsc::unbounded_channel(); @@ -442,11 +458,9 @@ pub async fn start_service_as_election_leader( let hummock_manager = hummock::HummockManager::new( env.clone(), - cluster_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), meta_metrics.clone(), compactor_manager.clone(), - catalog_manager.clone(), compactor_streams_change_tx, ) .await @@ -462,16 +476,22 @@ pub async fn start_service_as_election_leader( prometheus_http_query::Client::from_str(x).unwrap() }); let prometheus_selector = opts.prometheus_selector.unwrap_or_default(); - let diagnose_command = Arc::new(risingwave_meta::manager::diagnose::DiagnoseCommand::new( - cluster_manager.clone(), - catalog_manager.clone(), - fragment_manager.clone(), - hummock_manager.clone(), - env.event_log_manager_ref(), - prometheus_client.clone(), - prometheus_selector.clone(), - )); + let diagnose_command = match &metadata_manager { + MetadataManager::V1(mgr) => Some(Arc::new( + risingwave_meta::manager::diagnose::DiagnoseCommand::new( + mgr.cluster_manager.clone(), + mgr.catalog_manager.clone(), + mgr.fragment_manager.clone(), + hummock_manager.clone(), + env.event_log_manager_ref(), + prometheus_client.clone(), + prometheus_selector.clone(), + ), + )), + MetadataManager::V2(_) => None, + }; + let trace_state = otlp_embedded::State::new(opts.cached_traces_num); let trace_srv = otlp_embedded::TraceServiceImpl::new(trace_state.clone()); @@ -481,10 +501,8 @@ pub async fn start_service_as_election_leader( dashboard_addr: *dashboard_addr, prometheus_client, prometheus_selector, - cluster_manager: cluster_manager.clone(), - fragment_manager: fragment_manager.clone(), + metadata_manager: metadata_manager.clone(), compute_clients: ComputeClientPool::default(), - meta_store: env.meta_store_ref(), ui_path: address_info.ui_path, diagnose_command, trace_state, @@ -505,8 +523,7 @@ pub async fn start_service_as_election_leader( SourceManager::new( env.clone(), barrier_scheduler.clone(), - catalog_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), meta_metrics.clone(), ) .await @@ -519,9 +536,7 @@ pub async fn start_service_as_election_leader( let barrier_manager = Arc::new(GlobalBarrierManager::new( scheduled_barriers, env.clone(), - cluster_manager.clone(), - catalog_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), hummock_manager.clone(), source_manager.clone(), sink_manager.clone(), @@ -538,26 +553,31 @@ pub async fn start_service_as_election_leader( let stream_manager = Arc::new( GlobalStreamManager::new( env.clone(), - fragment_manager.clone(), + metadata_manager.clone(), barrier_scheduler.clone(), - cluster_manager.clone(), source_manager.clone(), hummock_manager.clone(), - catalog_manager.clone(), ) .unwrap(), ); - hummock_manager - .purge( - &catalog_manager - .list_tables() - .await - .into_iter() - .map(|t| t.id) - .collect_vec(), - ) - .await; + let all_state_table_ids = match &metadata_manager { + MetadataManager::V1(mgr) => mgr + .catalog_manager + .list_tables() + .await + .into_iter() + .map(|t| t.id) + .collect_vec(), + MetadataManager::V2(mgr) => mgr + .catalog_controller + .list_all_state_table_ids() + .await? + .into_iter() + .map(|id| id as u32) + .collect_vec(), + }; + hummock_manager.purge(&all_state_table_ids).await; // Initialize services. let backup_manager = BackupManager::new( @@ -586,47 +606,40 @@ pub async fn start_service_as_election_leader( let ddl_srv = DdlServiceImpl::new( env.clone(), aws_cli.clone(), - catalog_manager.clone(), + metadata_manager.clone(), stream_manager.clone(), source_manager.clone(), - cluster_manager.clone(), - fragment_manager.clone(), barrier_manager.clone(), sink_manager.clone(), ) .await; - let user_srv = UserServiceImpl::new(env.clone(), catalog_manager.clone()); + let user_srv = UserServiceImpl::new(env.clone(), metadata_manager.clone()); let scale_srv = ScaleServiceImpl::new( - fragment_manager.clone(), - cluster_manager.clone(), + metadata_manager.clone(), source_manager, - catalog_manager.clone(), stream_manager.clone(), barrier_manager.clone(), ); - let cluster_srv = ClusterServiceImpl::new(cluster_manager.clone()); + let cluster_srv = ClusterServiceImpl::new(metadata_manager.clone()); let stream_srv = StreamServiceImpl::new( env.clone(), barrier_scheduler.clone(), stream_manager.clone(), - catalog_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), ); let sink_coordination_srv = SinkCoordinationServiceImpl::new(sink_manager); let hummock_srv = HummockServiceImpl::new( hummock_manager.clone(), vacuum_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), ); let notification_srv = NotificationServiceImpl::new( env.clone(), - catalog_manager.clone(), - cluster_manager.clone(), + metadata_manager.clone(), hummock_manager.clone(), - fragment_manager.clone(), backup_manager.clone(), serving_vnode_mapping.clone(), ); @@ -638,8 +651,8 @@ pub async fn start_service_as_election_leader( system_params_controller.clone(), ); let serving_srv = - ServingServiceImpl::new(serving_vnode_mapping.clone(), fragment_manager.clone()); - let cloud_srv = CloudServiceImpl::new(catalog_manager.clone(), aws_cli); + ServingServiceImpl::new(serving_vnode_mapping.clone(), metadata_manager.clone()); + let cloud_srv = CloudServiceImpl::new(metadata_manager.clone(), aws_cli); let event_log_srv = EventLogServiceImpl::new(env.event_log_manager_ref()); if let Some(prometheus_addr) = address_info.prometheus_addr { @@ -654,15 +667,13 @@ pub async fn start_service_as_election_leader( &env.opts, )); sub_tasks.push(start_worker_info_monitor( - cluster_manager.clone(), + metadata_manager.clone(), election_client.clone(), Duration::from_secs(env.opts.node_num_monitor_interval_sec), meta_metrics.clone(), )); sub_tasks.push(start_fragment_info_monitor( - cluster_manager.clone(), - catalog_manager, - fragment_manager.clone(), + metadata_manager.clone(), hummock_manager.clone(), meta_metrics.clone(), )); @@ -683,18 +694,24 @@ pub async fn start_service_as_election_leader( sub_tasks.push( serving::start_serving_vnode_mapping_worker( env.notification_manager_ref(), - cluster_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), serving_vnode_mapping, ) .await, ); if cfg!(not(test)) { - sub_tasks.push(ClusterManager::start_heartbeat_checker( - cluster_manager.clone(), - Duration::from_secs(1), - )); + let task = match &metadata_manager { + MetadataManager::V1(mgr) => ClusterManager::start_heartbeat_checker( + mgr.cluster_manager.clone(), + Duration::from_secs(1), + ), + MetadataManager::V2(mgr) => ClusterController::start_heartbeat_checker( + mgr.cluster_controller.clone(), + Duration::from_secs(1), + ), + }; + sub_tasks.push(task); sub_tasks.push(GlobalBarrierManager::start(barrier_manager)); if env.opts.enable_automatic_parallelism_control { @@ -720,7 +737,7 @@ pub async fn start_service_as_election_leader( let telemetry_manager = TelemetryManager::new( Arc::new(MetaTelemetryInfoFetcher::new(env.cluster_id().clone())), Arc::new(MetaReportCreator::new( - cluster_manager, + metadata_manager.clone(), meta_store.meta_store_type(), )), ); diff --git a/src/meta/service/src/cloud_service.rs b/src/meta/service/src/cloud_service.rs index b57553fdc93c4..3b490ef7b37d0 100644 --- a/src/meta/service/src/cloud_service.rs +++ b/src/meta/service/src/cloud_service.rs @@ -10,7 +10,7 @@ // distributed under the License is distributed on an "AS IS" BASIS, // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and -// limitations under the License.use risingwave +// limitations under the License. use std::collections::{BTreeMap, HashMap}; use std::sync::LazyLock; @@ -22,6 +22,7 @@ use risingwave_connector::source::kafka::private_link::insert_privatelink_broker use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, }; +use risingwave_meta::manager::MetadataManager; use risingwave_pb::catalog::connection::Info::PrivateLinkService; use risingwave_pb::cloud_service::cloud_service_server::CloudService; use risingwave_pb::cloud_service::rw_cloud_validate_source_response::{Error, ErrorType}; @@ -30,18 +31,17 @@ use risingwave_pb::cloud_service::{ }; use tonic::{Request, Response, Status}; -use crate::manager::CatalogManagerRef; use crate::rpc::cloud_provider::AwsEc2Client; pub struct CloudServiceImpl { - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, aws_client: Option, } impl CloudServiceImpl { - pub fn new(catalog_manager: CatalogManagerRef, aws_client: Option) -> Self { + pub fn new(metadata_manager: MetadataManager, aws_client: Option) -> Self { Self { - catalog_manager, + metadata_manager, aws_client, } } @@ -80,10 +80,20 @@ impl CloudService for CloudServiceImpl { let connection_id = connection_id_str.parse().map_err(|e| { Status::invalid_argument(format!("connection.id is not an integer: {}", e)) })?; - let connection = self - .catalog_manager - .get_connection_by_id(connection_id) - .await; + + let connection = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .get_connection_by_id(connection_id) + .await + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_connection_by_id(connection_id as _) + .await + } + }; + if let Err(e) = connection { return Ok(new_rwc_validate_fail_response( ErrorType::PrivatelinkConnectionNotFound, diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index 97a7b3749bbcd..e94c40fe52719 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_meta::manager::MetadataManager; +use risingwave_meta_model_v2::WorkerId; use risingwave_pb::common::worker_node::State; use risingwave_pb::meta::cluster_service_server::ClusterService; use risingwave_pb::meta::{ @@ -22,17 +24,16 @@ use risingwave_pb::meta::{ }; use tonic::{Request, Response, Status}; -use crate::manager::ClusterManagerRef; use crate::MetaError; #[derive(Clone)] pub struct ClusterServiceImpl { - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, } impl ClusterServiceImpl { - pub fn new(cluster_manager: ClusterManagerRef) -> Self { - ClusterServiceImpl { cluster_manager } + pub fn new(metadata_manager: MetadataManager) -> Self { + ClusterServiceImpl { metadata_manager } } } @@ -50,13 +51,13 @@ impl ClusterService for ClusterServiceImpl { .ok_or_else(|| MetaError::invalid_parameter("worker node property is not provided"))?; let resource = req.resource.unwrap_or_default(); let result = self - .cluster_manager + .metadata_manager .add_worker_node(worker_type, host, property, resource) .await; match result { - Ok(worker_node) => Ok(Response::new(AddWorkerNodeResponse { + Ok(worker_id) => Ok(Response::new(AddWorkerNodeResponse { status: None, - node: Some(worker_node), + node_id: Some(worker_id), })), Err(e) => { if e.is_invalid_worker() { @@ -65,7 +66,7 @@ impl ClusterService for ClusterServiceImpl { code: risingwave_pb::common::status::Code::UnknownWorker as i32, message: format!("{}", e), }), - node: None, + node_id: None, })); } Err(e.into()) @@ -83,9 +84,21 @@ impl ClusterService for ClusterServiceImpl { let schedulability = req.get_schedulability()?; let worker_ids = req.worker_ids; - self.cluster_manager - .update_schedulability(worker_ids, schedulability) - .await?; + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.cluster_manager + .update_schedulability(worker_ids, schedulability) + .await? + } + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .update_schedulability( + worker_ids.into_iter().map(|id| id as WorkerId).collect(), + schedulability, + ) + .await? + } + } Ok(Response::new(UpdateWorkerNodeSchedulabilityResponse { status: None, @@ -98,7 +111,14 @@ impl ClusterService for ClusterServiceImpl { ) -> Result, Status> { let req = request.into_inner(); let host = req.get_host()?.clone(); - self.cluster_manager.activate_worker_node(host).await?; + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.cluster_manager.activate_worker_node(host).await?, + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .activate_worker(req.node_id as _) + .await? + } + } Ok(Response::new(ActivateWorkerNodeResponse { status: None })) } @@ -108,7 +128,15 @@ impl ClusterService for ClusterServiceImpl { ) -> Result, Status> { let req = request.into_inner(); let host = req.get_host()?.clone(); - self.cluster_manager.delete_worker_node(host).await?; + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let _ = mgr.cluster_manager.delete_worker_node(host).await?; + } + MetadataManager::V2(mgr) => { + let _ = mgr.cluster_controller.delete_worker(host).await?; + } + } + Ok(Response::new(DeleteWorkerNodeResponse { status: None })) } @@ -125,9 +153,9 @@ impl ClusterService for ClusterServiceImpl { }; let node_list = self - .cluster_manager + .metadata_manager .list_worker_node(worker_type, worker_states) - .await; + .await?; Ok(Response::new(ListAllNodesResponse { status: None, nodes: node_list, diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 40352527a414f..f85f69322e6f8 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -16,33 +16,24 @@ use std::collections::HashMap; use std::sync::Arc; use anyhow::anyhow; -use rand::Rng; -use risingwave_common::catalog::TableId; use risingwave_common::util::column_index_mapping::ColIndexMapping; -use risingwave_common::util::stream_graph_visitor::visit_fragment; use risingwave_connector::sink::catalog::SinkId; -use risingwave_connector::source::cdc::CdcSourceType; -use risingwave_connector::source::UPSTREAM_SOURCE_KEY; +use risingwave_meta::manager::MetadataManager; +use risingwave_meta::rpc::ddl_controller::fill_table_stream_graph_info; use risingwave_pb::catalog::connection::private_link_service::{ PbPrivateLinkProvider, PrivateLinkProvider, }; use risingwave_pb::catalog::connection::PbPrivateLinkService; -use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::table::OptionalAssociatedSourceId; -use risingwave_pb::catalog::{connection, Comment, Connection, CreateType, PbSource, PbTable}; +use risingwave_pb::catalog::{connection, Comment, Connection, CreateType}; use risingwave_pb::ddl_service::ddl_service_server::DdlService; use risingwave_pb::ddl_service::drop_table_request::PbSourceId; use risingwave_pb::ddl_service::*; -use risingwave_pb::stream_plan::stream_node::NodeBody; -use risingwave_pb::stream_plan::PbStreamFragmentGraph; use tonic::{Request, Response, Status}; use crate::barrier::BarrierManagerRef; use crate::manager::sink_coordination::SinkCoordinatorManager; -use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, ConnectionId, FragmentManagerRef, IdCategory, - IdCategoryType, MetaSrvEnv, StreamingJob, -}; +use crate::manager::{ConnectionId, MetaSrvEnv, StreamingJob}; use crate::rpc::cloud_provider::AwsEc2Client; use crate::rpc::ddl_controller::{ DdlCommand, DdlController, DropMode, ReplaceTableInfo, StreamingJobId, @@ -54,7 +45,7 @@ use crate::{MetaError, MetaResult}; pub struct DdlServiceImpl { env: MetaSrvEnv, - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, sink_manager: SinkCoordinatorManager, ddl_controller: DdlController, aws_client: Arc>, @@ -65,29 +56,25 @@ impl DdlServiceImpl { pub async fn new( env: MetaSrvEnv, aws_client: Option, - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, stream_manager: GlobalStreamManagerRef, source_manager: SourceManagerRef, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, barrier_manager: BarrierManagerRef, sink_manager: SinkCoordinatorManager, ) -> Self { let aws_cli_ref = Arc::new(aws_client); let ddl_controller = DdlController::new( env.clone(), - catalog_manager.clone(), + metadata_manager.clone(), stream_manager, source_manager, - cluster_manager, - fragment_manager, barrier_manager, aws_cli_ref.clone(), ) .await; Self { env, - catalog_manager, + metadata_manager, ddl_controller, aws_client: aws_cli_ref, sink_manager, @@ -102,10 +89,10 @@ impl DdlServiceImpl { table.optional_associated_source_id { let source = source.as_mut().unwrap(); - let table_id = table.id; + source.id = source_id; fill_table_stream_graph_info( - Some((source, source_id)), - (&mut table, table_id), + source, + &mut table, TableJobType::General, &mut fragment_graph, ); @@ -132,9 +119,7 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let id = self.gen_unique_id::<{ IdCategory::Database }>().await?; - let mut database = req.get_db()?.clone(); - database.id = id; + let database = req.get_db()?.clone(); let version = self .ddl_controller .run_command(DdlCommand::CreateDatabase(database)) @@ -142,7 +127,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateDatabaseResponse { status: None, - database_id: id, version, })) } @@ -170,9 +154,7 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let id = self.gen_unique_id::<{ IdCategory::Schema }>().await?; - let mut schema = req.get_schema()?.clone(); - schema.id = id; + let schema = req.get_schema()?.clone(); let version = self .ddl_controller .run_command(DdlCommand::CreateSchema(schema)) @@ -180,7 +162,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateSchemaResponse { status: None, - schema_id: id, version, })) } @@ -206,16 +187,13 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let mut source = req.get_source()?.clone(); + let source = req.get_source()?.clone(); // validate connection before starting the DDL procedure if let Some(connection_id) = source.connection_id { self.validate_connection(connection_id).await?; } - let source_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - source.id = source_id; - match req.fragment_graph { None => { let version = self @@ -224,19 +202,10 @@ impl DdlService for DdlServiceImpl { .await?; Ok(Response::new(CreateSourceResponse { status: None, - source_id, version, })) } - Some(mut fragment_graph) => { - for fragment in fragment_graph.fragments.values_mut() { - visit_fragment(fragment, |node_body| { - if let NodeBody::Source(source_node) = node_body { - source_node.source_inner.as_mut().unwrap().source_id = source_id; - } - }); - } - + Some(fragment_graph) => { // The id of stream job has been set above let stream_job = StreamingJob::Source(source); let version = self @@ -250,7 +219,6 @@ impl DdlService for DdlServiceImpl { .await?; Ok(Response::new(CreateSourceResponse { status: None, - source_id, version, })) } @@ -292,7 +260,7 @@ impl DdlService for DdlServiceImpl { self.validate_connection(connection_id).await?; } - let mut stream_job = match &affected_table_change { + let stream_job = match &affected_table_change { None => StreamingJob::Sink(sink, None), Some(change) => { let table = change.table.clone().unwrap(); @@ -301,10 +269,6 @@ impl DdlService for DdlServiceImpl { } }; - let id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - - stream_job.set_id(id); - let command = DdlCommand::CreateStreamingJob( stream_job, fragment_graph, @@ -316,7 +280,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateSinkResponse { status: None, - sink_id: id, version, })) } @@ -360,10 +323,7 @@ impl DdlService for DdlServiceImpl { let create_type = mview.get_create_type().unwrap_or(CreateType::Foreground); let fragment_graph = req.get_fragment_graph()?.clone(); - let mut stream_job = StreamingJob::MaterializedView(mview); - let id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - stream_job.set_id(id); - + let stream_job = StreamingJob::MaterializedView(mview); let version = self .ddl_controller .run_command(DdlCommand::CreateStreamingJob( @@ -376,7 +336,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateMaterializedViewResponse { status: None, - table_id: id, version, })) } @@ -417,10 +376,7 @@ impl DdlService for DdlServiceImpl { let index_table = req.get_index_table()?.clone(); let fragment_graph = req.get_fragment_graph()?.clone(); - let mut stream_job = StreamingJob::Index(index, index_table); - let id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - stream_job.set_id(id); - + let stream_job = StreamingJob::Index(index, index_table); let version = self .ddl_controller .run_command(DdlCommand::CreateStreamingJob( @@ -433,7 +389,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateIndexResponse { status: None, - index_id: id, version, })) } @@ -467,9 +422,8 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let id = self.gen_unique_id::<{ IdCategory::Function }>().await?; - let mut function = req.get_function()?.clone(); - function.id = id; + let function = req.get_function()?.clone(); + let version = self .ddl_controller .run_command(DdlCommand::CreateFunction(function)) @@ -477,7 +431,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateFunctionResponse { status: None, - function_id: id, version, })) } @@ -505,29 +458,11 @@ impl DdlService for DdlServiceImpl { ) -> Result, Status> { let request = request.into_inner(); let job_type = request.get_job_type().unwrap_or_default(); - let mut source = request.source; - let mut mview = request.materialized_view.unwrap(); - let mut fragment_graph = request.fragment_graph.unwrap(); - let table_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - - // If we're creating a table with connector, we should additionally fill its ID first. - let source_id = if source.is_some() { - // Generate source id. - self.gen_unique_id::<{ IdCategory::Table }>().await? // TODO: Use source category - } else { - TableId::placeholder().into() - }; - - fill_table_stream_graph_info( - source.as_mut().map(|source| (source, source_id)), - (&mut mview, table_id), - job_type, - &mut fragment_graph, - ); - - let mut stream_job = StreamingJob::Table(source, mview, job_type); - stream_job.set_id(table_id); + let source = request.source; + let mview = request.materialized_view.unwrap(); + let fragment_graph = request.fragment_graph.unwrap(); + let stream_job = StreamingJob::Table(source, mview, job_type); let version = self .ddl_controller .run_command(DdlCommand::CreateStreamingJob( @@ -540,7 +475,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateTableResponse { status: None, - table_id, version, })) } @@ -574,9 +508,7 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let mut view = req.get_view()?.clone(); - let id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - view.id = id; + let view = req.get_view()?.clone(); let version = self .ddl_controller @@ -585,7 +517,6 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(CreateViewResponse { status: None, - view_id: id, version, })) } @@ -611,7 +542,10 @@ impl DdlService for DdlServiceImpl { &self, _request: Request, ) -> Result, Status> { - let tables = self.catalog_manager.list_tables().await; + let tables = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.list_tables().await, + MetadataManager::V2(mgr) => mgr.catalog_controller.list_all_state_tables().await?, + }; Ok(Response::new(RisectlListStateTablesResponse { tables })) } @@ -639,23 +573,32 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let database = self - .catalog_manager - .list_databases() - .await - .into_iter() - .find(|db| db.name == req.database_name); - if let Some(db) = database { - let table = self - .catalog_manager - .list_tables() - .await - .into_iter() - .find(|t| t.name == req.table_name && t.database_id == db.id); - Ok(Response::new(GetTableResponse { table })) - } else { - Ok(Response::new(GetTableResponse { table: None })) - } + let table = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let database = mgr + .catalog_manager + .list_databases() + .await + .into_iter() + .find(|db| db.name == req.database_name); + if let Some(db) = database { + mgr.catalog_manager + .list_tables() + .await + .into_iter() + .find(|t| t.name == req.table_name && t.database_id == db.id) + } else { + None + } + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_table_by_name(&req.database_name, &req.table_name) + .await? + } + }; + + Ok(Response::new(GetTableResponse { table })) } async fn alter_name( @@ -695,7 +638,7 @@ impl DdlService for DdlServiceImpl { let AlterOwnerRequest { object, owner_id } = request.into_inner(); let version = self .ddl_controller - .run_command(DdlCommand::AlterTableOwner(object.unwrap(), owner_id)) + .run_command(DdlCommand::AlterObjectOwner(object.unwrap(), owner_id)) .await?; Ok(Response::new(AlterOwnerResponse { status: None, @@ -779,9 +722,8 @@ impl DdlService for DdlServiceImpl { return Err(Status::invalid_argument("Privatelink provider unspecified")); } }; - let id = self.gen_unique_id::<{ IdCategory::Connection }>().await?; let connection = Connection { - id, + id: 0, schema_id: req.schema_id, database_id: req.database_id, name: req.name, @@ -795,10 +737,7 @@ impl DdlService for DdlServiceImpl { .run_command(DdlCommand::CreateConnection(connection)) .await?; - Ok(Response::new(CreateConnectionResponse { - connection_id: id, - version, - })) + Ok(Response::new(CreateConnectionResponse { version })) } } } @@ -807,7 +746,11 @@ impl DdlService for DdlServiceImpl { &self, _request: Request, ) -> Result, Status> { - let conns = self.catalog_manager.list_connections().await; + let conns = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.list_connections().await, + MetadataManager::V2(mgr) => mgr.catalog_controller.list_connections().await?, + }; + Ok(Response::new(ListConnectionsResponse { connections: conns, })) @@ -859,10 +802,26 @@ impl DdlService for DdlServiceImpl { &self, request: Request, ) -> Result, Status> { - let ret = self - .catalog_manager - .get_tables(&request.into_inner().table_ids) - .await; + let ret = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .get_tables(&request.into_inner().table_ids) + .await + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_table_by_ids( + request + .into_inner() + .table_ids + .into_iter() + .map(|id| id as _) + .collect(), + ) + .await? + } + }; + let mut tables = HashMap::default(); for table in ret { tables.insert(table.id, table); @@ -893,16 +852,19 @@ impl DdlService for DdlServiceImpl { } impl DdlServiceImpl { - async fn gen_unique_id(&self) -> MetaResult { - let id = self.env.id_gen_manager().generate::().await? as u32; - Ok(id) - } - async fn validate_connection(&self, connection_id: ConnectionId) -> MetaResult<()> { - let connection = self - .catalog_manager - .get_connection_by_id(connection_id) - .await?; + let connection = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .get_connection_by_id(connection_id) + .await? + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .get_connection_by_id(connection_id as _) + .await? + } + }; if let Some(connection::Info::PrivateLinkService(svc)) = &connection.info { // skip all checks for mock connection if svc.get_provider()? == PrivateLinkProvider::Mock { @@ -922,73 +884,3 @@ impl DdlServiceImpl { Ok(()) } } - -/// Fill in necessary information for table stream graph. -fn fill_table_stream_graph_info( - mut source_info: Option<(&mut PbSource, u32)>, - table_info: (&mut PbTable, u32), - table_job_type: TableJobType, - fragment_graph: &mut PbStreamFragmentGraph, -) { - let (table, table_id) = table_info; - for fragment in fragment_graph.fragments.values_mut() { - visit_fragment(fragment, |node_body| { - if let NodeBody::Source(source_node) = node_body { - if source_node.source_inner.is_none() { - // skip empty source for dml node - return; - } - - // If we're creating a table with connector, we should additionally fill its ID first. - if let Some(&mut (ref mut source, source_id)) = source_info.as_mut() { - source.id = source_id; - let mut source_count = 0; - - source_node.source_inner.as_mut().unwrap().source_id = source_id; - source_count += 1; - - // Generate a random server id for mysql cdc source if needed - // `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication - // group (that is, different from any other server id being used by any master or slave) - if let Some(connector) = source.with_properties.get(UPSTREAM_SOURCE_KEY) - && matches!( - CdcSourceType::from(connector.as_str()), - CdcSourceType::Mysql - ) - { - let props = &mut source_node.source_inner.as_mut().unwrap().with_properties; - let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); - props - .entry("server.id".to_string()) - .or_insert(rand_server_id.to_string()); - - // make these two `Source` consistent - props.clone_into(&mut source.with_properties); - } - - assert_eq!( - source_count, 1, - "require exactly 1 external stream source when creating table with a connector" - ); - - // Fill in the correct table id for source. - source.optional_associated_table_id = - Some(OptionalAssociatedTableId::AssociatedTableId(table_id)); - - // Fill in the correct source id for mview. - table.optional_associated_source_id = - Some(OptionalAssociatedSourceId::AssociatedSourceId(source_id)); - } - } - - // fill table id for cdc backfill - if let NodeBody::StreamCdcScan(node) = node_body - && table_job_type == TableJobType::SharedCdcSource - { - if let Some(table) = node.cdc_table_desc.as_mut() { - table.table_id = table_id; - } - } - }); - } -} diff --git a/src/meta/service/src/heartbeat_service.rs b/src/meta/service/src/heartbeat_service.rs index d537df0f30731..7770acc06db59 100644 --- a/src/meta/service/src/heartbeat_service.rs +++ b/src/meta/service/src/heartbeat_service.rs @@ -13,20 +13,19 @@ // limitations under the License. use itertools::Itertools; +use risingwave_meta::manager::MetadataManager; use risingwave_pb::meta::heartbeat_service_server::HeartbeatService; use risingwave_pb::meta::{HeartbeatRequest, HeartbeatResponse}; use tonic::{Request, Response, Status}; -use crate::manager::ClusterManagerRef; - #[derive(Clone)] pub struct HeartbeatServiceImpl { - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, } impl HeartbeatServiceImpl { - pub fn new(cluster_manager: ClusterManagerRef) -> Self { - HeartbeatServiceImpl { cluster_manager } + pub fn new(metadata_manager: MetadataManager) -> Self { + HeartbeatServiceImpl { metadata_manager } } } @@ -38,16 +37,20 @@ impl HeartbeatService for HeartbeatServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let result = self - .cluster_manager - .heartbeat( - req.node_id, - req.info - .into_iter() - .filter_map(|node_info| node_info.info) - .collect_vec(), - ) - .await; + let info = req + .info + .into_iter() + .filter_map(|node_info| node_info.info) + .collect_vec(); + let result = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.cluster_manager.heartbeat(req.node_id, info).await, + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .heartbeat(req.node_id as _, info) + .await + } + }; + match result { Ok(_) => Ok(Response::new(HeartbeatResponse { status: None })), Err(e) => { diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index f73510832298c..599b7a7cdb79d 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -19,6 +19,7 @@ use futures::StreamExt; use itertools::Itertools; use risingwave_common::catalog::{TableId, NON_RESERVED_SYS_CATALOG_ID}; use risingwave_hummock_sdk::version::HummockVersionDelta; +use risingwave_meta::manager::MetadataManager; 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; @@ -27,25 +28,24 @@ use tonic::{Request, Response, Status, Streaming}; 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, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, } impl HummockServiceImpl { pub fn new( hummock_manager: HummockManagerRef, vacuum_trigger: VacuumManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, ) -> Self { HummockServiceImpl { hummock_manager, vacuum_manager: vacuum_trigger, - fragment_manager, + metadata_manager, } } } @@ -242,13 +242,13 @@ impl HummockManagerService for HummockServiceImpl { } } - // get internal_table_id by fragment_manager + // get internal_table_id by metadata_manger if request.table_id >= NON_RESERVED_SYS_CATALOG_ID as u32 { // We need to make sure to use the correct table_id to filter sst let table_id = TableId::new(request.table_id); if let Ok(table_fragment) = self - .fragment_manager - .select_table_fragments_by_table_id(&table_id) + .metadata_manager + .get_job_fragments_by_id(&table_id) .await { option.internal_table_id = HashSet::from_iter(table_fragment.all_table_ids()); @@ -333,7 +333,7 @@ impl HummockManagerService for HummockServiceImpl { let workers = self .hummock_manager .list_workers(&pinned_versions.iter().map(|v| v.context_id).collect_vec()) - .await; + .await?; Ok(Response::new(RiseCtlGetPinnedVersionsSummaryResponse { summary: Some(PinnedVersionsSummary { pinned_versions, @@ -350,7 +350,7 @@ impl HummockManagerService for HummockServiceImpl { let workers = self .hummock_manager .list_workers(&pinned_snapshots.iter().map(|p| p.context_id).collect_vec()) - .await; + .await?; Ok(Response::new(RiseCtlGetPinnedSnapshotsSummaryResponse { summary: Some(PinnedSnapshotsSummary { pinned_snapshots, @@ -505,7 +505,7 @@ impl HummockManagerService for HummockServiceImpl { // check_context and add_compactor as a whole is not atomic, but compactor_manager will // remove invalid compactor eventually. - if !self.hummock_manager.check_context(context_id).await { + if !self.hummock_manager.check_context(context_id).await? { return Err(Status::new( tonic::Code::Internal, format!("invalid hummock context {}", context_id), diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index b6d7741124840..06f61e5f26e50 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -13,6 +13,8 @@ // limitations under the License. use itertools::Itertools; +use risingwave_meta::manager::MetadataManager; +use risingwave_meta::MetaResult; use risingwave_pb::backup_service::MetaBackupManifestId; use risingwave_pb::catalog::Table; use risingwave_pb::common::worker_node::State::Running; @@ -30,19 +32,14 @@ use tonic::{Request, Response, Status}; use crate::backup_restore::BackupManagerRef; use crate::hummock::HummockManagerRef; -use crate::manager::{ - Catalog, CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, MetaSrvEnv, Notification, - NotificationVersion, WorkerKey, -}; +use crate::manager::{Catalog, MetaSrvEnv, Notification, NotificationVersion, WorkerKey}; use crate::serving::ServingVnodeMappingRef; pub struct NotificationServiceImpl { env: MetaSrvEnv, - catalog_manager: CatalogManagerRef, - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, - fragment_manager: FragmentManagerRef, backup_manager: BackupManagerRef, serving_vnode_mapping: ServingVnodeMappingRef, } @@ -50,54 +47,112 @@ pub struct NotificationServiceImpl { impl NotificationServiceImpl { pub fn new( env: MetaSrvEnv, - catalog_manager: CatalogManagerRef, - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, - fragment_manager: FragmentManagerRef, backup_manager: BackupManagerRef, serving_vnode_mapping: ServingVnodeMappingRef, ) -> Self { Self { env, - catalog_manager, - cluster_manager, + metadata_manager, hummock_manager, - fragment_manager, backup_manager, serving_vnode_mapping, } } - async fn get_catalog_snapshot(&self) -> (Catalog, Vec, NotificationVersion) { - let catalog_guard = self.catalog_manager.get_catalog_core_guard().await; - let (databases, schemas, tables, sources, sinks, indexes, views, functions, connections) = - catalog_guard.database.get_catalog(); - let users = catalog_guard.user.list_users(); - let notification_version = self.env.notification_manager().current_version().await; - ( - ( - databases, - schemas, - tables, - sources, - sinks, - indexes, - views, - functions, - connections, - ), - users, - notification_version, - ) + async fn get_catalog_snapshot( + &self, + ) -> MetaResult<(Catalog, Vec, NotificationVersion)> { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let catalog_guard = mgr.catalog_manager.get_catalog_core_guard().await; + let ( + databases, + schemas, + tables, + sources, + sinks, + indexes, + views, + functions, + connections, + ) = catalog_guard.database.get_catalog(); + let users = catalog_guard.user.list_users(); + let notification_version = self.env.notification_manager().current_version().await; + Ok(( + ( + databases, + schemas, + tables, + sources, + sinks, + indexes, + views, + functions, + connections, + ), + users, + notification_version, + )) + } + MetadataManager::V2(mgr) => { + let catalog_guard = mgr.catalog_controller.get_inner_read_guard().await; + let ( + ( + databases, + schemas, + tables, + sources, + sinks, + indexes, + views, + functions, + connections, + ), + users, + ) = catalog_guard.snapshot().await?; + let notification_version = self.env.notification_manager().current_version().await; + Ok(( + ( + databases, + schemas, + tables, + sources, + sinks, + indexes, + views, + functions, + connections, + ), + users, + notification_version, + )) + } + } } async fn get_parallel_unit_mapping_snapshot( &self, - ) -> (Vec, NotificationVersion) { - let fragment_guard = self.fragment_manager.get_fragment_read_guard().await; - let parallel_unit_mappings = fragment_guard.all_running_fragment_mappings().collect_vec(); - let notification_version = self.env.notification_manager().current_version().await; - (parallel_unit_mappings, notification_version) + ) -> MetaResult<(Vec, NotificationVersion)> { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let fragment_guard = mgr.fragment_manager.get_fragment_read_guard().await; + let parallel_unit_mappings = + fragment_guard.all_running_fragment_mappings().collect_vec(); + let notification_version = self.env.notification_manager().current_version().await; + Ok((parallel_unit_mappings, notification_version)) + } + MetadataManager::V2(mgr) => { + let fragment_guard = mgr.catalog_controller.get_inner_read_guard().await; + let parallel_unit_mappings = fragment_guard + .all_running_fragment_mappings() + .await? + .collect_vec(); + let notification_version = self.env.notification_manager().current_version().await; + Ok((parallel_unit_mappings, notification_version)) + } + } } fn get_serving_vnode_mappings(&self) -> Vec { @@ -111,47 +166,72 @@ impl NotificationServiceImpl { .collect() } - async fn get_worker_node_snapshot(&self) -> (Vec, NotificationVersion) { - let cluster_guard = self.cluster_manager.get_cluster_core_guard().await; - let nodes = cluster_guard.list_worker_node(Some(WorkerType::ComputeNode), Some(Running)); - let notification_version = self.env.notification_manager().current_version().await; - (nodes, notification_version) + async fn get_worker_node_snapshot(&self) -> MetaResult<(Vec, NotificationVersion)> { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let cluster_guard = mgr.cluster_manager.get_cluster_core_guard().await; + let nodes = + cluster_guard.list_worker_node(Some(WorkerType::ComputeNode), Some(Running)); + let notification_version = self.env.notification_manager().current_version().await; + Ok((nodes, notification_version)) + } + MetadataManager::V2(mgr) => { + let cluster_guard = mgr.cluster_controller.get_inner_read_guard().await; + let nodes = cluster_guard + .list_workers(Some(WorkerType::ComputeNode.into()), Some(Running.into())) + .await?; + let notification_version = self.env.notification_manager().current_version().await; + Ok((nodes, notification_version)) + } + } } - async fn get_tables_and_creating_tables_snapshot(&self) -> (Vec, NotificationVersion) { - let catalog_guard = self.catalog_manager.get_catalog_core_guard().await; - let tables = catalog_guard.database.list_tables(); - let notification_version = self.env.notification_manager().current_version().await; - (tables, notification_version) + async fn get_tables_and_creating_tables_snapshot( + &self, + ) -> MetaResult<(Vec
, NotificationVersion)> { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let catalog_guard = mgr.catalog_manager.get_catalog_core_guard().await; + let tables = catalog_guard.database.list_tables(); + let notification_version = self.env.notification_manager().current_version().await; + Ok((tables, notification_version)) + } + MetadataManager::V2(mgr) => { + let catalog_guard = mgr.catalog_controller.get_inner_read_guard().await; + let tables = catalog_guard.list_all_state_tables().await?; + let notification_version = self.env.notification_manager().current_version().await; + Ok((tables, notification_version)) + } + } } - async fn compactor_subscribe(&self) -> MetaSnapshot { - let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await; + async fn compactor_subscribe(&self) -> MetaResult { + let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await?; - MetaSnapshot { + Ok(MetaSnapshot { tables, version: Some(SnapshotVersion { catalog_version, ..Default::default() }), ..Default::default() - } + }) } - async fn frontend_subscribe(&self) -> MetaSnapshot { + async fn frontend_subscribe(&self) -> MetaResult { let ( (databases, schemas, tables, sources, sinks, indexes, views, functions, connections), users, catalog_version, - ) = self.get_catalog_snapshot().await; + ) = self.get_catalog_snapshot().await?; let (parallel_unit_mappings, parallel_unit_mapping_version) = - self.get_parallel_unit_mapping_snapshot().await; + self.get_parallel_unit_mapping_snapshot().await?; let serving_parallel_unit_mappings = self.get_serving_vnode_mappings(); - let (nodes, worker_node_version) = self.get_worker_node_snapshot().await; + let (nodes, worker_node_version) = self.get_worker_node_snapshot().await?; let hummock_snapshot = Some(self.hummock_manager.latest_snapshot()); - MetaSnapshot { + Ok(MetaSnapshot { databases, schemas, sources, @@ -172,16 +252,16 @@ impl NotificationServiceImpl { worker_node_version, }), ..Default::default() - } + }) } - async fn hummock_subscribe(&self) -> MetaSnapshot { - let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await; + async fn hummock_subscribe(&self) -> MetaResult { + let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await?; let hummock_version = self.hummock_manager.get_current_version().await; let hummock_write_limits = self.hummock_manager.write_limits().await; let meta_backup_manifest_id = self.backup_manager.manifest().manifest_id; - MetaSnapshot { + Ok(MetaSnapshot { tables, hummock_version: Some(hummock_version.to_protobuf()), version: Some(SnapshotVersion { @@ -195,7 +275,7 @@ impl NotificationServiceImpl { write_limits: hummock_write_limits, }), ..Default::default() - } + }) } fn compute_subscribe(&self) -> MetaSnapshot { @@ -225,18 +305,18 @@ impl NotificationService for NotificationServiceImpl { .await; let meta_snapshot = match subscribe_type { - SubscribeType::Compactor => self.compactor_subscribe().await, + SubscribeType::Compactor => self.compactor_subscribe().await?, SubscribeType::Frontend => { self.hummock_manager .pin_snapshot(req.get_worker_id()) .await?; - self.frontend_subscribe().await + self.frontend_subscribe().await? } SubscribeType::Hummock => { self.hummock_manager .pin_version(req.get_worker_id()) .await?; - self.hummock_subscribe().await + self.hummock_subscribe().await? } SubscribeType::Compute => self.compute_subscribe(), SubscribeType::Unspecified => unreachable!(), diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index 08aff6da75d12..acfca997d8040 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -15,8 +15,9 @@ use std::collections::HashMap; use std::sync::Arc; +use risingwave_meta::manager::MetadataManager; use risingwave_meta::model::TableParallelism; -use risingwave_meta::stream::{ScaleController, ScaleControllerRef}; +use risingwave_meta::stream::{ScaleController, ScaleControllerRef, TableRevision}; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::scale_service_server::ScaleService; use risingwave_pb::meta::{ @@ -27,47 +28,49 @@ use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; use tonic::{Request, Response, Status}; use crate::barrier::BarrierManagerRef; -use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef}; use crate::model::MetadataModel; use crate::stream::{ GlobalStreamManagerRef, ParallelUnitReschedule, RescheduleOptions, SourceManagerRef, }; pub struct ScaleServiceImpl { - fragment_manager: FragmentManagerRef, - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, source_manager: SourceManagerRef, - catalog_manager: CatalogManagerRef, stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, - scale_controller: ScaleControllerRef, + scale_controller: Option, } impl ScaleServiceImpl { pub fn new( - fragment_manager: FragmentManagerRef, - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, source_manager: SourceManagerRef, - catalog_manager: CatalogManagerRef, stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, ) -> Self { - let scale_controller = Arc::new(ScaleController::new( - fragment_manager.clone(), - cluster_manager.clone(), - source_manager.clone(), - stream_manager.env.clone(), - )); + let scale_controller = match &metadata_manager { + MetadataManager::V1(_) => Some(Arc::new(ScaleController::new( + &metadata_manager, + source_manager.clone(), + stream_manager.env.clone(), + ))), + MetadataManager::V2(_) => None, + }; Self { - fragment_manager, - cluster_manager, + metadata_manager, source_manager, - catalog_manager, stream_manager, barrier_manager, scale_controller, } } + + async fn get_revision(&self) -> TableRevision { + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.fragment_manager.get_revision().await, + MetadataManager::V2(_) => unimplemented!("support table revision in v2"), + } + } } #[async_trait::async_trait] @@ -79,19 +82,28 @@ impl ScaleService for ScaleServiceImpl { ) -> Result, Status> { let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; - let table_fragments = self - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - .map(|tf| tf.to_protobuf()) - .collect(); + let table_fragments = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .values() + .map(|tf| tf.to_protobuf()) + .collect(), + MetadataManager::V2(mgr) => mgr + .catalog_controller + .table_fragments() + .await? + .values() + .cloned() + .collect(), + }; let worker_nodes = self - .cluster_manager + .metadata_manager .list_worker_node(Some(WorkerType::ComputeNode), None) - .await; + .await?; let actor_splits = self .source_manager @@ -108,11 +120,10 @@ impl ScaleService for ScaleServiceImpl { }) .collect(); - let sources = self.catalog_manager.list_sources().await; - + let sources = self.metadata_manager.list_sources().await?; let source_infos = sources.into_iter().map(|s| (s.id, s)).collect(); - let revision = self.fragment_manager.get_revision().await.inner(); + let revision = self.get_revision().await.inner(); Ok(Response::new(GetClusterInfoResponse { worker_nodes, @@ -130,6 +141,10 @@ impl ScaleService for ScaleServiceImpl { ) -> Result, Status> { self.barrier_manager.check_status_running().await?; + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("only available in v1"); + }; + let RescheduleRequest { reschedules, revision, @@ -138,7 +153,7 @@ impl ScaleService for ScaleServiceImpl { let _reschedule_job_lock = self.stream_manager.reschedule_lock.write().await; - let current_revision = self.fragment_manager.get_revision().await; + let current_revision = self.get_revision().await; if revision != current_revision.inner() { return Ok(Response::new(RescheduleResponse { @@ -148,7 +163,7 @@ impl ScaleService for ScaleServiceImpl { } let table_parallelisms = { - let guard = self.fragment_manager.get_fragment_read_guard().await; + let guard = mgr.fragment_manager.get_fragment_read_guard().await; let mut table_parallelisms = HashMap::new(); for (table_id, table) in guard.table_fragments() { @@ -192,7 +207,7 @@ impl ScaleService for ScaleServiceImpl { ) .await?; - let next_revision = self.fragment_manager.get_revision().await; + let next_revision = self.get_revision().await; Ok(Response::new(RescheduleResponse { success: true, @@ -211,7 +226,7 @@ impl ScaleService for ScaleServiceImpl { let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; - let current_revision = self.fragment_manager.get_revision().await; + let current_revision = self.get_revision().await; if req.revision != current_revision.inner() { return Ok(Response::new(GetReschedulePlanResponse { @@ -225,9 +240,14 @@ impl ScaleService for ScaleServiceImpl { .policy .ok_or_else(|| Status::invalid_argument("policy is required"))?; - let plan = self.scale_controller.get_reschedule_plan(policy).await?; + let Some(scale_controller) = &self.scale_controller else { + return Err(Status::unimplemented( + "reschedule plan is not supported in v2", + )); + }; + let plan = scale_controller.get_reschedule_plan(policy).await?; - let next_revision = self.fragment_manager.get_revision().await; + let next_revision = self.get_revision().await; // generate reschedule plan will not change the revision assert_eq!(current_revision, next_revision); diff --git a/src/meta/service/src/serving_service.rs b/src/meta/service/src/serving_service.rs index 8af3b1ce64d1c..d1b013e078e0f 100644 --- a/src/meta/service/src/serving_service.rs +++ b/src/meta/service/src/serving_service.rs @@ -13,28 +13,28 @@ // limitations under the License. use itertools::Itertools; +use risingwave_meta::manager::MetadataManager; use risingwave_pb::meta::serving_service_server::ServingService; use risingwave_pb::meta::{ FragmentParallelUnitMapping, GetServingVnodeMappingsRequest, GetServingVnodeMappingsResponse, }; use tonic::{Request, Response, Status}; -use crate::manager::FragmentManagerRef; use crate::serving::ServingVnodeMappingRef; pub struct ServingServiceImpl { serving_vnode_mapping: ServingVnodeMappingRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, } impl ServingServiceImpl { pub fn new( serving_vnode_mapping: ServingVnodeMappingRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, ) -> Self { Self { serving_vnode_mapping, - fragment_manager, + metadata_manager, } } } @@ -55,16 +55,27 @@ impl ServingService for ServingServiceImpl { }) .collect(); let fragment_to_table = { - let guard = self.fragment_manager.get_fragment_read_guard().await; - guard - .table_fragments() - .iter() - .flat_map(|(table_id, tf)| { - tf.fragment_ids() - .map(|fragment_id| (fragment_id, table_id.table_id)) - .collect_vec() - }) - .collect() + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let guard = mgr.fragment_manager.get_fragment_read_guard().await; + guard + .table_fragments() + .iter() + .flat_map(|(table_id, tf)| { + tf.fragment_ids() + .map(|fragment_id| (fragment_id, table_id.table_id)) + .collect_vec() + }) + .collect() + } + MetadataManager::V2(mgr) => mgr + .catalog_controller + .fragment_job_mapping() + .await? + .into_iter() + .map(|(fragment_id, job_id)| (fragment_id as u32, job_id as u32)) + .collect(), + } }; Ok(Response::new(GetServingVnodeMappingsResponse { mappings, diff --git a/src/meta/service/src/stream_service.rs b/src/meta/service/src/stream_service.rs index 4fdd6eae07fad..53ce748915c02 100644 --- a/src/meta/service/src/stream_service.rs +++ b/src/meta/service/src/stream_service.rs @@ -16,6 +16,7 @@ use std::collections::{HashMap, HashSet}; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_meta::manager::MetadataManager; use risingwave_meta::model::ActorId; use risingwave_meta::stream::ThrottleConfig; use risingwave_meta_model_v2::SourceId; @@ -24,11 +25,14 @@ use risingwave_pb::meta::list_table_fragments_response::{ ActorInfo, FragmentInfo, TableFragmentInfo, }; use risingwave_pb::meta::stream_manager_service_server::StreamManagerService; +use risingwave_pb::meta::table_fragments::actor_status::PbActorState; +use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; +use risingwave_pb::meta::table_fragments::PbState; use risingwave_pb::meta::*; use tonic::{Request, Response, Status}; use crate::barrier::{BarrierScheduler, Command}; -use crate::manager::{CatalogManagerRef, FragmentManagerRef, MetaSrvEnv}; +use crate::manager::MetaSrvEnv; use crate::stream::GlobalStreamManagerRef; pub type TonicResponse = Result, Status>; @@ -38,8 +42,7 @@ pub struct StreamServiceImpl { env: MetaSrvEnv, barrier_scheduler: BarrierScheduler, stream_manager: GlobalStreamManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, } impl StreamServiceImpl { @@ -47,15 +50,13 @@ impl StreamServiceImpl { env: MetaSrvEnv, barrier_scheduler: BarrierScheduler, stream_manager: GlobalStreamManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, ) -> Self { StreamServiceImpl { env, barrier_scheduler, stream_manager, - catalog_manager, - fragment_manager, + metadata_manager, } } } @@ -104,14 +105,18 @@ impl StreamManagerService for StreamServiceImpl { request: Request, ) -> Result, Status> { let request = request.into_inner(); + let MetadataManager::V1(mgr) = &self.metadata_manager else { + return Err(Status::unimplemented("not supported in v2")); + }; + let actor_to_apply = match request.kind() { ThrottleTarget::Source => { - self.fragment_manager + mgr.fragment_manager .update_source_rate_limit_by_source_id(request.id as SourceId, request.rate) .await? } ThrottleTarget::Mv => { - self.fragment_manager + mgr.fragment_manager .update_mv_rate_limit_by_table_id(TableId::from(request.id), request.rate) .await? } @@ -146,11 +151,20 @@ impl StreamManagerService for StreamServiceImpl { ) -> TonicResponse { let req = request.into_inner(); let table_ids = match req.jobs.unwrap() { - Jobs::Infos(infos) => { - self.catalog_manager + Jobs::Infos(infos) => match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .find_creating_streaming_job_ids(infos.infos) + .await + } + MetadataManager::V2(mgr) => mgr + .catalog_controller .find_creating_streaming_job_ids(infos.infos) - .await - } + .await? + .into_iter() + .map(|id| id as _) + .collect(), + }, Jobs::Ids(jobs) => jobs.job_ids, }; @@ -174,36 +188,72 @@ impl StreamManagerService for StreamServiceImpl { ) -> Result, Status> { let req = request.into_inner(); let table_ids = HashSet::::from_iter(req.table_ids); - let core = self.fragment_manager.get_fragment_read_guard().await; - let info = core - .table_fragments() - .values() - .filter(|tf| table_ids.contains(&tf.table_id().table_id)) - .map(|tf| { - ( - tf.table_id().table_id, - TableFragmentInfo { - fragments: tf - .fragments - .iter() - .map(|(&id, fragment)| FragmentInfo { - id, - actors: fragment - .actors + + let info = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + core.table_fragments() + .values() + .filter(|tf| table_ids.contains(&tf.table_id().table_id)) + .map(|tf| { + ( + tf.table_id().table_id, + TableFragmentInfo { + fragments: tf + .fragments .iter() - .map(|actor| ActorInfo { - id: actor.actor_id, - node: actor.nodes.clone(), - dispatcher: actor.dispatcher.clone(), + .map(|(&id, fragment)| FragmentInfo { + id, + actors: fragment + .actors + .iter() + .map(|actor| ActorInfo { + id: actor.actor_id, + node: actor.nodes.clone(), + dispatcher: actor.dispatcher.clone(), + }) + .collect_vec(), }) .collect_vec(), - }) - .collect_vec(), - ctx: Some(tf.ctx.to_protobuf()), - }, - ) - }) - .collect::>(); + ctx: Some(tf.ctx.to_protobuf()), + }, + ) + }) + .collect() + } + MetadataManager::V2(mgr) => { + let mut info = HashMap::new(); + for job_id in table_ids { + let pb_table_fragments = mgr + .catalog_controller + .get_job_fragments_by_id(job_id as _) + .await?; + info.insert( + pb_table_fragments.table_id, + TableFragmentInfo { + fragments: pb_table_fragments + .fragments + .into_iter() + .map(|(id, fragment)| FragmentInfo { + id, + actors: fragment + .actors + .into_iter() + .map(|actor| ActorInfo { + id: actor.actor_id, + node: actor.nodes, + dispatcher: actor.dispatcher, + }) + .collect_vec(), + }) + .collect_vec(), + ctx: pb_table_fragments.ctx, + }, + ); + } + info + } + }; Ok(Response::new(ListTableFragmentsResponse { table_fragments: info, @@ -215,21 +265,36 @@ impl StreamManagerService for StreamServiceImpl { &self, _request: Request, ) -> Result, Status> { - let core = self.fragment_manager.get_fragment_read_guard().await; + let states = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + core.table_fragments() + .values() + .map( + |tf| list_table_fragment_states_response::TableFragmentState { + table_id: tf.table_id().table_id, + state: tf.state() as i32, + parallelism: Some(tf.assigned_parallelism.into()), + }, + ) + .collect_vec() + } + MetadataManager::V2(mgr) => { + let job_states = mgr.catalog_controller.list_streaming_job_states().await?; + job_states + .into_iter() + .map(|(table_id, state)| { + list_table_fragment_states_response::TableFragmentState { + table_id: table_id as _, + state: PbState::from(state) as _, + parallelism: None, // TODO: support parallelism. + } + }) + .collect_vec() + } + }; - Ok(Response::new(ListTableFragmentStatesResponse { - states: core - .table_fragments() - .values() - .map( - |tf| list_table_fragment_states_response::TableFragmentState { - table_id: tf.table_id().table_id, - state: tf.state() as i32, - parallelism: Some(tf.assigned_parallelism.into()), - }, - ) - .collect_vec(), - })) + Ok(Response::new(ListTableFragmentStatesResponse { states })) } #[cfg_attr(coverage, coverage(off))] @@ -237,27 +302,52 @@ impl StreamManagerService for StreamServiceImpl { &self, _request: Request, ) -> Result, Status> { - let core = self.fragment_manager.get_fragment_read_guard().await; - - Ok(Response::new(ListFragmentDistributionResponse { - distributions: core - .table_fragments() - .values() - .flat_map(|tf| { - let table_id = tf.table_id().table_id; - tf.fragments.iter().map(move |(&fragment_id, fragment)| { + let distributions = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + core.table_fragments() + .values() + .flat_map(|tf| { + let table_id = tf.table_id().table_id; + tf.fragments.iter().map(move |(&fragment_id, fragment)| { + list_fragment_distribution_response::FragmentDistribution { + fragment_id, + table_id, + distribution_type: fragment.distribution_type, + state_table_ids: fragment.state_table_ids.clone(), + upstream_fragment_ids: fragment.upstream_fragment_ids.clone(), + fragment_type_mask: fragment.fragment_type_mask, + parallelism: fragment.actors.len() as _, + } + }) + }) + .collect_vec() + } + MetadataManager::V2(mgr) => { + let fragment_descs = mgr.catalog_controller.list_fragment_descs().await?; + fragment_descs + .into_iter() + .map(|fragment_desc| { list_fragment_distribution_response::FragmentDistribution { - fragment_id, - table_id, - distribution_type: fragment.distribution_type, - state_table_ids: fragment.state_table_ids.clone(), - upstream_fragment_ids: fragment.upstream_fragment_ids.clone(), - fragment_type_mask: fragment.fragment_type_mask, - parallelism: fragment.actors.len() as _, + fragment_id: fragment_desc.fragment_id as _, + table_id: fragment_desc.job_id as _, + distribution_type: PbFragmentDistributionType::from( + fragment_desc.distribution_type, + ) as _, + state_table_ids: fragment_desc.state_table_ids.into_u32_array(), + upstream_fragment_ids: fragment_desc + .upstream_fragment_id + .into_u32_array(), + fragment_type_mask: fragment_desc.fragment_type_mask as _, + parallelism: fragment_desc.parallelism as _, } }) - }) - .collect_vec(), + .collect_vec() + } + }; + + Ok(Response::new(ListFragmentDistributionResponse { + distributions, })) } @@ -266,24 +356,38 @@ impl StreamManagerService for StreamServiceImpl { &self, _request: Request, ) -> Result, Status> { - let core = self.fragment_manager.get_fragment_read_guard().await; - - Ok(Response::new(ListActorStatesResponse { - states: core - .table_fragments() - .values() - .flat_map(|tf| { - let actor_to_fragment = tf.actor_fragment_mapping(); - tf.actor_status.iter().map(move |(&actor_id, status)| { - list_actor_states_response::ActorState { - actor_id, - fragment_id: actor_to_fragment[&actor_id], - state: status.state, - parallel_unit_id: status.parallel_unit.as_ref().unwrap().id, - } + let states = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let core = mgr.fragment_manager.get_fragment_read_guard().await; + core.table_fragments() + .values() + .flat_map(|tf| { + let actor_to_fragment = tf.actor_fragment_mapping(); + tf.actor_status.iter().map(move |(&actor_id, status)| { + list_actor_states_response::ActorState { + actor_id, + fragment_id: actor_to_fragment[&actor_id], + state: status.state, + parallel_unit_id: status.parallel_unit.as_ref().unwrap().id, + } + }) }) - }) - .collect_vec(), - })) + .collect_vec() + } + MetadataManager::V2(mgr) => { + let actor_locations = mgr.catalog_controller.list_actor_locations().await?; + actor_locations + .into_iter() + .map(|actor_location| list_actor_states_response::ActorState { + actor_id: actor_location.actor_id as _, + fragment_id: actor_location.fragment_id as _, + state: PbActorState::from(actor_location.status) as _, + parallel_unit_id: actor_location.parallel_unit_id as _, + }) + .collect_vec() + } + }; + + Ok(Response::new(ListActorStatesResponse { states })) } } diff --git a/src/meta/service/src/user_service.rs b/src/meta/service/src/user_service.rs index e8817394b6efe..5d47d97ab9780 100644 --- a/src/meta/service/src/user_service.rs +++ b/src/meta/service/src/user_service.rs @@ -13,6 +13,8 @@ // limitations under the License. use itertools::Itertools; +use risingwave_meta::manager::MetadataManager; +use risingwave_meta_model_v2::UserId; use risingwave_pb::user::grant_privilege::Object; use risingwave_pb::user::update_user_request::UpdateField; use risingwave_pb::user::user_service_server::UserService; @@ -23,20 +25,19 @@ use risingwave_pb::user::{ }; use tonic::{Request, Response, Status}; -use crate::manager::{CatalogManagerRef, IdCategory, MetaSrvEnv}; +use crate::manager::{IdCategory, MetaSrvEnv}; use crate::MetaResult; pub struct UserServiceImpl { env: MetaSrvEnv, - - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, } impl UserServiceImpl { - pub fn new(env: MetaSrvEnv, catalog_manager: CatalogManagerRef) -> Self { + pub fn new(env: MetaSrvEnv, metadata_manager: MetadataManager) -> Self { Self { env, - catalog_manager, + metadata_manager, } } @@ -50,10 +51,20 @@ impl UserServiceImpl { let mut expanded_privileges = Vec::new(); for privilege in privileges { if let Some(Object::AllTablesSchemaId(schema_id)) = &privilege.object { - let tables = self - .catalog_manager - .list_readonly_table_ids(*schema_id) - .await; + let tables = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .list_readonly_table_ids(*schema_id) + .await + } + MetadataManager::V2(mgr) => mgr + .catalog_controller + .list_readonly_table_ids(*schema_id as _) + .await? + .into_iter() + .map(|id| id as _) + .collect(), + }; for table_id in tables { let mut privilege = privilege.clone(); privilege.object = Some(Object::TableId(table_id)); @@ -66,7 +77,18 @@ impl UserServiceImpl { expanded_privileges.push(privilege); } } else if let Some(Object::AllDmlTablesSchemaId(schema_id)) = &privilege.object { - let tables = self.catalog_manager.list_dml_table_ids(*schema_id).await; + let tables = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager.list_dml_table_ids(*schema_id).await + } + MetadataManager::V2(mgr) => mgr + .catalog_controller + .list_dml_table_ids(*schema_id as _) + .await? + .into_iter() + .map(|id| id as _) + .collect(), + }; for table_id in tables { let mut privilege = privilege.clone(); privilege.object = Some(Object::TableId(table_id)); @@ -78,8 +100,19 @@ impl UserServiceImpl { } expanded_privileges.push(privilege); } - } else if let Some(Object::AllSourcesSchemaId(source_id)) = &privilege.object { - let sources = self.catalog_manager.list_source_ids(*source_id).await; + } else if let Some(Object::AllSourcesSchemaId(schema_id)) = &privilege.object { + let sources = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager.list_source_ids(*schema_id).await + } + MetadataManager::V2(mgr) => mgr + .catalog_controller + .list_source_ids(*schema_id as _) + .await? + .into_iter() + .map(|id| id as _) + .collect(), + }; for source_id in sources { let mut privilege = privilege.clone(); privilege.object = Some(Object::SourceId(source_id)); @@ -113,14 +146,23 @@ impl UserService for UserServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let id = self - .env - .id_gen_manager() - .generate::<{ IdCategory::User }>() - .await? as u32; - let mut user = req.get_user()?.clone(); - user.id = id; - let version = self.catalog_manager.create_user(&user).await?; + let version = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let id = self + .env + .id_gen_manager() + .generate::<{ IdCategory::User }>() + .await? as u32; + let mut user = req.get_user()?.clone(); + user.id = id; + mgr.catalog_manager.create_user(&user).await? + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .create_user(req.get_user()?.clone()) + .await? + } + }; Ok(Response::new(CreateUserResponse { status: None, @@ -134,7 +176,10 @@ impl UserService for UserServiceImpl { request: Request, ) -> Result, Status> { let req = request.into_inner(); - let version = self.catalog_manager.drop_user(req.user_id).await?; + let version = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.drop_user(req.user_id).await?, + MetadataManager::V2(mgr) => mgr.catalog_controller.drop_user(req.user_id as _).await?, + }; Ok(Response::new(DropUserResponse { status: None, @@ -154,10 +199,19 @@ impl UserService for UserServiceImpl { .map(|i| UpdateField::try_from(*i).unwrap()) .collect_vec(); let user = req.get_user()?.clone(); - let version = self - .catalog_manager - .update_user(&user, &update_fields) - .await?; + + let version = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .update_user(&user, &update_fields) + .await? + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .update_user(user, &update_fields) + .await? + } + }; Ok(Response::new(UpdateUserResponse { status: None, @@ -174,10 +228,19 @@ impl UserService for UserServiceImpl { let new_privileges = self .expand_privilege(req.get_privileges(), Some(req.with_grant_option)) .await?; - let version = self - .catalog_manager - .grant_privilege(&req.user_ids, &new_privileges, req.granted_by) - .await?; + let version = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .grant_privilege(&req.user_ids, &new_privileges, req.granted_by) + .await? + } + MetadataManager::V2(mgr) => { + let user_ids: Vec<_> = req.get_user_ids().iter().map(|id| *id as UserId).collect(); + mgr.catalog_controller + .grant_privilege(user_ids, &new_privileges, req.granted_by as _) + .await? + } + }; Ok(Response::new(GrantPrivilegeResponse { status: None, @@ -192,17 +255,33 @@ impl UserService for UserServiceImpl { ) -> Result, Status> { let req = request.into_inner(); let privileges = self.expand_privilege(req.get_privileges(), None).await?; - let version = self - .catalog_manager - .revoke_privilege( - &req.user_ids, - &privileges, - req.granted_by, - req.revoke_by, - req.revoke_grant_option, - req.cascade, - ) - .await?; + let version = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .revoke_privilege( + &req.user_ids, + &privileges, + req.granted_by, + req.revoke_by, + req.revoke_grant_option, + req.cascade, + ) + .await? + } + MetadataManager::V2(mgr) => { + let user_ids: Vec<_> = req.get_user_ids().iter().map(|id| *id as UserId).collect(); + mgr.catalog_controller + .revoke_privilege( + user_ids, + &privileges, + Some(req.granted_by as _), + req.revoke_by as _, + req.revoke_grant_option, + req.cascade, + ) + .await? + } + }; Ok(Response::new(RevokePrivilegeResponse { status: None, diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 42d16a436b3b9..37bf9036c8482 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; use futures::future::try_join_all; @@ -40,7 +40,7 @@ use super::info::BarrierActorInfo; use super::trace::TracedEpoch; use crate::barrier::CommandChanges; use crate::hummock::HummockManagerRef; -use crate::manager::{CatalogManagerRef, DdlType, FragmentManagerRef, WorkerId}; +use crate::manager::{DdlType, MetadataManager, WorkerId}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::stream::{ build_actor_connector_splits, ScaleControllerRef, SourceManagerRef, SplitAssignment, @@ -115,6 +115,14 @@ pub enum Command { /// It does not include internal table ids. DropStreamingJobs(HashSet), + /// `DropStreamingJobsV2` command generates a `Stop` barrier by the given actor info. + /// This is used by new SQL metastore and catalog has already been dropped. + /// + /// Barriers from the actors to be dropped will STILL be collected. + /// After the barrier is collected, it notifies the local stream manager of compute nodes to + /// drop actors. + DropStreamingJobsV2(HashMap>>), + /// `CreateStreamingJob` command generates a `Add` barrier by given info. /// /// Barriers from the actors to be created, which is marked as `Inactive` at first, will STILL @@ -208,6 +216,9 @@ impl Command { table_fragments, .. } => CommandChanges::CreateTable(table_fragments.table_id()), Command::DropStreamingJobs(table_ids) => CommandChanges::DropTables(table_ids.clone()), + Command::DropStreamingJobsV2(job_info) => { + CommandChanges::DropTables(job_info.keys().cloned().collect()) + } Command::CancelStreamingJob(table_fragments) => { CommandChanges::DropTables(std::iter::once(table_fragments.table_id()).collect()) } @@ -255,8 +266,8 @@ impl Command { /// [`CommandContext`] is used for generating barrier and doing post stuffs according to the given /// [`Command`]. pub struct CommandContext { - pub fragment_manager: FragmentManagerRef, - catalog_manager: CatalogManagerRef, + pub metadata_manager: MetadataManager, + hummock_manager: HummockManagerRef, client_pool: StreamClientPoolRef, @@ -276,7 +287,7 @@ pub struct CommandContext { source_manager: SourceManagerRef, - scale_controller: ScaleControllerRef, + scale_controller: Option, /// The tracing span of this command. /// @@ -289,8 +300,7 @@ pub struct CommandContext { impl CommandContext { #[allow(clippy::too_many_arguments)] pub(super) fn new( - fragment_manager: FragmentManagerRef, - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, client_pool: StreamClientPoolRef, info: BarrierActorInfo, @@ -300,12 +310,11 @@ impl CommandContext { command: Command, kind: BarrierKind, source_manager: SourceManagerRef, - scale_controller: ScaleControllerRef, + scale_controller: Option, span: tracing::Span, ) -> Self { Self { - fragment_manager, - catalog_manager, + metadata_manager, hummock_manager, client_pool, info: Arc::new(info), @@ -373,7 +382,21 @@ impl CommandContext { } Command::DropStreamingJobs(table_ids) => { - let actors = self.fragment_manager.get_table_actor_ids(table_ids).await?; + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!("only available in v1"); + }; + + let actors = mgr.fragment_manager.get_table_actor_ids(table_ids).await?; + Some(Mutation::Stop(StopMutation { actors })) + } + + Command::DropStreamingJobsV2(job_info) => { + let actors: Vec = job_info + .values() + .flat_map(|v| v.values()) + .flatten() + .copied() + .collect(); Some(Mutation::Stop(StopMutation { actors })) } @@ -416,6 +439,7 @@ impl CommandContext { init_split_assignment, }) = replace_table { + // TODO: support in v2. let update = Self::generate_update_mutation_for_replace_table( old_table_fragments, merge_updates, @@ -453,13 +477,16 @@ impl CommandContext { ), Command::RescheduleFragment { reschedules, .. } => { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("implement scale functions in v2"); + }; let mut dispatcher_update = HashMap::new(); for reschedule in reschedules.values() { for &(upstream_fragment_id, dispatcher_id) in &reschedule.upstream_fragment_dispatcher_ids { // Find the actors of the upstream fragment. - let upstream_actor_ids = self + let upstream_actor_ids = mgr .fragment_manager .get_running_actors_of_fragment(upstream_fragment_id) .await?; @@ -493,7 +520,7 @@ impl CommandContext { for (&fragment_id, reschedule) in reschedules { for &downstream_fragment_id in &reschedule.downstream_fragment_ids { // Find the actors of the downstream fragment. - let downstream_actor_ids = self + let downstream_actor_ids = mgr .fragment_manager .get_running_actors_of_fragment(downstream_fragment_id) .await?; @@ -755,7 +782,10 @@ impl CommandContext { Command::Resume(_) => {} Command::SourceSplitAssignment(split_assignment) => { - self.fragment_manager + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("implement config change funcs in v2"); + }; + mgr.fragment_manager .update_actor_splits_by_split_assignment(split_assignment) .await?; self.source_manager @@ -764,15 +794,29 @@ impl CommandContext { } Command::DropStreamingJobs(table_ids) => { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!("only available in v1"); + }; // Tell compute nodes to drop actors. - let node_actors = self.fragment_manager.table_node_actors(table_ids).await?; + let node_actors = mgr.fragment_manager.table_node_actors(table_ids).await?; self.clean_up(node_actors).await?; // Drop fragment info in meta store. - self.fragment_manager + mgr.fragment_manager .drop_table_fragments_vec(table_ids) .await?; } + Command::DropStreamingJobsV2(job_info) => { + let mut node_actors: BTreeMap> = BTreeMap::new(); + for worker_actor_ids in job_info.values() { + for (worker_id, actor_ids) in worker_actor_ids { + node_actors.entry(*worker_id).or_default().extend(actor_ids); + } + } + // Tell compute nodes to drop actors. + self.clean_up(node_actors).await?; + } + Command::CancelStreamingJob(table_fragments) => { tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); let node_actors = table_fragments.worker_actor_ids(); @@ -794,39 +838,46 @@ impl CommandContext { .unregister_table_ids_fail_fast(&table_ids) .await; - // NOTE(kwannoel): At this point, catalog manager has persisted the tables already. - // We need to cleanup the table state. So we can do it here. - // The logic is the same as above, for hummock_manager.unregister_table_ids. - if let Err(e) = self - .catalog_manager - .cancel_create_table_procedure( - table_fragments.table_id().table_id, - table_fragments.internal_table_ids(), - ) - .await - { - let table_id = table_fragments.table_id().table_id; - tracing::warn!( - table_id, - reason=?e, - "cancel_create_table_procedure failed for CancelStreamingJob", - ); - // If failed, check that table is not in meta store. - // If any table is, just panic, let meta do bootstrap recovery. - // Otherwise our persisted state is dirty. - let mut table_ids = table_fragments.internal_table_ids(); - table_ids.push(table_id); - self.catalog_manager.assert_tables_deleted(table_ids).await; - } + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + // NOTE(kwannoel): At this point, catalog manager has persisted the tables already. + // We need to cleanup the table state. So we can do it here. + // The logic is the same as above, for hummock_manager.unregister_table_ids. + if let Err(e) = mgr + .catalog_manager + .cancel_create_table_procedure( + table_fragments.table_id().table_id, + table_fragments.internal_table_ids(), + ) + .await + { + let table_id = table_fragments.table_id().table_id; + tracing::warn!( + table_id, + reason=?e, + "cancel_create_table_procedure failed for CancelStreamingJob", + ); + // If failed, check that table is not in meta store. + // If any table is, just panic, let meta do bootstrap recovery. + // Otherwise our persisted state is dirty. + let mut table_ids = table_fragments.internal_table_ids(); + table_ids.push(table_id); + mgr.catalog_manager.assert_tables_deleted(table_ids).await; + } - // We need to drop table fragments here, - // since this is not done in stream manager (foreground ddl) - // OR barrier manager (background ddl) - self.fragment_manager - .drop_table_fragments_vec(&HashSet::from_iter(std::iter::once( - table_fragments.table_id(), - ))) - .await?; + // We need to drop table fragments here, + // since this is not done in stream manager (foreground ddl) + // OR barrier manager (background ddl) + mgr.fragment_manager + .drop_table_fragments_vec(&HashSet::from_iter(std::iter::once( + table_fragments.table_id(), + ))) + .await?; + } + MetadataManager::V2(_mgr) => { + unimplemented!("implement cancel for sql backend") + } + } } Command::CreateStreamingJob { @@ -838,46 +889,55 @@ impl CommandContext { replace_table, .. } => { - let mut dependent_table_actors = Vec::with_capacity(upstream_mview_actors.len()); - for (table_id, actors) in upstream_mview_actors { - let downstream_actors = dispatchers - .iter() - .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id)) - .map(|(&k, v)| (k, v.clone())) - .collect(); - dependent_table_actors.push((*table_id, downstream_actors)); - } - self.fragment_manager - .post_create_table_fragments( - &table_fragments.table_id(), - dependent_table_actors, - init_split_assignment.clone(), - ) - .await?; + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let mut dependent_table_actors = + Vec::with_capacity(upstream_mview_actors.len()); + for (table_id, actors) in upstream_mview_actors { + let downstream_actors = dispatchers + .iter() + .filter(|(upstream_actor_id, _)| actors.contains(upstream_actor_id)) + .map(|(&k, v)| (k, v.clone())) + .collect(); + dependent_table_actors.push((*table_id, downstream_actors)); + } + mgr.fragment_manager + .post_create_table_fragments( + &table_fragments.table_id(), + dependent_table_actors, + init_split_assignment.clone(), + ) + .await?; - if let Some(ReplaceTablePlan { - old_table_fragments, - new_table_fragments, - merge_updates, - dispatchers, - init_split_assignment: _, - }) = replace_table - { - let table_ids = - HashSet::from_iter(std::iter::once(old_table_fragments.table_id())); - // Tell compute nodes to drop actors. - let node_actors = self.fragment_manager.table_node_actors(&table_ids).await?; - self.clean_up(node_actors).await?; - - // Drop fragment info in meta store. - self.fragment_manager - .post_replace_table( + if let Some(ReplaceTablePlan { old_table_fragments, new_table_fragments, merge_updates, dispatchers, - ) - .await?; + init_split_assignment: _, + }) = replace_table + { + let table_ids = + HashSet::from_iter(std::iter::once(old_table_fragments.table_id())); + // Tell compute nodes to drop actors. + let node_actors = + mgr.fragment_manager.table_node_actors(&table_ids).await?; + self.clean_up(node_actors).await?; + + // Drop fragment info in meta store. + mgr.fragment_manager + .post_replace_table( + old_table_fragments, + new_table_fragments, + merge_updates, + dispatchers, + ) + .await?; + } + } + MetadataManager::V2(_) => { + unimplemented!("support post collect in v2"); + } } // Extract the fragments that include source operators. @@ -898,6 +958,8 @@ impl CommandContext { } => { let node_dropped_actors = self .scale_controller + .as_ref() + .unwrap() .post_apply_reschedule(reschedules, table_parallelism) .await?; self.clean_up(node_dropped_actors).await?; @@ -910,14 +972,17 @@ impl CommandContext { dispatchers, .. }) => { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("implement replace funcs in v2"); + }; let table_ids = HashSet::from_iter(std::iter::once(old_table_fragments.table_id())); // Tell compute nodes to drop actors. - let node_actors = self.fragment_manager.table_node_actors(&table_ids).await?; + let node_actors = mgr.fragment_manager.table_node_actors(&table_ids).await?; self.clean_up(node_actors).await?; // Drop fragment info in meta store. - self.fragment_manager + mgr.fragment_manager .post_replace_table( old_table_fragments, new_table_fragments, diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 9018673f11e48..854de53f37708 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -32,6 +32,7 @@ use risingwave_hummock_sdk::table_watermark::{ merge_multiple_new_table_watermarks, TableWatermarks, }; use risingwave_hummock_sdk::{ExtendedSstableInfo, HummockSstableObjectId}; +use risingwave_meta_model_v2::ObjectId; use risingwave_pb::catalog::table::TableType; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -59,10 +60,7 @@ use crate::barrier::progress::{CreateMviewProgressTracker, TrackingJob}; use crate::barrier::BarrierEpochState::{Completed, InFlight}; use crate::hummock::{CommitEpochInfo, HummockManagerRef}; use crate::manager::sink_coordination::SinkCoordinatorManager; -use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, LocalNotification, MetaSrvEnv, - WorkerId, -}; +use crate::manager::{LocalNotification, MetaSrvEnv, MetadataManager, WorkerId}; use crate::model::{ActorId, BarrierManagerState, TableFragments}; use crate::rpc::metrics::MetaMetrics; use crate::stream::{ScaleController, ScaleControllerRef, SourceManagerRef}; @@ -185,17 +183,13 @@ pub struct GlobalBarrierManager { /// The max barrier nums in flight in_flight_barrier_nums: usize, - cluster_manager: ClusterManagerRef, - - pub catalog_manager: CatalogManagerRef, - - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, source_manager: SourceManagerRef, - scale_controller: ScaleControllerRef, + scale_controller: Option, sink_manager: SinkCoordinatorManager, @@ -563,9 +557,7 @@ impl GlobalBarrierManager { pub fn new( scheduled_barriers: schedule::ScheduledBarriers, env: MetaSrvEnv, - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, source_manager: SourceManagerRef, sink_manager: SinkCoordinatorManager, @@ -575,20 +567,21 @@ impl GlobalBarrierManager { let in_flight_barrier_nums = env.opts.in_flight_barrier_nums; let tracker = CreateMviewProgressTracker::new(); - let scale_controller = Arc::new(ScaleController::new( - fragment_manager.clone(), - cluster_manager.clone(), - source_manager.clone(), - env.clone(), - )); + + let scale_controller = match &metadata_manager { + MetadataManager::V1(_) => Some(Arc::new(ScaleController::new( + &metadata_manager, + source_manager.clone(), + env.clone(), + ))), + MetadataManager::V2(_) => None, + }; Self { enable_recovery, status: Mutex::new(BarrierManagerStatus::Starting), scheduled_barriers, in_flight_barrier_nums, - cluster_manager, - catalog_manager, - fragment_manager, + metadata_manager, hummock_manager, source_manager, scale_controller, @@ -631,8 +624,11 @@ impl GlobalBarrierManager { /// Check whether we should pause on bootstrap from the system parameter and reset it. async fn take_pause_on_bootstrap(&self) -> MetaResult { - let pm = self.env.system_params_manager(); - let paused = pm.get_params().await.pause_on_next_bootstrap(); + let paused = self + .env + .system_params_reader() + .await + .pause_on_next_bootstrap(); if paused { tracing::warn!( "The cluster will bootstrap with all data sources paused as specified by the system parameter `{}`. \ @@ -640,8 +636,16 @@ impl GlobalBarrierManager { To resume the data sources, either restart the cluster again or use `risectl meta resume`.", PAUSE_ON_NEXT_BOOTSTRAP_KEY ); - pm.set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned())) - .await?; + if let Some(system_ctl) = self.env.system_params_controller() { + system_ctl + .set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned())) + .await?; + } else { + self.env + .system_params_manager() + .set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned())) + .await?; + } } Ok(paused) } @@ -650,11 +654,7 @@ impl GlobalBarrierManager { async fn run(&self, mut shutdown_rx: Receiver<()>) { // Initialize the barrier manager. let interval = Duration::from_millis( - self.env - .system_params_manager() - .get_params() - .await - .barrier_interval_ms() as u64, + self.env.system_params_reader().await.barrier_interval_ms() as u64, ); tracing::info!( "Starting barrier manager with: interval={:?}, enable_recovery={}, in_flight_barrier_nums={}", @@ -663,11 +663,21 @@ impl GlobalBarrierManager { self.in_flight_barrier_nums, ); - if !self.enable_recovery && self.fragment_manager.has_any_table_fragments().await { - panic!( - "Some streaming jobs already exist in meta, please start with recovery enabled \ + if !self.enable_recovery { + let job_exist = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.fragment_manager.has_any_table_fragments().await, + MetadataManager::V2(mgr) => mgr + .catalog_controller + .has_any_streaming_jobs() + .await + .unwrap(), + }; + if job_exist { + panic!( + "Some streaming jobs already exist in meta, please start with recovery enabled \ or clean up the metadata using `./risedev clean-data`" - ); + ); + } } let mut state = { @@ -785,8 +795,7 @@ impl GlobalBarrierManager { span.record("epoch", curr_epoch.value().0); let command_ctx = Arc::new(CommandContext::new( - self.fragment_manager.clone(), - self.catalog_manager.clone(), + self.metadata_manager.clone(), self.hummock_manager.clone(), self.env.stream_client_pool_ref(), info, @@ -1234,16 +1243,45 @@ impl GlobalBarrierManager { ) -> BarrierActorInfo { checkpoint_control.pre_resolve(command); - let check_state = |s: ActorState, table_id: TableId, actor_id: ActorId| { - checkpoint_control.can_actor_send_or_collect(s, table_id, actor_id) - }; - let all_nodes = self - .cluster_manager - .list_active_streaming_compute_nodes() - .await; - let all_actor_infos = self.fragment_manager.load_all_actors(check_state).await; + let info = match &self.metadata_manager { + MetadataManager::V1(mgr) => { + let check_state = |s: ActorState, table_id: TableId, actor_id: ActorId| { + checkpoint_control.can_actor_send_or_collect(s, table_id, actor_id) + }; + let all_nodes = mgr + .cluster_manager + .list_active_streaming_compute_nodes() + .await; + let all_actor_infos = mgr.fragment_manager.load_all_actors(check_state).await; - let info = BarrierActorInfo::resolve(all_nodes, all_actor_infos); + BarrierActorInfo::resolve(all_nodes, all_actor_infos) + } + MetadataManager::V2(mgr) => { + let check_state = |s: ActorState, table_id: ObjectId, actor_id: i32| { + checkpoint_control.can_actor_send_or_collect( + s, + TableId::new(table_id as _), + actor_id as _, + ) + }; + let all_nodes = mgr + .cluster_controller + .list_active_streaming_workers() + .await + .unwrap(); + let pu_mappings = all_nodes + .iter() + .flat_map(|node| node.parallel_units.iter().map(|pu| (pu.id, pu.clone()))) + .collect(); + let all_actor_infos = mgr + .catalog_controller + .load_all_actors(&pu_mappings, check_state) + .await + .unwrap(); + + BarrierActorInfo::resolve(all_nodes, all_actor_infos) + } + }; checkpoint_control.post_resolve(command); @@ -1254,18 +1292,39 @@ impl GlobalBarrierManager { let mut ddl_progress = self.tracker.lock().await.gen_ddl_progress(); // If not in tracker, means the first barrier not collected yet. // In that case just return progress 0. - for table in self.catalog_manager.list_persisted_creating_tables().await { - if table.table_type != TableType::MaterializedView as i32 { - continue; + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + for table in mgr.catalog_manager.list_persisted_creating_tables().await { + if table.table_type != TableType::MaterializedView as i32 { + continue; + } + if let Entry::Vacant(e) = ddl_progress.entry(table.id) { + e.insert(DdlProgress { + id: table.id as u64, + statement: table.definition, + progress: "0.0%".into(), + }); + } + } } - if let Entry::Vacant(e) = ddl_progress.entry(table.id) { - e.insert(DdlProgress { - id: table.id as u64, - statement: table.definition, - progress: "0.0%".into(), - }); + MetadataManager::V2(mgr) => { + let mviews = mgr + .catalog_controller + .list_background_creating_mviews() + .await + .unwrap(); + for mview in mviews { + if let Entry::Vacant(e) = ddl_progress.entry(mview.table_id as _) { + e.insert(DdlProgress { + id: mview.table_id as u64, + statement: mview.definition, + progress: "0.0%".into(), + }); + } + } } } + ddl_progress.into_values().collect() } } diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index c6b3921077530..fede60d5eb939 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -29,7 +29,7 @@ use crate::barrier::{ Command, TableActorMap, TableDefinitionMap, TableFragmentMap, TableNotifierMap, TableUpstreamMvCountMap, }; -use crate::manager::{DdlType, FragmentManager, FragmentManagerRef}; +use crate::manager::{DdlType, MetadataManager}; use crate::model::{ActorId, TableFragments}; use crate::MetaResult; @@ -159,10 +159,10 @@ pub enum TrackingJob { } impl TrackingJob { - fn fragment_manager(&self) -> &FragmentManager { + fn metadata_manager(&self) -> &MetadataManager { match self { - TrackingJob::New(command) => command.context.fragment_manager.as_ref(), - TrackingJob::Recovered(recovered) => recovered.fragment_manager.as_ref(), + TrackingJob::New(command) => &command.context.metadata_manager, + TrackingJob::Recovered(recovered) => &recovered.metadata_manager, } } @@ -191,9 +191,14 @@ impl TrackingJob { // Update the state of the table fragments from `Creating` to `Created`, so that the // fragments can be scaled. if let Some(table_fragments) = table_fragments { - self.fragment_manager() - .mark_table_fragments_created(table_fragments.table_id()) - .await?; + match self.metadata_manager() { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .mark_table_fragments_created(table_fragments.table_id()) + .await?; + } + MetadataManager::V2(_) => {} + } } Ok(()) } @@ -230,7 +235,7 @@ impl TrackingJob { pub struct RecoveredTrackingJob { pub fragments: TableFragments, pub finished: Notifier, - pub fragment_manager: FragmentManagerRef, + pub metadata_manager: MetadataManager, } /// The command tracking by the [`CreateMviewProgressTracker`]. @@ -282,7 +287,7 @@ impl CreateMviewProgressTracker { version_stats: HummockVersionStats, mut finished_notifiers: TableNotifierMap, mut table_fragment_map: TableFragmentMap, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, ) -> Self { let mut actor_map = HashMap::new(); let mut progress_map = HashMap::new(); @@ -317,7 +322,7 @@ impl CreateMviewProgressTracker { let tracking_job = TrackingJob::Recovered(RecoveredTrackingJob { fragments: table_fragment_map.remove(&creating_table_id).unwrap(), finished: finished_notifiers.remove(&creating_table_id).unwrap(), - fragment_manager: fragment_manager.clone(), + metadata_manager: metadata_manager.clone(), }); progress_map.insert(creating_table_id, (progress, tracking_job)); } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 2350e2c4eedf0..53fc94e7c7c5e 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -41,7 +41,7 @@ use crate::barrier::info::BarrierActorInfo; use crate::barrier::notifier::Notifier; use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{CheckpointControl, Command, GlobalBarrierManager}; -use crate::manager::WorkerId; +use crate::manager::{MetadataManager, WorkerId}; use crate::model::{BarrierManagerState, MigrationPlan}; use crate::stream::{build_actor_connector_splits, RescheduleOptions, TableResizePolicy}; use crate::MetaResult; @@ -68,57 +68,72 @@ impl GlobalBarrierManager { .await } - /// Please look at `CatalogManager::clean_dirty_tables` for more details. - /// This should only be called for bootstrap recovery. - async fn clean_dirty_tables(&self) -> MetaResult<()> { - let fragment_manager = self.fragment_manager.clone(); - self.catalog_manager - .clean_dirty_tables(fragment_manager) - .await?; - Ok(()) - } - - /// Clean up all dirty streaming jobs. - async fn clean_dirty_fragments(&self) -> MetaResult<()> { - let stream_job_ids = self.catalog_manager.list_stream_job_ids().await?; - let to_drop_table_fragments = self - .fragment_manager - .list_dirty_table_fragments(|tf| !stream_job_ids.contains(&tf.table_id().table_id)) - .await; - let to_drop_streaming_ids = to_drop_table_fragments - .iter() - .map(|t| t.table_id()) - .collect(); - - debug!("clean dirty table fragments: {:?}", to_drop_streaming_ids); + /// Clean catalogs for creating streaming jobs that are in foreground mode or table fragments not persisted. + async fn clean_dirty_streaming_jobs(&self) -> MetaResult<()> { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + // Please look at `CatalogManager::clean_dirty_tables` for more details. + mgr.catalog_manager + .clean_dirty_tables(mgr.fragment_manager.clone()) + .await?; - self.fragment_manager - .drop_table_fragments_vec(&to_drop_streaming_ids) - .await?; + // Clean dirty fragments. + let stream_job_ids = mgr.catalog_manager.list_stream_job_ids().await?; + let to_drop_table_fragments = mgr + .fragment_manager + .list_dirty_table_fragments(|tf| { + !stream_job_ids.contains(&tf.table_id().table_id) + }) + .await; + let to_drop_streaming_ids = to_drop_table_fragments + .iter() + .map(|t| t.table_id()) + .collect(); + debug!("clean dirty table fragments: {:?}", to_drop_streaming_ids); + + mgr.fragment_manager + .drop_table_fragments_vec(&to_drop_streaming_ids) + .await?; - // unregister compaction group for dirty table fragments. - self.hummock_manager - .unregister_table_fragments_vec(&to_drop_table_fragments) - .await; + // unregister compaction group for dirty table fragments. + self.hummock_manager + .unregister_table_fragments_vec(&to_drop_table_fragments) + .await; - // clean up source connector dirty changes. - self.source_manager - .drop_source_fragments(&to_drop_table_fragments) - .await; + // clean up source connector dirty changes. + self.source_manager + .drop_source_fragments(&to_drop_table_fragments) + .await; + } + MetadataManager::V2(_) => { + unimplemented!("support clean dirty streaming jobs in v2"); + } + } Ok(()) } async fn recover_background_mv_progress(&self) -> MetaResult<()> { - let creating_tables = self.catalog_manager.list_creating_background_mvs().await; - let creating_table_ids = creating_tables - .iter() - .map(|t| TableId { table_id: t.id }) - .collect_vec(); + match &self.metadata_manager { + MetadataManager::V1(_) => self.recover_background_mv_progress_v1().await, + MetadataManager::V2(_) => self.recover_background_mv_progress_v2(), + } + } + + async fn recover_background_mv_progress_v1(&self) -> MetaResult<()> { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!() + }; + let mviews = mgr.catalog_manager.list_creating_background_mvs().await; + let creating_mview_ids = mviews.iter().map(|m| TableId::new(m.id)).collect_vec(); + let mview_definitions = mviews + .into_iter() + .map(|m| (TableId::new(m.id), m.definition)) + .collect::>(); let mut senders = HashMap::new(); let mut receivers = Vec::new(); - for table_id in creating_table_ids.iter().copied() { + for table_id in creating_mview_ids.iter().copied() { let (finished_tx, finished_rx) = oneshot::channel(); senders.insert( table_id, @@ -128,34 +143,30 @@ impl GlobalBarrierManager { }, ); - let fragments = self + let fragments = mgr .fragment_manager .select_table_fragments_by_table_id(&table_id) .await?; let internal_table_ids = fragments.internal_table_ids(); - let internal_tables = self.catalog_manager.get_tables(&internal_table_ids).await; - let table = self.catalog_manager.get_tables(&[table_id.table_id]).await; + let internal_tables = mgr.catalog_manager.get_tables(&internal_table_ids).await; + let table = mgr.catalog_manager.get_tables(&[table_id.table_id]).await; assert_eq!(table.len(), 1, "should only have 1 materialized table"); let table = table.into_iter().next().unwrap(); receivers.push((table, internal_tables, finished_rx)); } - let table_map = self + let table_map = mgr .fragment_manager - .get_table_id_stream_scan_actor_mapping(&creating_table_ids) + .get_table_id_stream_scan_actor_mapping(&creating_mview_ids) .await; - let table_fragment_map = self + let table_fragment_map = mgr .fragment_manager - .get_table_id_table_fragment_map(&creating_table_ids) + .get_table_id_table_fragment_map(&creating_mview_ids) .await?; - let upstream_mv_counts = self + let upstream_mv_counts = mgr .fragment_manager - .get_upstream_relation_counts(&creating_table_ids) + .get_upstream_relation_counts(&creating_mview_ids) .await; - let definitions: HashMap<_, _> = creating_tables - .into_iter() - .map(|t| (TableId { table_id: t.id }, t.definition)) - .collect(); let version_stats = self.hummock_manager.get_version_stats().await; // If failed, enter recovery mode. { @@ -163,15 +174,15 @@ impl GlobalBarrierManager { *tracker = CreateMviewProgressTracker::recover( table_map.into(), upstream_mv_counts.into(), - definitions.into(), + mview_definitions.into(), version_stats, senders.into(), table_fragment_map.into(), - self.fragment_manager.clone(), + self.metadata_manager.clone(), ); } for (table, internal_tables, finished) in receivers { - let catalog_manager = self.catalog_manager.clone(); + let catalog_manager = mgr.catalog_manager.clone(); tokio::spawn(async move { let res: MetaResult<()> = try { tracing::debug!("recovering stream job {}", table.id); @@ -204,6 +215,10 @@ impl GlobalBarrierManager { Ok(()) } + fn recover_background_mv_progress_v2(&self) -> MetaResult<()> { + unimplemented!("support recover background mv progress in v2"); + } + /// Recovery the whole cluster from the latest epoch. /// /// If `paused_reason` is `Some`, all data sources (including connectors and DMLs) will be @@ -222,12 +237,9 @@ impl GlobalBarrierManager { .await; tracing::info!("recovery start!"); - self.clean_dirty_tables() - .await - .expect("clean dirty tables should not fail"); - self.clean_dirty_fragments() + self.clean_dirty_streaming_jobs() .await - .expect("clean dirty fragments"); + .expect("clean dirty streaming jobs"); self.sink_manager.reset().await; let retry_strategy = Self::get_retry_strategy(); @@ -249,31 +261,32 @@ impl GlobalBarrierManager { // This is a quick path to accelerate the process of dropping streaming jobs. Not that // some table fragments might have been cleaned as dirty, but it's fine since the drop // interface is idempotent. - let to_drop_tables = self.scheduled_barriers.pre_apply_drop_scheduled().await; - self.fragment_manager - .drop_table_fragments_vec(&to_drop_tables) - .await?; + if let MetadataManager::V1(mgr) = &self.metadata_manager { + let to_drop_tables = + self.scheduled_barriers.pre_apply_drop_scheduled().await; + mgr.fragment_manager + .drop_table_fragments_vec(&to_drop_tables) + .await?; + } // Resolve actor info for recovery. If there's no actor to recover, most of the // following steps will be no-op, while the compute nodes will still be reset. - let mut info = self.resolve_actor_info_for_recovery().await; - - if self.env.opts.enable_scale_in_when_recovery { + let info = if self.env.opts.enable_scale_in_when_recovery { + let info = self.resolve_actor_info_for_recovery().await; let scaled = self.scale_actors(&info).await.inspect_err(|err| { warn!(err = ?err, "scale actors failed"); })?; if scaled { - info = self.resolve_actor_info_for_recovery().await; + self.resolve_actor_info_for_recovery().await + } else { + info } } else { // Migrate actors in expired CN to newly joined one. - let migrated = self.migrate_actors(&info).await.inspect_err(|err| { + self.migrate_actors().await.inspect_err(|err| { warn!(err = ?err, "migrate actors failed"); - })?; - if migrated { - info = self.resolve_actor_info_for_recovery().await; - } - } + })? + }; // Reset all compute nodes, stop and drop existing actors. self.reset_compute_nodes(&info).await.inspect_err(|err| { @@ -303,8 +316,7 @@ impl GlobalBarrierManager { // Inject the `Initial` barrier to initialize all executors. let command_ctx = Arc::new(CommandContext::new( - self.fragment_manager.clone(), - self.catalog_manager.clone(), + self.metadata_manager.clone(), self.hummock_manager.clone(), self.env.stream_client_pool_ref(), info, @@ -374,8 +386,24 @@ impl GlobalBarrierManager { } /// Migrate actors in expired CNs to newly joined ones, return true if any actor is migrated. - async fn migrate_actors(&self, info: &BarrierActorInfo) -> MetaResult { - debug!("start migrate actors."); + async fn migrate_actors(&self) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(_) => self.migrate_actors_v1().await, + MetadataManager::V2(_) => self.migrate_actors_v2(), + } + } + + fn migrate_actors_v2(&self) -> MetaResult { + unimplemented!("support migrate actors in v2"); + } + + /// Migrate actors in expired CNs to newly joined ones, return true if any actor is migrated. + async fn migrate_actors_v1(&self) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!() + }; + + let info = self.resolve_actor_info_for_recovery().await; // 1. get expired workers. let expired_workers: HashSet = info @@ -386,21 +414,41 @@ impl GlobalBarrierManager { .collect(); if expired_workers.is_empty() { debug!("no expired workers, skipping."); - return Ok(false); + return Ok(info); } + + debug!("start migrate actors."); let migration_plan = self.generate_migration_plan(expired_workers).await?; // 2. start to migrate fragment one-by-one. - self.fragment_manager + mgr.fragment_manager .migrate_fragment_actors(&migration_plan) .await?; // 3. remove the migration plan. migration_plan.delete(self.env.meta_store()).await?; - debug!("migrate actors succeed."); - Ok(true) + + let info = self.resolve_actor_info_for_recovery().await; + Ok(info) } async fn scale_actors(&self, info: &BarrierActorInfo) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(_) => self.scale_actors_v1(info).await, + MetadataManager::V2(_) => self.scale_actors_v2(info), + } + } + + fn scale_actors_v2(&self, _info: &BarrierActorInfo) -> MetaResult { + let MetadataManager::V2(_mgr) = &self.metadata_manager else { + unreachable!() + }; + unimplemented!("implement auto-scale funcs in sql backend") + } + + async fn scale_actors_v1(&self, info: &BarrierActorInfo) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!() + }; debug!("start scaling-in offline actors."); let expired_workers: HashSet = info @@ -416,7 +464,7 @@ impl GlobalBarrierManager { } let table_parallelisms = { - let guard = self.fragment_manager.get_fragment_read_guard().await; + let guard = mgr.fragment_manager.get_fragment_read_guard().await; guard .table_fragments() @@ -425,7 +473,7 @@ impl GlobalBarrierManager { .collect() }; - let workers = self + let workers = mgr .cluster_manager .list_active_streaming_compute_nodes() .await; @@ -444,6 +492,8 @@ impl GlobalBarrierManager { let plan = self .scale_controller + .as_ref() + .unwrap() .generate_table_resize_plan(TableResizePolicy { worker_ids: schedulable_worker_ids, table_parallelisms, @@ -452,6 +502,8 @@ impl GlobalBarrierManager { let (reschedule_fragment, applied_reschedules) = self .scale_controller + .as_ref() + .unwrap() .prepare_reschedule_command( plan, RescheduleOptions { @@ -463,6 +515,8 @@ impl GlobalBarrierManager { if let Err(e) = self .scale_controller + .as_ref() + .unwrap() .post_apply_reschedule(&reschedule_fragment, &Default::default()) .await { @@ -471,7 +525,7 @@ impl GlobalBarrierManager { e.to_string() ); - self.fragment_manager + mgr.fragment_manager .cancel_apply_reschedules(applied_reschedules) .await; @@ -488,9 +542,13 @@ impl GlobalBarrierManager { &self, expired_workers: HashSet, ) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unreachable!() + }; + let mut cached_plan = MigrationPlan::get(self.env.meta_store()).await?; - let all_worker_parallel_units = self.fragment_manager.all_worker_parallel_units().await; + let all_worker_parallel_units = mgr.fragment_manager.all_worker_parallel_units().await; let (expired_inuse_workers, inuse_workers): (Vec<_>, Vec<_>) = all_worker_parallel_units .into_iter() @@ -534,7 +592,7 @@ impl GlobalBarrierManager { let start = Instant::now(); // if in-used expire parallel units are not empty, should wait for newly joined worker. 'discovery: while !to_migrate_parallel_units.is_empty() { - let mut new_parallel_units = self + let mut new_parallel_units = mgr .cluster_manager .list_active_streaming_parallel_units() .await; @@ -614,7 +672,7 @@ impl GlobalBarrierManager { .flatten_ok() .try_collect()?; - let mut node_actors = self.fragment_manager.all_node_actors(false).await; + let mut node_actors = self.metadata_manager.all_node_actors(false).await?; info.actor_map.iter().map(|(node_id, actors)| { let new_actors = node_actors.remove(node_id).unwrap_or_default(); diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index e06560bdcbcdf..ccfd2ea03d08e 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -431,6 +431,7 @@ impl ScheduledBarriers { self.inner .num_uncheckpointed_barrier .load(Ordering::Relaxed) + + 1 >= self.inner.checkpoint_frequency.load(Ordering::Relaxed) || self.inner.force_checkpoint.load(Ordering::Relaxed) } diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index e9db4a30bacad..bc37e17cb33df 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -75,6 +75,7 @@ pub struct CatalogController { #[derive(Clone, Default)] pub struct ReleaseContext { pub(crate) streaming_jobs: Vec, + #[allow(dead_code)] pub(crate) state_table_ids: Vec, pub(crate) source_ids: Vec, pub(crate) connections: Vec, diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index b6b2097d789f4..3c6314309abf4 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1004,7 +1004,7 @@ impl CatalogController { } /// Get the actor ids of the fragment with `fragment_id` with `Running` status. - pub async fn get_running_actors_by_fragment( + pub async fn get_running_actors_of_fragment( &self, fragment_id: FragmentId, ) -> MetaResult> { diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 0ab0326bcfcc1..d6d6891a12e8d 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![expect(dead_code, reason = "WIP")] - use anyhow::anyhow; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::{ diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 704bcb21480ab..f884b0bdeca7b 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -36,20 +36,17 @@ use tower_http::cors::{self, CorsLayer}; use tower_http::services::ServeDir; use crate::manager::diagnose::DiagnoseCommandRef; -use crate::manager::{ClusterManagerRef, FragmentManagerRef}; -use crate::storage::MetaStoreRef; +use crate::manager::MetadataManager; #[derive(Clone)] pub struct DashboardService { pub dashboard_addr: SocketAddr, pub prometheus_client: Option, pub prometheus_selector: String, - pub cluster_manager: ClusterManagerRef, - pub fragment_manager: FragmentManagerRef, + pub metadata_manager: MetadataManager, pub compute_clients: ComputeClientPool, pub ui_path: Option, - pub meta_store: MetaStoreRef, - pub diagnose_command: DiagnoseCommandRef, + pub diagnose_command: Option, pub trace_state: otlp_embedded::StateRef, } @@ -72,8 +69,7 @@ pub(super) mod handlers { use super::*; use crate::manager::WorkerId; - use crate::model::TableFragments; - use crate::storage::MetaStoreRef; + use crate::model::MetadataModel; pub struct DashboardError(anyhow::Error); pub type Result = std::result::Result; @@ -108,78 +104,87 @@ pub(super) mod handlers { .map_err(|_| anyhow!("invalid worker type")) .map_err(err)?; let mut result = srv - .cluster_manager + .metadata_manager .list_worker_node(Some(worker_type), None) - .await; + .await + .map_err(err)?; result.sort_unstable_by_key(|n| n.id); Ok(result.into()) } async fn list_table_catalogs_inner( - meta_store: &MetaStoreRef, + metadata_manager: &MetadataManager, table_type: TableType, ) -> Result>> { - use crate::model::MetadataModel; - - let results = Table::list(meta_store) - .await - .map_err(err)? - .into_iter() - .filter(|t| t.table_type() == table_type) - .collect(); + let tables = match metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.list_tables_by_type(table_type).await, + MetadataManager::V2(mgr) => mgr + .catalog_controller + .list_tables_by_type(table_type.into()) + .await + .map_err(err)?, + }; - Ok(Json(results)) + Ok(Json(tables)) } pub async fn list_materialized_views( Extension(srv): Extension, ) -> Result>> { - list_table_catalogs_inner(&srv.meta_store, TableType::MaterializedView).await + list_table_catalogs_inner(&srv.metadata_manager, TableType::MaterializedView).await } pub async fn list_tables(Extension(srv): Extension) -> Result>> { - list_table_catalogs_inner(&srv.meta_store, TableType::Table).await + list_table_catalogs_inner(&srv.metadata_manager, TableType::Table).await } pub async fn list_indexes(Extension(srv): Extension) -> Result>> { - list_table_catalogs_inner(&srv.meta_store, TableType::Index).await + list_table_catalogs_inner(&srv.metadata_manager, TableType::Index).await } pub async fn list_internal_tables( Extension(srv): Extension, ) -> Result>> { - list_table_catalogs_inner(&srv.meta_store, TableType::Internal).await + list_table_catalogs_inner(&srv.metadata_manager, TableType::Internal).await } pub async fn list_sources(Extension(srv): Extension) -> Result>> { - use crate::model::MetadataModel; + let sources = srv.metadata_manager.list_sources().await.map_err(err)?; - let sources = Source::list(&srv.meta_store).await.map_err(err)?; Ok(Json(sources)) } pub async fn list_sinks(Extension(srv): Extension) -> Result>> { - use crate::model::MetadataModel; + let sinks = match &srv.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.list_sinks().await, + MetadataManager::V2(mgr) => mgr.catalog_controller.list_sinks().await.map_err(err)?, + }; - let sinks = Sink::list(&srv.meta_store).await.map_err(err)?; Ok(Json(sinks)) } pub async fn list_views(Extension(srv): Extension) -> Result>> { - use crate::model::MetadataModel; + let views = match &srv.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.list_views().await, + MetadataManager::V2(mgr) => mgr.catalog_controller.list_views().await.map_err(err)?, + }; - let sinks = View::list(&srv.meta_store).await.map_err(err)?; - Ok(Json(sinks)) + Ok(Json(views)) } pub async fn list_actors( Extension(srv): Extension, ) -> Result>> { - let mut node_actors = srv.fragment_manager.all_node_actors(true).await; + let mut node_actors = srv + .metadata_manager + .all_node_actors(true) + .await + .map_err(err)?; let nodes = srv - .cluster_manager + .metadata_manager .list_active_streaming_compute_nodes() - .await; + .await + .map_err(err)?; let actors = nodes .into_iter() .map(|node| ActorLocation { @@ -194,14 +199,25 @@ pub(super) mod handlers { pub async fn list_fragments( Extension(srv): Extension, ) -> Result>> { - use crate::model::MetadataModel; + let table_fragments = match &srv.metadata_manager { + MetadataManager::V1(mgr) => mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .values() + .map(|tf| tf.to_protobuf()) + .collect_vec(), + MetadataManager::V2(mgr) => mgr + .catalog_controller + .table_fragments() + .await + .map_err(err)? + .values() + .cloned() + .collect_vec(), + }; - let table_fragments = TableFragments::list(&srv.meta_store) - .await - .map_err(err)? - .into_iter() - .map(|x| x.to_protobuf()) - .collect_vec(); Ok(Json(table_fragments)) } @@ -231,9 +247,10 @@ pub(super) mod handlers { Extension(srv): Extension, ) -> Result> { let worker_nodes = srv - .cluster_manager + .metadata_manager .list_worker_node(Some(WorkerType::ComputeNode), None) - .await; + .await + .map_err(err)?; dump_await_tree_inner(&worker_nodes, &srv.compute_clients).await } @@ -243,12 +260,12 @@ pub(super) mod handlers { Extension(srv): Extension, ) -> Result> { let worker_node = srv - .cluster_manager + .metadata_manager .get_worker_by_id(worker_id) .await - .context("worker node not found") .map_err(err)? - .worker_node; + .context("worker node not found") + .map_err(err)?; dump_await_tree_inner(std::iter::once(&worker_node), &srv.compute_clients).await } @@ -258,12 +275,12 @@ pub(super) mod handlers { Extension(srv): Extension, ) -> Result> { let worker_node = srv - .cluster_manager + .metadata_manager .get_worker_by_id(worker_id) .await - .context("worker node not found") .map_err(err)? - .worker_node; + .context("worker node not found") + .map_err(err)?; let client = srv.compute_clients.get(&worker_node).await.map_err(err)?; @@ -277,12 +294,12 @@ pub(super) mod handlers { Extension(srv): Extension, ) -> Result> { let worker_node = srv - .cluster_manager + .metadata_manager .get_worker_by_id(worker_id) .await - .context("worker node not found") .map_err(err)? - .worker_node; + .context("worker node not found") + .map_err(err)?; let client = srv.compute_clients.get(&worker_node).await.map_err(err)?; @@ -310,12 +327,12 @@ pub(super) mod handlers { let collapsed_file_name = format!("{}.{}", file_name, COLLAPSED_SUFFIX); let worker_node = srv - .cluster_manager + .metadata_manager .get_worker_by_id(worker_id) .await - .context("worker node not found") .map_err(err)? - .worker_node; + .context("worker node not found") + .map_err(err)?; let client = srv.compute_clients.get(&worker_node).await.map_err(err)?; @@ -335,7 +352,13 @@ pub(super) mod handlers { } pub async fn diagnose(Extension(srv): Extension) -> Result { - Ok(srv.diagnose_command.report().await) + let report = if let Some(cmd) = &srv.diagnose_command { + cmd.report().await + } else { + "Not supported in sql-backend".to_string() + }; + + Ok(report) } } diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index 02f825c9ad800..1cf3e580ed44c 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -75,11 +75,13 @@ impl HummockManager { } /// Checks whether `context_id` is valid. - pub async fn check_context(&self, context_id: HummockContextId) -> bool { - self.cluster_manager + pub async fn check_context(&self, context_id: HummockContextId) -> Result { + Ok(self + .metadata_manager() .get_worker_by_id(context_id) .await - .is_some() + .map_err(|err| Error::MetaStore(err.into()))? + .is_some()) } /// Release invalid contexts, aka worker node ids which are no longer valid in `ClusterManager`. @@ -103,7 +105,7 @@ impl HummockManager { let mut invalid_context_ids = vec![]; for active_context_id in &active_context_ids { - if !self.check_context(*active_context_id).await { + if !self.check_context(*active_context_id).await? { invalid_context_ids.push(*active_context_id); } } @@ -129,7 +131,7 @@ impl HummockManager { continue; } } - if !self.check_context(*context_id).await { + if !self.check_context(*context_id).await? { return Err(Error::InvalidSst(*sst_id)); } } diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index eec8531058b94..b4506ac30974f 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -28,7 +28,7 @@ use risingwave_pb::hummock::FullScanTask; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::{commit_multi_var, read_lock, write_lock, ResponseEvent}; use crate::hummock::HummockManager; -use crate::manager::ClusterManagerRef; +use crate::manager::MetadataManager; use crate::model::{BTreeMapTransaction, ValTransaction}; use crate::storage::Transaction; @@ -172,7 +172,7 @@ impl HummockManager { let spin_interval = Duration::from_secs(self.env.opts.collect_gc_watermark_spin_interval_sec); let watermark = - collect_global_gc_watermark(self.cluster_manager().clone(), spin_interval).await?; + collect_global_gc_watermark(self.metadata_manager().clone(), spin_interval).await?; metrics.full_gc_last_object_id_watermark.set(watermark as _); let candidate_sst_number = object_ids.len(); metrics @@ -201,15 +201,19 @@ impl HummockManager { /// Returns a global GC watermark. The watermark only guards SSTs created before this /// invocation. pub async fn collect_global_gc_watermark( - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, spin_interval: Duration, ) -> Result { let mut global_watermark = HummockSstableObjectId::MAX; let workers = [ - cluster_manager.list_active_streaming_compute_nodes().await, - cluster_manager + metadata_manager + .list_active_streaming_compute_nodes() + .await + .map_err(|err| Error::MetaStore(err.into()))?, + metadata_manager .list_worker_node(Some(WorkerType::Compactor), Some(Running)) - .await, + .await + .map_err(|err| Error::MetaStore(err.into()))?, ] .concat(); @@ -224,11 +228,14 @@ pub async fn collect_global_gc_watermark( // which doesn't correctly guard target SSTs. // The second heartbeat guarantees its watermark is took after the start of this method. let worker_id = worker.id; - let cluster_manager_clone = cluster_manager.clone(); + let metadata_manager_clone = metadata_manager.clone(); worker_futures.push(tokio::spawn(async move { let mut init_version_id: Option = None; loop { - let worker_info = match cluster_manager_clone.get_worker_by_id(worker_id).await { + let worker_info = match metadata_manager_clone + .get_worker_info_by_id(worker_id) + .await + { None => { return None; } diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index e4870f66d381c..0ea5c51904c21 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -60,6 +60,7 @@ use risingwave_pb::hummock::{ PbCompactionGroupInfo, SstableInfo, SubscribeCompactionEventRequest, TableOption, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use thiserror_ext::AsReport; use tokio::sync::mpsc::{UnboundedReceiver, UnboundedSender}; use tokio::sync::oneshot::Sender; use tokio::sync::RwLockWriteGuard; @@ -81,7 +82,7 @@ use crate::hummock::metrics_utils::{ }; use crate::hummock::{CompactorManagerRef, TASK_NORMAL}; use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, IdCategory, MetaSrvEnv, TableId, + ClusterManagerRef, FragmentManagerRef, IdCategory, MetaSrvEnv, MetadataManager, TableId, META_NODE_ID, }; use crate::model::{ @@ -115,10 +116,8 @@ const HISTORY_TABLE_INFO_STATISTIC_TIME: usize = 240; // succeeds, the in-mem state will be updated by the way. pub struct HummockManager { pub env: MetaSrvEnv, - pub cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, /// Lock order: compaction, versioning, compaction_group_manager. /// - Lock compaction first, then versioning, and finally compaction_group_manager. /// - This order should be strictly followed to prevent deadlock. @@ -299,11 +298,9 @@ impl CommitEpochInfo { impl HummockManager { pub async fn new( env: MetaSrvEnv, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, metrics: Arc, compactor_manager: CompactorManagerRef, - catalog_manager: CatalogManagerRef, compactor_streams_change_tx: UnboundedSender<( u32, Streaming, @@ -312,12 +309,10 @@ impl HummockManager { let compaction_group_manager = Self::build_compaction_group_manager(&env).await?; Self::new_impl( env, - cluster_manager, - fragment_manager, + metadata_manager, metrics, compactor_manager, compaction_group_manager, - catalog_manager, compactor_streams_change_tx, ) .await @@ -342,14 +337,14 @@ impl HummockManager { .await .unwrap(); let catalog_manager = Arc::new(CatalogManager::new(env.clone()).await.unwrap()); + let metadata_manager = + MetadataManager::new_v1(cluster_manager, catalog_manager, fragment_manager); Self::new_impl( env, - cluster_manager, - fragment_manager, + metadata_manager, metrics, compactor_manager, compaction_group_manager, - catalog_manager, compactor_streams_change_tx, ) .await @@ -358,19 +353,16 @@ impl HummockManager { async fn new_impl( env: MetaSrvEnv, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, metrics: Arc, compactor_manager: CompactorManagerRef, compaction_group_manager: tokio::sync::RwLock, - catalog_manager: CatalogManagerRef, compactor_streams_change_tx: UnboundedSender<( u32, Streaming, )>, ) -> Result { - let sys_params_manager = env.system_params_manager(); - let sys_params = sys_params_manager.get_params().await; + let sys_params = env.system_params_reader().await; let state_store_url = sys_params.state_store(); let state_store_dir: &str = sys_params.data_directory(); let deterministic_mode = env.opts.compaction_deterministic_test; @@ -409,6 +401,7 @@ impl HummockManager { } let checkpoint_path = version_checkpoint_path(state_store_dir); let (tx, rx) = tokio::sync::mpsc::unbounded_channel(); + let instance = HummockManager { env, versioning: MonitoredRwLock::new( @@ -420,9 +413,7 @@ impl HummockManager { Default::default(), ), metrics, - cluster_manager, - catalog_manager, - fragment_manager, + metadata_manager, compaction_group_manager, // compaction_request_channel: parking_lot::RwLock::new(None), compactor_manager, @@ -566,7 +557,7 @@ impl HummockManager { if let Some(context_id) = context_id { if context_id == META_NODE_ID { // Using the preserved meta id is allowed. - } else if !self.check_context(context_id).await { + } else if !self.check_context(context_id).await? { // The worker is not found in cluster. return Err(Error::InvalidContext(context_id)); } @@ -806,7 +797,11 @@ impl HummockManager { // TODO: `get_all_table_options` will hold catalog_manager async lock, to avoid holding the // lock in compaction_guard, take out all table_options in advance there may be a // waste of resources here, need to add a more efficient filter in catalog_manager - let all_table_id_to_option = self.catalog_manager.get_all_table_options().await; + let all_table_id_to_option = self + .metadata_manager + .get_all_table_options() + .await + .map_err(|err| Error::MetaStore(err.into()))?; let mut table_to_vnode_partition = match self .group_to_table_vnode_partition .read() @@ -2022,8 +2017,8 @@ impl HummockManager { .await } - pub fn cluster_manager(&self) -> &ClusterManagerRef { - &self.cluster_manager + pub fn metadata_manager(&self) -> &MetadataManager { + &self.metadata_manager } fn notify_last_version_delta(&self, versioning: &Versioning) { @@ -2235,8 +2230,9 @@ impl HummockManager { }; if let Some(mv_id_to_all_table_ids) = hummock_manager - .fragment_manager - .get_mv_id_to_internal_table_ids_mapping() + .metadata_manager + .get_job_id_to_internal_table_ids_mapping() + .await { trigger_mv_stat( &hummock_manager.metrics, @@ -2475,13 +2471,19 @@ impl HummockManager { /// * For state-table whose throughput less than `min_table_split_write_throughput`, do not /// increase it size of base-level. async fn on_handle_check_split_multi_group(&self) { - let params = self.env.system_params_manager().get_params().await; + let params = self.env.system_params_reader().await; let barrier_interval_ms = params.barrier_interval_ms() as u64; let checkpoint_secs = std::cmp::max( 1, params.checkpoint_frequency() * barrier_interval_ms / 1000, ); - let created_tables = self.catalog_manager.get_created_table_ids().await; + let created_tables = match self.metadata_manager.get_created_table_ids().await { + Ok(created_tables) => created_tables, + Err(err) => { + tracing::warn!(error = %err.as_report(), "failed to fetch created table ids"); + return; + } + }; let created_tables: HashSet = HashSet::from_iter(created_tables); let table_write_throughput = self.history_table_throughput.read().clone(); let mut group_infos = self.calculate_compaction_group_statistic().await; @@ -3170,12 +3172,18 @@ async fn write_exclusive_cluster_id( let cluster_id_dir = format!("{}/{}/", state_store_dir, CLUSTER_ID_DIR); let cluster_id_full_path = format!("{}{}", cluster_id_dir, CLUSTER_ID_NAME); match object_store.read(&cluster_id_full_path, ..).await { - Ok(cluster_id) => Err(ObjectError::internal(format!( - "Data directory is already used by another cluster with id {:?}, path {}.", - String::from_utf8(cluster_id.to_vec()).unwrap(), - cluster_id_full_path, - )) - .into()), + Ok(stored_cluster_id) => { + let stored_cluster_id = String::from_utf8(stored_cluster_id.to_vec()).unwrap(); + if cluster_id.deref() == stored_cluster_id { + return Ok(()); + } + + Err(ObjectError::internal(format!( + "Data directory is already used by another cluster with id {:?}, path {}.", + stored_cluster_id, cluster_id_full_path, + )) + .into()) + } Err(e) => { if e.is_object_not_found_error() { object_store diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 3580e1d35dfbe..b06661ec0e6cb 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -44,6 +44,7 @@ use crate::hummock::model::CompactionGroup; use crate::hummock::HummockManager; use crate::model::{ValTransaction, VarTransaction}; use crate::storage::Transaction; +use crate::MetaResult; /// `HummockVersionSafePoint` prevents hummock versions GE than it from being GC. /// It's used by meta node itself to temporarily pin versions. @@ -189,14 +190,18 @@ impl HummockManager { pub async fn list_workers( &self, context_ids: &[HummockContextId], - ) -> HashMap { + ) -> MetaResult> { let mut workers = HashMap::new(); for context_id in context_ids { - if let Some(worker) = self.cluster_manager.get_worker_by_id(*context_id).await { - workers.insert(*context_id, worker.worker_node); + if let Some(worker_node) = self + .metadata_manager() + .get_worker_by_id(*context_id) + .await? + { + workers.insert(*context_id, worker_node); } } - workers + Ok(workers) } #[named] diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 0a6ddc9404170..fc77e9284fca8 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -235,9 +235,9 @@ impl HummockMetaClient for MockHummockMetaClient { UnboundedSender, BoxStream<'static, CompactionEventItem>, )> { - let worker_node = self + let context_id = self .hummock_manager - .cluster_manager() + .metadata_manager() .add_worker_node( WorkerType::Compactor, HostAddress { @@ -249,7 +249,6 @@ impl HummockMetaClient for MockHummockMetaClient { ) .await .unwrap(); - let context_id = worker_node.id; let _compactor_rx = self .hummock_manager .compactor_manager_ref_for_test() diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 5b6e8f2245a60..a9c7459d745d9 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -329,6 +329,14 @@ impl DatabaseManager { self.sources.values().cloned().collect_vec() } + pub fn list_sinks(&self) -> Vec { + self.sinks.values().cloned().collect_vec() + } + + pub fn list_views(&self) -> Vec { + self.views.values().cloned().collect_vec() + } + pub fn list_source_ids(&self, schema_id: SchemaId) -> Vec { self.sources .values() diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index cce6ee4ff2bad..f272e6d1ff198 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -120,7 +120,6 @@ use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; use risingwave_pb::meta::cancel_creating_jobs_request::CreatingJobInfo; use risingwave_pb::meta::relation::RelationInfo; -use risingwave_pb::meta::table_fragments::State; use risingwave_pb::meta::{Relation, RelationGroup}; pub(crate) use {commit_meta, commit_meta_with_trx}; @@ -876,8 +875,8 @@ impl CatalogManager { let fragment: TableFragments = fragment; // 3. For those in initial state (i.e. not running / created), // we should purge them. - if fragment.state() == State::Initial { - tracing::debug!("cleaning table_id no initial state: {:#?}", table.id); + if fragment.is_initial() { + tracing::debug!("cleaning table_id with initial state: {:#?}", table.id); tables_to_clean.push(table); continue; } else { @@ -3175,6 +3174,18 @@ impl CatalogManager { self.core.lock().await.database.list_tables() } + pub async fn list_tables_by_type(&self, table_type: TableType) -> Vec
{ + self.core + .lock() + .await + .database + .tables + .values() + .filter(|table| table.table_type == table_type as i32) + .cloned() + .collect_vec() + } + /// Lists table catalogs for mviews, without their internal tables. pub async fn list_creating_background_mvs(&self) -> Vec
{ self.core @@ -3217,6 +3228,14 @@ impl CatalogManager { self.core.lock().await.database.list_sources() } + pub async fn list_sinks(&self) -> Vec { + self.core.lock().await.database.list_sinks() + } + + pub async fn list_views(&self) -> Vec { + self.core.lock().await.database.list_views() + } + pub async fn list_source_ids(&self, schema_id: SchemaId) -> Vec { self.core.lock().await.database.list_source_ids(schema_id) } diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index d5a05bbbfe87b..2bfcdf70eaed2 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -16,6 +16,7 @@ use std::ops::Deref; use std::sync::Arc; use risingwave_common::config::{CompactionConfig, DefaultParallelism}; +use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_meta_model_v2::prelude::Cluster; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{ConnectorClient, StreamClientPool, StreamClientPoolRef}; @@ -357,6 +358,13 @@ impl MetaSrvEnv { self.idle_manager.deref() } + pub async fn system_params_reader(&self) -> SystemParamsReader { + if let Some(system_ctl) = &self.system_params_controller { + return system_ctl.get_params().await; + } + self.system_params_manager.get_params().await + } + pub fn system_params_manager_ref(&self) -> SystemParamsManagerRef { self.system_params_manager.clone() } diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs new file mode 100644 index 0000000000000..67f077f553167 --- /dev/null +++ b/src/meta/src/manager/metadata.rs @@ -0,0 +1,369 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; + +use risingwave_common::catalog::{TableId, TableOption}; +use risingwave_pb::catalog::PbSource; +use risingwave_pb::common::worker_node::{PbResource, State}; +use risingwave_pb::common::{HostAddress, PbWorkerNode, PbWorkerType, WorkerType}; +use risingwave_pb::ddl_service::TableJobType; +use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; +use risingwave_pb::meta::table_fragments::Fragment; +use risingwave_pb::stream_plan::PbStreamActor; + +use crate::controller::catalog::CatalogControllerRef; +use crate::controller::cluster::{ClusterControllerRef, WorkerExtraInfo}; +use crate::manager::{ + CatalogManagerRef, ClusterManagerRef, FragmentManagerRef, StreamingClusterInfo, WorkerId, +}; +use crate::model::{ActorId, FragmentId, MetadataModel, TableFragments}; +use crate::MetaResult; + +#[derive(Clone)] +pub enum MetadataManager { + V1(MetadataManagerV1), + V2(MetadataManagerV2), +} + +#[derive(Clone)] +pub struct MetadataManagerV1 { + pub cluster_manager: ClusterManagerRef, + pub catalog_manager: CatalogManagerRef, + pub fragment_manager: FragmentManagerRef, +} + +#[derive(Clone)] +pub struct MetadataManagerV2 { + pub cluster_controller: ClusterControllerRef, + pub catalog_controller: CatalogControllerRef, +} + +impl MetadataManager { + pub fn new_v1( + cluster_manager: ClusterManagerRef, + catalog_manager: CatalogManagerRef, + fragment_manager: FragmentManagerRef, + ) -> Self { + Self::V1(MetadataManagerV1 { + cluster_manager, + catalog_manager, + fragment_manager, + }) + } + + pub fn new_v2( + cluster_controller: ClusterControllerRef, + catalog_controller: CatalogControllerRef, + ) -> Self { + Self::V2(MetadataManagerV2 { + cluster_controller, + catalog_controller, + }) + } + + pub async fn get_worker_by_id(&self, worker_id: WorkerId) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr + .cluster_manager + .get_worker_by_id(worker_id) + .await + .map(|w| w.worker_node)), + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .get_worker_by_id(worker_id as _) + .await + } + } + } + + pub async fn count_worker_node(&self) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr.cluster_manager.count_worker_node().await), + MetadataManager::V2(mgr) => { + let node_map = mgr.cluster_controller.count_worker_by_type().await?; + Ok(node_map + .into_iter() + .map(|(ty, cnt)| (ty.into(), cnt as u64)) + .collect()) + } + } + } + + pub async fn get_worker_info_by_id(&self, worker_id: WorkerId) -> Option { + match &self { + MetadataManager::V1(mgr) => mgr + .cluster_manager + .get_worker_by_id(worker_id) + .await + .map(Into::into), + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .get_worker_info_by_id(worker_id as _) + .await + } + } + } + + pub async fn add_worker_node( + &self, + r#type: PbWorkerType, + host_address: HostAddress, + property: AddNodeProperty, + resource: PbResource, + ) -> MetaResult { + match &self { + MetadataManager::V1(mgr) => mgr + .cluster_manager + .add_worker_node(r#type, host_address, property, resource) + .await + .map(|w| w.id), + MetadataManager::V2(mgr) => mgr + .cluster_controller + .add_worker(r#type, host_address, property, resource) + .await + .map(|id| id as WorkerId), + } + } + + pub async fn list_worker_node( + &self, + worker_type: Option, + worker_state: Option, + ) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr + .cluster_manager + .list_worker_node(worker_type, worker_state) + .await), + MetadataManager::V2(mgr) => { + mgr.cluster_controller + .list_workers(worker_type.map(Into::into), worker_state.map(Into::into)) + .await + } + } + } + + pub async fn list_active_streaming_compute_nodes(&self) -> MetaResult> { + match self { + MetadataManager::V1(mgr) => Ok(mgr + .cluster_manager + .list_active_streaming_compute_nodes() + .await), + MetadataManager::V2(mgr) => { + mgr.cluster_controller.list_active_streaming_workers().await + } + } + } + + pub async fn list_sources(&self) -> MetaResult> { + match self { + MetadataManager::V1(mgr) => Ok(mgr.catalog_manager.list_sources().await), + MetadataManager::V2(mgr) => mgr.catalog_controller.list_sources().await, + } + } + + pub async fn get_upstream_root_fragments( + &self, + upstream_table_ids: &HashSet, + table_job_type: Option, + ) -> MetaResult> { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .get_upstream_root_fragments(upstream_table_ids, table_job_type) + .await + } + MetadataManager::V2(mgr) => { + let upstream_root_fragments = mgr + .catalog_controller + .get_upstream_root_fragments( + upstream_table_ids + .iter() + .map(|id| id.table_id as _) + .collect(), + table_job_type, + ) + .await?; + Ok(upstream_root_fragments + .into_iter() + .map(|(id, fragment)| ((id as u32).into(), fragment)) + .collect()) + } + } + } + + pub async fn get_streaming_cluster_info(&self) -> MetaResult { + match self { + MetadataManager::V1(mgr) => Ok(mgr.cluster_manager.get_streaming_cluster_info().await), + MetadataManager::V2(mgr) => mgr.cluster_controller.get_streaming_cluster_info().await, + } + } + + pub async fn get_all_table_options(&self) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr.catalog_manager.get_all_table_options().await), + MetadataManager::V2(mgr) => mgr + .catalog_controller + .get_all_table_options() + .await + .map(|tops| tops.into_iter().map(|(id, opt)| (id as u32, opt)).collect()), + } + } + + pub async fn get_table_name_type_mapping(&self) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => { + Ok(mgr.catalog_manager.get_table_name_and_type_mapping().await) + } + MetadataManager::V2(mgr) => { + let mappings = mgr.catalog_controller.get_table_name_type_mapping().await?; + Ok(mappings + .into_iter() + .map(|(id, value)| (id as u32, value)) + .collect()) + } + } + } + + pub async fn get_created_table_ids(&self) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr.catalog_manager.get_created_table_ids().await), + MetadataManager::V2(mgr) => { + let table_ids = mgr.catalog_controller.get_created_table_ids().await?; + Ok(table_ids.into_iter().map(|id| id as u32).collect()) + } + } + } + + pub async fn get_job_id_to_internal_table_ids_mapping(&self) -> Option)>> { + match &self { + MetadataManager::V1(mgr) => mgr + .fragment_manager + .get_mv_id_to_internal_table_ids_mapping(), + MetadataManager::V2(mgr) => { + let job_internal_table_ids = + mgr.catalog_controller.get_job_internal_table_ids().await; + job_internal_table_ids.map(|ids| { + ids.into_iter() + .map(|(id, internal_ids)| { + ( + id as u32, + internal_ids.into_iter().map(|id| id as u32).collect(), + ) + }) + .collect() + }) + } + } + } + + pub async fn get_job_fragments_by_id(&self, id: &TableId) -> MetaResult { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .select_table_fragments_by_table_id(id) + .await + } + MetadataManager::V2(mgr) => { + let pb_table_fragments = mgr + .catalog_controller + .get_job_fragments_by_id(id.table_id as _) + .await?; + Ok(TableFragments::from_protobuf(pb_table_fragments)) + } + } + } + + pub async fn get_running_actors_of_fragment( + &self, + id: FragmentId, + ) -> MetaResult> { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .get_running_actors_of_fragment(id) + .await + } + MetadataManager::V2(mgr) => { + let actor_ids = mgr + .catalog_controller + .get_running_actors_of_fragment(id as _) + .await?; + Ok(actor_ids.into_iter().map(|id| id as ActorId).collect()) + } + } + } + + pub async fn get_job_fragments_by_ids( + &self, + ids: &[TableId], + ) -> MetaResult> { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .select_table_fragments_by_ids(ids) + .await + } + MetadataManager::V2(mgr) => { + let mut table_fragments = vec![]; + for id in ids { + let pb_table_fragments = mgr + .catalog_controller + .get_job_fragments_by_id(id.table_id as _) + .await?; + table_fragments.push(TableFragments::from_protobuf(pb_table_fragments)); + } + Ok(table_fragments) + } + } + } + + pub async fn all_node_actors( + &self, + include_inactive: bool, + ) -> MetaResult>> { + match &self { + MetadataManager::V1(mgr) => { + Ok(mgr.fragment_manager.all_node_actors(include_inactive).await) + } + MetadataManager::V2(mgr) => { + let table_fragments = mgr.catalog_controller.table_fragments().await?; + let mut actor_maps = HashMap::new(); + for (_, fragments) in table_fragments { + let tf = TableFragments::from_protobuf(fragments); + for (node_id, actor_ids) in tf.worker_actors(include_inactive) { + let node_actor_ids = actor_maps.entry(node_id).or_insert_with(Vec::new); + node_actor_ids.extend(actor_ids); + } + } + Ok(actor_maps) + } + } + } + + pub async fn drop_streaming_job_by_ids(&self, table_ids: &HashSet) -> MetaResult<()> { + match self { + MetadataManager::V1(mgr) => { + mgr.fragment_manager + .drop_table_fragments_vec(table_ids) + .await + } + MetadataManager::V2(_) => { + // Do nothing. Need to refine drop and cancel process. + Ok(()) + } + } + } +} diff --git a/src/meta/src/manager/mod.rs b/src/meta/src/manager/mod.rs index a811cfd855311..4ab91c388792d 100644 --- a/src/meta/src/manager/mod.rs +++ b/src/meta/src/manager/mod.rs @@ -19,6 +19,7 @@ mod env; pub mod event_log; mod id; mod idle; +mod metadata; mod notification; pub mod sink_coordination; mod streaming_job; @@ -29,6 +30,7 @@ pub use cluster::{WorkerKey, *}; pub use env::{MetaSrvEnv, *}; pub use id::*; pub use idle::*; +pub use metadata::*; pub use notification::{LocalNotification, MessageStatus, NotificationManagerRef, *}; pub use risingwave_meta_model_v2::prelude; pub use streaming_job::*; diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 71e31a55591b3..b5d63256ccb6e 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -119,10 +119,8 @@ impl StreamingJob { index.index_table_id = id; index_table.id = id; } - StreamingJob::Source(_) => { - // The id of source is set in `DdlServiceImpl::create_source`, - // so do nothing here. - unreachable!() + StreamingJob::Source(src) => { + src.id = id; } } } @@ -269,6 +267,20 @@ impl StreamingJob { } } + // TODO: record all objects instead. + pub fn dependent_relations(&self) -> Vec { + match self { + StreamingJob::MaterializedView(table) => table.dependent_relations.clone(), + StreamingJob::Sink(sink, _) => sink.dependent_relations.clone(), + StreamingJob::Table(_, table, _) => table.dependent_relations.clone(), + StreamingJob::Index(index, index_table) => { + assert_eq!(index.primary_table_id, index_table.dependent_relations[0]); + vec![] + } + StreamingJob::Source(_) => vec![], + } + } + pub fn is_source_job(&self) -> bool { matches!(self, StreamingJob::Source(_)) } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 2b31f713b63c7..cd2c71bf2e122 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -253,6 +253,11 @@ impl TableFragments { self.state == State::Created } + /// Returns whether the table fragments is in `Initial` state. + pub fn is_initial(&self) -> bool { + self.state == State::Initial + } + /// Set the table ID. // TODO: remove this workaround for replacing table. pub fn set_table_id(&mut self, table_id: TableId) { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 720cc7ce0987f..9221af4d4e6e3 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -20,20 +20,30 @@ use std::time::Duration; use anyhow::anyhow; use itertools::Itertools; +use rand::Rng; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::{ParallelUnitMapping, VirtualNode}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; -use risingwave_common::util::stream_graph_visitor::{visit_stream_node, visit_stream_node_cont}; +use risingwave_common::util::stream_graph_visitor::{ + visit_fragment, visit_stream_node, visit_stream_node_cont, +}; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::source::cdc::CdcSourceType; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, + UPSTREAM_SOURCE_KEY, }; +use risingwave_meta_model_v2::object::ObjectType; +use risingwave_meta_model_v2::ObjectId; use risingwave_pb::catalog::connection::private_link_service::PbPrivateLinkProvider; +use risingwave_pb::catalog::connection::PrivateLinkService; +use risingwave_pb::catalog::source::OptionalAssociatedTableId; +use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ - connection, Comment, Connection, CreateType, Database, Function, Schema, Sink, Source, Table, - View, + connection, Comment, Connection, CreateType, Database, Function, PbSource, PbTable, Schema, + Sink, Source, Table, View, }; use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{ @@ -43,7 +53,7 @@ use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::PbTableParallelism; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, + Dispatcher, DispatcherType, FragmentTypeFlag, MergeNode, PbStreamFragmentGraph, StreamFragmentGraph as StreamFragmentGraphProto, }; use tokio::sync::Semaphore; @@ -52,11 +62,12 @@ use tracing::log::warn; use tracing::Instrument; use crate::barrier::BarrierManagerRef; +use crate::controller::catalog::{CatalogControllerRef, ReleaseContext}; use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, - IdCategory, IndexId, LocalNotification, MetaSrvEnv, NotificationVersion, RelationIdEnum, - SchemaId, SinkId, SourceId, StreamingClusterInfo, StreamingJob, TableId, UserId, ViewId, - IGNORED_NOTIFICATION_VERSION, + CatalogManagerRef, ConnectionId, DatabaseId, FragmentManagerRef, FunctionId, IdCategory, + IdCategoryType, IndexId, LocalNotification, MetaSrvEnv, MetadataManager, MetadataManagerV1, + NotificationVersion, RelationIdEnum, SchemaId, SinkId, SourceId, StreamingClusterInfo, + StreamingJob, TableId, UserId, ViewId, IGNORED_NOTIFICATION_VERSION, }; use crate::model::{FragmentId, StreamContext, TableFragments, TableParallelism}; use crate::rpc::cloud_provider::AwsEc2Client; @@ -130,7 +141,7 @@ pub enum DdlCommand { AlterName(alter_name_request::Object, String), ReplaceTable(ReplaceTableInfo), AlterSourceColumn(Source), - AlterTableOwner(Object, UserId), + AlterObjectOwner(Object, UserId), AlterSetSchema(alter_set_schema_request::Object, SchemaId), CreateConnection(Connection), DropConnection(ConnectionId), @@ -155,30 +166,27 @@ impl DdlCommand { #[derive(Clone)] pub struct DdlController { - env: MetaSrvEnv, + pub(crate) env: MetaSrvEnv, - catalog_manager: CatalogManagerRef, - stream_manager: GlobalStreamManagerRef, - source_manager: SourceManagerRef, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, + pub(crate) metadata_manager: MetadataManager, + pub(crate) stream_manager: GlobalStreamManagerRef, + pub(crate) source_manager: SourceManagerRef, barrier_manager: BarrierManagerRef, aws_client: Arc>, // The semaphore is used to limit the number of concurrent streaming job creation. - creating_streaming_job_permits: Arc, + pub(crate) creating_streaming_job_permits: Arc, } #[derive(Clone)] pub struct CreatingStreamingJobPermit { - semaphore: Arc, + pub(crate) semaphore: Arc, } impl CreatingStreamingJobPermit { async fn new(env: &MetaSrvEnv) -> Self { let mut permits = env - .system_params_manager() - .get_params() + .system_params_reader() .await .max_concurrent_creating_streaming_jobs() as usize; if permits == 0 { @@ -231,28 +239,29 @@ impl CreatingStreamingJobPermit { impl DdlController { pub async fn new( env: MetaSrvEnv, - catalog_manager: CatalogManagerRef, + metadata_manager: MetadataManager, stream_manager: GlobalStreamManagerRef, source_manager: SourceManagerRef, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, barrier_manager: BarrierManagerRef, aws_client: Arc>, ) -> Self { let creating_streaming_job_permits = Arc::new(CreatingStreamingJobPermit::new(&env).await); Self { env, - catalog_manager, + metadata_manager, stream_manager, source_manager, - cluster_manager, - fragment_manager, barrier_manager, aws_client, creating_streaming_job_permits, } } + async fn gen_unique_id(&self) -> MetaResult { + let id = self.env.id_gen_manager().generate::().await? as u32; + Ok(id) + } + /// `run_command` spawns a tokio coroutine to execute the target ddl command. When the client /// has been interrupted during executing, the request will be cancelled by tonic. Since we have /// a lot of logic for revert, status management, notification and so on, ensuring consistency @@ -305,7 +314,7 @@ impl DdlController { .await } DdlCommand::AlterName(relation, name) => ctrl.alter_name(relation, &name).await, - DdlCommand::AlterTableOwner(object, owner_id) => { + DdlCommand::AlterObjectOwner(object, owner_id) => { ctrl.alter_owner(object, owner_id).await } DdlCommand::AlterSetSchema(object, new_schema_id) => { @@ -329,6 +338,16 @@ impl DdlController { self.barrier_manager.get_ddl_progress().await } + async fn create_database(&self, mut database: Database) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + database.id = self.gen_unique_id::<{ IdCategory::Database }>().await?; + mgr.catalog_manager.create_database(&database).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_database(database).await, + } + } + pub async fn alter_parallelism( &self, table_id: u32, @@ -339,14 +358,14 @@ impl DdlController { .await } - async fn create_database(&self, database: Database) -> MetaResult { - self.catalog_manager.create_database(&database).await - } - - async fn drop_database(&self, database_id: DatabaseId) -> MetaResult { + async fn drop_database_v1( + &self, + catalog_manager: &CatalogManagerRef, + database_id: DatabaseId, + ) -> MetaResult { // 1. drop all catalogs in this database. let (version, streaming_ids, source_ids, connections_dropped) = - self.catalog_manager.drop_database(database_id).await?; + catalog_manager.drop_database(database_id).await?; // 2. Unregister source connector worker. self.source_manager.unregister_sources(source_ids).await; // 3. drop streaming jobs. @@ -361,33 +380,100 @@ impl DdlController { Ok(version) } - async fn create_schema(&self, schema: Schema) -> MetaResult { - self.catalog_manager.create_schema(&schema).await + async fn drop_database_v2( + &self, + catalog_controller: &CatalogControllerRef, + database_id: DatabaseId, + ) -> MetaResult { + let ( + ReleaseContext { + streaming_jobs, + source_ids, + connections, + .. + }, + version, + ) = catalog_controller.drop_database(database_id as _).await?; + self.source_manager + .unregister_sources(source_ids.into_iter().map(|id| id as _).collect()) + .await; + self.stream_manager + .drop_streaming_jobs( + streaming_jobs + .into_iter() + .map(|id| (id as u32).into()) + .collect(), + ) + .await; + for svc in connections { + self.delete_vpc_endpoint_v2(svc.into_inner()).await?; + } + Ok(version) + } + + async fn drop_database(&self, database_id: DatabaseId) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + self.drop_database_v1(&mgr.catalog_manager, database_id) + .await + } + MetadataManager::V2(mgr) => { + self.drop_database_v2(&mgr.catalog_controller, database_id) + .await + } + } + } + + async fn create_schema(&self, mut schema: Schema) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + schema.id = self.gen_unique_id::<{ IdCategory::Schema }>().await?; + mgr.catalog_manager.create_schema(&schema).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_schema(schema).await, + } } async fn drop_schema(&self, schema_id: SchemaId) -> MetaResult { - self.catalog_manager.drop_schema(schema_id).await + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.drop_schema(schema_id).await, + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .drop_schema(schema_id as _, DropMode::Restrict) + .await + } + } } async fn create_source(&self, mut source: Source) -> MetaResult { - // set the initialized_at_epoch to the current epoch. - source.initialized_at_epoch = Some(Epoch::now().0); - source.initialized_at_cluster_version = Some(current_cluster_version()); + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + source.id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + // set the initialized_at_epoch to the current epoch. + source.initialized_at_epoch = Some(Epoch::now().0); + source.initialized_at_cluster_version = Some(current_cluster_version()); + + mgr.catalog_manager + .start_create_source_procedure(&source) + .await?; - self.catalog_manager - .start_create_source_procedure(&source) - .await?; + if let Err(e) = self.source_manager.register_source(&source).await { + mgr.catalog_manager + .cancel_create_source_procedure(&source) + .await?; + return Err(e); + } - if let Err(e) = self.source_manager.register_source(&source).await { - self.catalog_manager - .cancel_create_source_procedure(&source) - .await?; - return Err(e); + mgr.catalog_manager + .finish_create_source_procedure(source, vec![]) + .await + } + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .create_source(source, Some(self.source_manager.clone())) + .await + } } - - self.catalog_manager - .finish_create_source_procedure(source, vec![]) - .await } async fn drop_source( @@ -395,12 +481,15 @@ impl DdlController { source_id: SourceId, drop_mode: DropMode, ) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support drop source in v2"); + }; // 1. Drop source in catalog. - let (version, streaming_job_ids) = self + let (version, streaming_job_ids) = mgr .catalog_manager .drop_relation( RelationIdEnum::Source(source_id), - self.fragment_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await?; @@ -420,19 +509,39 @@ impl DdlController { // Maybe we can unify `alter_source_column` and `alter_source_name`. async fn alter_source_column(&self, source: Source) -> MetaResult { - self.catalog_manager.alter_source_column(source).await + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.alter_source_column(source).await, + MetadataManager::V2(mgr) => mgr.catalog_controller.alter_source_column(source).await, + } } - async fn create_function(&self, function: Function) -> MetaResult { - self.catalog_manager.create_function(&function).await + async fn create_function(&self, mut function: Function) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + function.id = self.gen_unique_id::<{ IdCategory::Function }>().await?; + mgr.catalog_manager.create_function(&function).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_function(function).await, + } } async fn drop_function(&self, function_id: FunctionId) -> MetaResult { - self.catalog_manager.drop_function(function_id).await + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.drop_function(function_id).await, + MetadataManager::V2(mgr) => { + mgr.catalog_controller.drop_function(function_id as _).await + } + } } - async fn create_view(&self, view: View) -> MetaResult { - self.catalog_manager.create_view(&view).await + async fn create_view(&self, mut view: View) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + view.id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + mgr.catalog_manager.create_view(&view).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_view(view).await, + } } async fn drop_view( @@ -440,11 +549,14 @@ impl DdlController { view_id: ViewId, drop_mode: DropMode, ) -> MetaResult { - let (version, streaming_job_ids) = self + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support drop view in v2"); + }; + let (version, streaming_job_ids) = mgr .catalog_manager .drop_relation( RelationIdEnum::View(view_id), - self.fragment_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await?; @@ -454,15 +566,32 @@ impl DdlController { Ok(version) } - async fn create_connection(&self, connection: Connection) -> MetaResult { - self.catalog_manager.create_connection(connection).await + async fn create_connection( + &self, + mut connection: Connection, + ) -> MetaResult { + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + connection.id = self.gen_unique_id::<{ IdCategory::Connection }>().await?; + mgr.catalog_manager.create_connection(connection).await + } + MetadataManager::V2(mgr) => mgr.catalog_controller.create_connection(connection).await, + } } async fn drop_connection( &self, connection_id: ConnectionId, ) -> MetaResult { - let (version, connection) = self.catalog_manager.drop_connection(connection_id).await?; + let (version, connection) = match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.drop_connection(connection_id).await?, + MetadataManager::V2(mgr) => { + mgr.catalog_controller + .drop_connection(connection_id as _) + .await? + } + }; + self.delete_vpc_endpoint(&connection).await?; Ok(version) } @@ -484,13 +613,53 @@ impl DdlController { Ok(()) } + async fn delete_vpc_endpoint_v2(&self, svc: PrivateLinkService) -> MetaResult<()> { + // delete AWS vpc endpoint + if svc.get_provider()? == PbPrivateLinkProvider::Aws { + if let Some(aws_cli) = self.aws_client.as_ref() { + aws_cli.delete_vpc_endpoint(&svc.endpoint_id).await?; + } else { + warn!( + "AWS client is not initialized, skip deleting vpc endpoint {}", + svc.endpoint_id + ); + } + } + Ok(()) + } + async fn create_streaming_job( &self, mut stream_job: StreamingJob, - fragment_graph: StreamFragmentGraphProto, + mut fragment_graph: StreamFragmentGraphProto, create_type: CreateType, affected_table_replace_info: Option, ) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support create streaming job in v2"); + }; + let id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + stream_job.set_id(id); + + match &mut stream_job { + StreamingJob::Table(Some(src), table, job_type) => { + // If we're creating a table with connector, we should additionally fill its ID first. + src.id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + fill_table_stream_graph_info(src, table, *job_type, &mut fragment_graph); + } + StreamingJob::Source(_) => { + // set the inner source id of source node. + for fragment in fragment_graph.fragments.values_mut() { + visit_fragment(fragment, |node_body| { + if let NodeBody::Source(source_node) = node_body { + source_node.source_inner.as_mut().unwrap().source_id = id; + } + }); + } + } + _ => {} + } + tracing::debug!( id = stream_job.id(), definition = stream_job.definition(), @@ -507,14 +676,23 @@ impl DdlController { let stream_ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); tracing::debug!(id = stream_job.id(), "preparing stream job"); - let fragment_graph = self - .prepare_stream_job(&mut stream_job, fragment_graph) - .await?; - // Update the corresponding 'initiated_at' field. + // 1. Build fragment graph. + let fragment_graph = + StreamFragmentGraph::new(&self.env, fragment_graph, &stream_job).await?; + let internal_tables = fragment_graph.internal_tables().into_values().collect_vec(); + + // 2. Set the graph-related fields and freeze the `stream_job`. + stream_job.set_table_fragment_id(fragment_graph.table_fragment_id()); + stream_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); stream_job.mark_initialized(); - let mut internal_tables = vec![]; + // 3. Persist tables. + mgr.catalog_manager + .start_create_stream_job_procedure(&stream_job, internal_tables.clone()) + .await?; + + // 4. Build and persist stream job. let result = try { tracing::debug!(id = stream_job.id(), "building stream job"); let (ctx, table_fragments) = self @@ -526,8 +704,6 @@ impl DdlController { ) .await?; - internal_tables = ctx.internal_tables(); - // Do some type-specific work for each type of stream job. match stream_job { StreamingJob::Table(None, ref table, TableJobType::SharedCdcSource) => { @@ -556,6 +732,7 @@ impl DdlController { } _ => {} } + (ctx, table_fragments) }; @@ -570,15 +747,23 @@ impl DdlController { match create_type { CreateType::Foreground | CreateType::Unspecified => { - self.create_streaming_job_inner(stream_job, table_fragments, ctx, internal_tables) - .await + self.create_streaming_job_inner( + mgr, + stream_job, + table_fragments, + ctx, + internal_tables, + ) + .await } CreateType::Background => { let ctrl = self.clone(); + let mgr = mgr.clone(); let stream_job_id = stream_job.id(); let fut = async move { let result = ctrl .create_streaming_job_inner( + &mgr, stream_job, table_fragments, ctx, @@ -601,7 +786,10 @@ impl DdlController { } /// Validates the connect properties in the `cdc_table_desc` stored in the `StreamCdcScan` node - async fn validate_cdc_table(table: &Table, table_fragments: &TableFragments) -> MetaResult<()> { + pub(crate) async fn validate_cdc_table( + table: &Table, + table_fragments: &TableFragments, + ) -> MetaResult<()> { let stream_scan_fragment = table_fragments .fragments .values() @@ -647,6 +835,7 @@ impl DdlController { // Meanwhile, the Dispatcher corresponding to the upstream of the merge will also be added to the replace table context here. async fn inject_replace_table_job_for_table_sink( &self, + mgr: &MetadataManagerV1, stream_ctx: StreamContext, sink: Option<&Sink>, creating_sink_table_fragments: Option<&TableFragments>, @@ -658,11 +847,15 @@ impl DdlController { }: ReplaceTableInfo, ) -> MetaResult<(StreamingJob, ReplaceTableContext, TableFragments)> { let fragment_graph = self - .prepare_replace_table(&mut streaming_job, fragment_graph) + .prepare_replace_table( + mgr.catalog_manager.clone(), + &mut streaming_job, + fragment_graph, + ) .await?; let (mut replace_table_ctx, mut table_fragments) = self - .build_replace_table(stream_ctx, &streaming_job, fragment_graph, None) + .build_replace_table(mgr, stream_ctx, &streaming_job, fragment_graph, None) .await?; let mut union_fragment_id = None; @@ -702,7 +895,7 @@ impl DdlController { ); } - let [table_catalog]: [_; 1] = self + let [table_catalog]: [_; 1] = mgr .catalog_manager .get_tables(&[table.id]) .await @@ -712,7 +905,7 @@ impl DdlController { assert_eq!(table_catalog.incoming_sinks, table.incoming_sinks); { - let guard = self.fragment_manager.get_fragment_read_guard().await; + let guard = mgr.fragment_manager.get_fragment_read_guard().await; for sink_id in &table_catalog.incoming_sinks { if let Some(dropping_sink_id) = dropping_sink_id @@ -859,6 +1052,7 @@ impl DdlController { /// Let the stream manager to create the actors, and do some cleanup work after it fails or finishes. async fn create_streaming_job_inner( &self, + mgr: &MetadataManagerV1, stream_job: StreamingJob, table_fragments: TableFragments, ctx: CreateStreamingJobContext, @@ -867,22 +1061,31 @@ impl DdlController { let job_id = stream_job.id(); tracing::debug!(id = job_id, "creating stream job"); - let result = self - .stream_manager - .create_streaming_job(table_fragments, ctx) - .await; + let result = try { + // Add table fragments to meta store with state: `State::Initial`. + mgr.fragment_manager + .start_create_table_fragments(table_fragments.clone()) + .await?; + + self.stream_manager + .create_streaming_job(table_fragments, ctx) + .await? + }; if let Err(e) = result { match stream_job.create_type() { CreateType::Background => { tracing::error!(id = job_id, error = ?e, "finish stream job failed"); - if let Err(err) = self + let should_cancel = match mgr .fragment_manager .select_table_fragments_by_table_id(&job_id.into()) .await - && err.is_fragment_not_found() { - // If the table fragments are not found, it means that the stream job has not been created. + Err(err) => err.is_fragment_not_found(), + Ok(table_fragments) => table_fragments.is_initial(), + }; + if should_cancel { + // If the table fragments are not found or in initial state, it means that the stream job has not been created. // We need to cancel the stream job. self.cancel_stream_job(&stream_job, internal_tables, Some(&e)) .await?; @@ -900,7 +1103,9 @@ impl DdlController { }; tracing::debug!(id = job_id, "finishing stream job"); - let version = self.finish_stream_job(stream_job, internal_tables).await?; + let version = self + .finish_stream_job(mgr, stream_job, internal_tables) + .await?; tracing::debug!(id = job_id, "finished stream job"); Ok(version) @@ -912,22 +1117,25 @@ impl DdlController { drop_mode: DropMode, target_replace_info: Option, ) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support drop streaming job in v2"); + }; let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; let (mut version, streaming_job_ids) = match job_id { StreamingJobId::MaterializedView(table_id) => { - self.catalog_manager + mgr.catalog_manager .drop_relation( RelationIdEnum::Table(table_id), - self.fragment_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await? } StreamingJobId::Sink(sink_id) => { - self.catalog_manager + mgr.catalog_manager .drop_relation( RelationIdEnum::Sink(sink_id), - self.fragment_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await? @@ -936,16 +1144,17 @@ impl DdlController { self.drop_table_inner( source_id, table_id, - self.fragment_manager.clone(), + mgr.catalog_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await? } StreamingJobId::Index(index_id) => { - self.catalog_manager + mgr.catalog_manager .drop_relation( RelationIdEnum::Index(index_id), - self.fragment_manager.clone(), + mgr.fragment_manager.clone(), drop_mode, ) .await? @@ -962,6 +1171,7 @@ impl DdlController { let (streaming_job, context, table_fragments) = self .inject_replace_table_job_for_table_sink( + mgr, stream_ctx, None, None, @@ -975,7 +1185,13 @@ impl DdlController { .await?; version = self - .finish_replace_table(&streaming_job, None, None, Some(sink_id)) + .finish_replace_table( + mgr.catalog_manager.clone(), + &streaming_job, + None, + None, + Some(sink_id), + ) .await?; } @@ -986,34 +1202,6 @@ impl DdlController { Ok(version) } - /// Creates [`StreamFragmentGraph`] from the protobuf message - /// (allocate and fill ID for fragments, internal tables, and the table in the local graph), - /// and does some preparation work. - async fn prepare_stream_job( - &self, - stream_job: &mut StreamingJob, - fragment_graph: StreamFragmentGraphProto, - ) -> MetaResult { - // 1. Build fragment graph. - let fragment_graph = - StreamFragmentGraph::new(fragment_graph, self.env.id_gen_manager_ref(), stream_job) - .await?; - - let internal_tables = fragment_graph.internal_tables().into_values().collect_vec(); - - // 2. Set the graph-related fields and freeze the `stream_job`. - stream_job.set_table_fragment_id(fragment_graph.table_fragment_id()); - stream_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); - let stream_job = &*stream_job; - - // 3. Mark current relation as "creating" and add reference count to dependent relations. - self.catalog_manager - .start_create_stream_job_procedure(stream_job, internal_tables) - .await?; - - Ok(fragment_graph) - } - fn resolve_stream_parallelism( &self, default_parallelism: Option, @@ -1056,7 +1244,7 @@ impl DdlController { /// Builds the actor graph: /// - Schedule the fragments based on their distribution /// - Expand each fragment into one or several actors - async fn build_stream_job( + pub(crate) async fn build_stream_job( &self, stream_ctx: StreamContext, stream_job: &StreamingJob, @@ -1072,7 +1260,7 @@ impl DdlController { // contains all information needed for building the actor graph. let upstream_root_fragments = self - .fragment_manager + .metadata_manager .get_upstream_root_fragments( fragment_graph.dependent_table_ids(), stream_job.table_job_type(), @@ -1096,7 +1284,7 @@ impl DdlController { )?; // 2. Build the actor graph. - let cluster_info = self.cluster_manager.get_streaming_cluster_info().await; + let cluster_info = self.metadata_manager.get_streaming_cluster_info().await?; let parallelism = self.resolve_stream_parallelism(default_parallelism, &cluster_info)?; @@ -1110,7 +1298,7 @@ impl DdlController { dispatchers, merge_updates, } = actor_graph_builder - .generate_graph(self.env.id_gen_manager_ref(), stream_job, expr_context) + .generate_graph(&self.env, stream_job, expr_context) .await?; assert!(merge_updates.is_empty()); @@ -1136,9 +1324,13 @@ impl DdlController { let StreamingJob::Sink(s, _) = stream_job else { bail!("additional replace table event only occurs when sinking into table"); }; + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support replace table in v2"); + }; Some( self.inject_replace_table_job_for_table_sink( + mgr, stream_ctx, Some(s), Some(&table_fragments), @@ -1172,9 +1364,11 @@ impl DdlController { .chain(stream_job.table().cloned()) .collect_vec(); - self.catalog_manager - .mark_creating_tables(&creating_tables) - .await; + if let MetadataManager::V1(mgr) = &self.metadata_manager { + mgr.catalog_manager + .mark_creating_tables(&creating_tables) + .await; + } Ok((ctx, table_fragments)) } @@ -1187,6 +1381,9 @@ impl DdlController { internal_tables: Vec
, error: Option<&impl ToString>, ) -> MetaResult<()> { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support cancel streaming job in v2"); + }; let error = error.map(ToString::to_string).unwrap_or_default(); let event = risingwave_pb::meta::event_log::EventCreateStreamJobFail { id: stream_job.id(), @@ -1204,7 +1401,7 @@ impl DdlController { match stream_job { StreamingJob::MaterializedView(table) => { // barrier manager will do the cleanup. - let result = self + let result = mgr .catalog_manager .cancel_create_table_procedure(table.id, creating_internal_table_ids.clone()) .await; @@ -1214,17 +1411,17 @@ impl DdlController { } } StreamingJob::Sink(sink, target_table) => { - self.catalog_manager + mgr.catalog_manager .cancel_create_sink_procedure(sink, target_table) .await; } StreamingJob::Table(source, table, ..) => { if let Some(source) = source { - self.catalog_manager + mgr.catalog_manager .cancel_create_table_procedure_with_source(source, table) .await; } else { - let result = self + let result = mgr .catalog_manager .cancel_create_table_procedure( table.id, @@ -1239,18 +1436,18 @@ impl DdlController { } StreamingJob::Index(index, table) => { creating_internal_table_ids.push(table.id); - self.catalog_manager + mgr.catalog_manager .cancel_create_index_procedure(index, table) .await; } StreamingJob::Source(source) => { - self.catalog_manager + mgr.catalog_manager .cancel_create_source_procedure(source) .await?; } } // 2. unmark creating tables. - self.catalog_manager + mgr.catalog_manager .unmark_creating_tables(&creating_internal_table_ids, true) .await; Ok(()) @@ -1259,6 +1456,7 @@ impl DdlController { /// `finish_stream_job` finishes a stream job and clean some states. async fn finish_stream_job( &self, + mgr: &MetadataManagerV1, mut stream_job: StreamingJob, internal_tables: Vec
, ) -> MetaResult { @@ -1271,14 +1469,14 @@ impl DdlController { let version = match stream_job { StreamingJob::MaterializedView(table) => { creating_internal_table_ids.push(table.id); - self.catalog_manager + mgr.catalog_manager .finish_create_table_procedure(internal_tables, table) .await? } StreamingJob::Sink(sink, target_table) => { let sink_id = sink.id; - let mut version = self + let mut version = mgr .catalog_manager .finish_create_sink_procedure(internal_tables, sink) .await?; @@ -1288,7 +1486,13 @@ impl DdlController { StreamingJob::Table(source, table, TableJobType::Unspecified); version = self - .finish_replace_table(&streaming_job, None, Some(sink_id), None) + .finish_replace_table( + mgr.catalog_manager.clone(), + &streaming_job, + None, + Some(sink_id), + None, + ) .await?; } @@ -1297,30 +1501,30 @@ impl DdlController { StreamingJob::Table(source, table, ..) => { creating_internal_table_ids.push(table.id); if let Some(source) = source { - self.catalog_manager + mgr.catalog_manager .finish_create_table_procedure_with_source(source, table, internal_tables) .await? } else { - self.catalog_manager + mgr.catalog_manager .finish_create_table_procedure(internal_tables, table) .await? } } StreamingJob::Index(index, table) => { creating_internal_table_ids.push(table.id); - self.catalog_manager + mgr.catalog_manager .finish_create_index_procedure(internal_tables, index, table) .await? } StreamingJob::Source(source) => { - self.catalog_manager + mgr.catalog_manager .finish_create_source_procedure(source, internal_tables) .await? } }; // 2. unmark creating tables. - self.catalog_manager + mgr.catalog_manager .unmark_creating_tables(&creating_internal_table_ids, false) .await; @@ -1331,6 +1535,7 @@ impl DdlController { &self, source_id: Option, table_id: TableId, + catalog_manager: CatalogManagerRef, fragment_manager: FragmentManagerRef, drop_mode: DropMode, ) -> MetaResult<( @@ -1340,8 +1545,7 @@ impl DdlController { if let Some(source_id) = source_id { // Drop table and source in catalog. Check `source_id` if it is the table's // `associated_source_id`. Indexes also need to be dropped atomically. - let (version, delete_jobs) = self - .catalog_manager + let (version, delete_jobs) = catalog_manager .drop_relation( RelationIdEnum::Table(table_id), fragment_manager.clone(), @@ -1354,7 +1558,7 @@ impl DdlController { .await; Ok((version, delete_jobs)) } else { - self.catalog_manager + catalog_manager .drop_relation(RelationIdEnum::Table(table_id), fragment_manager, drop_mode) .await } @@ -1366,16 +1570,20 @@ impl DdlController { fragment_graph: StreamFragmentGraphProto, table_col_index_mapping: Option, ) -> MetaResult { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support replace table in v2"); + }; let _reschedule_job_lock = self.stream_manager.reschedule_lock.read().await; let stream_ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); let fragment_graph = self - .prepare_replace_table(&mut stream_job, fragment_graph) + .prepare_replace_table(mgr.catalog_manager.clone(), &mut stream_job, fragment_graph) .await?; let result = try { let (ctx, table_fragments) = self .build_replace_table( + mgr, stream_ctx, &stream_job, fragment_graph, @@ -1383,6 +1591,11 @@ impl DdlController { ) .await?; + // Add table fragments to meta store with state: `State::Initial`. + mgr.fragment_manager + .start_create_table_fragments(table_fragments.clone()) + .await?; + self.stream_manager .replace_table(table_fragments, ctx) .await?; @@ -1390,11 +1603,18 @@ impl DdlController { match result { Ok(_) => { - self.finish_replace_table(&stream_job, table_col_index_mapping, None, None) - .await + self.finish_replace_table( + mgr.catalog_manager.clone(), + &stream_job, + table_col_index_mapping, + None, + None, + ) + .await } Err(err) => { - self.cancel_replace_table(&stream_job).await?; + self.cancel_replace_table(mgr.catalog_manager.clone(), &stream_job) + .await?; Err(err) } } @@ -1405,13 +1625,13 @@ impl DdlController { /// assertions and uses a different method to mark in the catalog. async fn prepare_replace_table( &self, + catalog_manager: CatalogManagerRef, stream_job: &mut StreamingJob, fragment_graph: StreamFragmentGraphProto, ) -> MetaResult { // 1. Build fragment graph. let fragment_graph = - StreamFragmentGraph::new(fragment_graph, self.env.id_gen_manager_ref(), stream_job) - .await?; + StreamFragmentGraph::new(&self.env, fragment_graph, stream_job).await?; // 2. Set the graph-related fields and freeze the `stream_job`. stream_job.set_table_fragment_id(fragment_graph.table_fragment_id()); @@ -1419,7 +1639,7 @@ impl DdlController { let stream_job = &*stream_job; // 3. Mark current relation as "updating". - self.catalog_manager + catalog_manager .start_replace_table_procedure(stream_job) .await?; @@ -1430,6 +1650,7 @@ impl DdlController { /// fragments. async fn build_replace_table( &self, + mgr: &MetadataManagerV1, stream_ctx: StreamContext, stream_job: &StreamingJob, mut fragment_graph: StreamFragmentGraph, @@ -1439,12 +1660,12 @@ impl DdlController { let default_parallelism = fragment_graph.default_parallelism(); let expr_context = stream_ctx.to_expr_context(); - let old_table_fragments = self + let old_table_fragments = mgr .fragment_manager .select_table_fragments_by_table_id(&id.into()) .await?; let old_internal_table_ids = old_table_fragments.internal_table_ids(); - let old_internal_tables = self + let old_internal_tables = mgr .catalog_manager .get_tables(&old_internal_table_ids) .await; @@ -1453,10 +1674,10 @@ impl DdlController { // 1. Resolve the edges to the downstream fragments, extend the fragment graph to a complete // graph that contains all information needed for building the actor graph. - let original_table_fragment = self.fragment_manager.get_mview_fragment(id.into()).await?; + let original_table_fragment = mgr.fragment_manager.get_mview_fragment(id.into()).await?; // Map the column indices in the dispatchers with the given mapping. - let downstream_fragments = self + let downstream_fragments = mgr .fragment_manager .get_downstream_fragments(id.into()) .await? @@ -1482,7 +1703,7 @@ impl DdlController { )?; // 2. Build the actor graph. - let cluster_info = self.cluster_manager.get_streaming_cluster_info().await; + let cluster_info = mgr.cluster_manager.get_streaming_cluster_info().await; let parallelism = self.resolve_stream_parallelism(default_parallelism, &cluster_info)?; let actor_graph_builder = @@ -1495,7 +1716,7 @@ impl DdlController { dispatchers, merge_updates, } = actor_graph_builder - .generate_graph(self.env.id_gen_manager_ref(), stream_job, expr_context) + .generate_graph(&self.env, stream_job, expr_context) .await?; assert!(dispatchers.is_empty()); @@ -1541,6 +1762,7 @@ impl DdlController { async fn finish_replace_table( &self, + catalog_manager: CatalogManagerRef, stream_job: &StreamingJob, table_col_index_mapping: Option, creating_sink_id: Option, @@ -1550,7 +1772,7 @@ impl DdlController { unreachable!("unexpected job: {stream_job:?}") }; - self.catalog_manager + catalog_manager .finish_replace_table_procedure( source, table, @@ -1561,8 +1783,12 @@ impl DdlController { .await } - async fn cancel_replace_table(&self, stream_job: &StreamingJob) -> MetaResult<()> { - self.catalog_manager + async fn cancel_replace_table( + &self, + catalog_manager: CatalogManagerRef, + stream_job: &StreamingJob, + ) -> MetaResult<()> { + catalog_manager .cancel_replace_table_procedure(stream_job) .await } @@ -1572,40 +1798,58 @@ impl DdlController { relation: alter_name_request::Object, new_name: &str, ) -> MetaResult { - match relation { - alter_name_request::Object::TableId(table_id) => { - self.catalog_manager - .alter_table_name(table_id, new_name) - .await - } - alter_name_request::Object::ViewId(view_id) => { - self.catalog_manager - .alter_view_name(view_id, new_name) - .await - } - alter_name_request::Object::IndexId(index_id) => { - self.catalog_manager - .alter_index_name(index_id, new_name) - .await - } - alter_name_request::Object::SinkId(sink_id) => { - self.catalog_manager - .alter_sink_name(sink_id, new_name) - .await - } - alter_name_request::Object::SourceId(source_id) => { - self.catalog_manager - .alter_source_name(source_id, new_name) - .await - } - alter_name_request::Object::SchemaId(schema_id) => { - self.catalog_manager - .alter_schema_name(schema_id, new_name) - .await - } - alter_name_request::Object::DatabaseId(database_id) => { - self.catalog_manager - .alter_database_name(database_id, new_name) + match &self.metadata_manager { + MetadataManager::V1(mgr) => match relation { + alter_name_request::Object::TableId(table_id) => { + mgr.catalog_manager + .alter_table_name(table_id, new_name) + .await + } + alter_name_request::Object::ViewId(view_id) => { + mgr.catalog_manager.alter_view_name(view_id, new_name).await + } + alter_name_request::Object::IndexId(index_id) => { + mgr.catalog_manager + .alter_index_name(index_id, new_name) + .await + } + alter_name_request::Object::SinkId(sink_id) => { + mgr.catalog_manager.alter_sink_name(sink_id, new_name).await + } + alter_name_request::Object::SourceId(source_id) => { + mgr.catalog_manager + .alter_source_name(source_id, new_name) + .await + } + alter_name_request::Object::SchemaId(schema_id) => { + mgr.catalog_manager + .alter_schema_name(schema_id, new_name) + .await + } + alter_name_request::Object::DatabaseId(database_id) => { + mgr.catalog_manager + .alter_database_name(database_id, new_name) + .await + } + }, + MetadataManager::V2(mgr) => { + let (obj_type, id) = match relation { + alter_name_request::Object::TableId(id) => (ObjectType::Table, id as ObjectId), + alter_name_request::Object::ViewId(id) => (ObjectType::View, id as ObjectId), + alter_name_request::Object::IndexId(id) => (ObjectType::Index, id as ObjectId), + alter_name_request::Object::SinkId(id) => (ObjectType::Sink, id as ObjectId), + alter_name_request::Object::SourceId(id) => { + (ObjectType::Source, id as ObjectId) + } + alter_name_request::Object::SchemaId(id) => { + (ObjectType::Schema, id as ObjectId) + } + alter_name_request::Object::DatabaseId(id) => { + (ObjectType::Database, id as ObjectId) + } + }; + mgr.catalog_controller + .alter_name(obj_type, id, new_name) .await } } @@ -1616,9 +1860,26 @@ impl DdlController { object: Object, owner_id: UserId, ) -> MetaResult { - self.catalog_manager - .alter_owner(self.fragment_manager.clone(), object, owner_id) - .await + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .alter_owner(mgr.fragment_manager.clone(), object, owner_id) + .await + } + MetadataManager::V2(mgr) => { + let (obj_type, id) = match object { + Object::TableId(id) => (ObjectType::Table, id as ObjectId), + Object::ViewId(id) => (ObjectType::View, id as ObjectId), + Object::SourceId(id) => (ObjectType::Source, id as ObjectId), + Object::SinkId(id) => (ObjectType::Sink, id as ObjectId), + Object::SchemaId(id) => (ObjectType::Schema, id as ObjectId), + Object::DatabaseId(id) => (ObjectType::Database, id as ObjectId), + }; + mgr.catalog_controller + .alter_owner(obj_type, id, owner_id as _) + .await + } + } } async fn alter_set_schema( @@ -1626,22 +1887,66 @@ impl DdlController { object: alter_set_schema_request::Object, new_schema_id: SchemaId, ) -> MetaResult { - self.catalog_manager - .alter_set_schema(self.fragment_manager.clone(), object, new_schema_id) - .await + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + mgr.catalog_manager + .alter_set_schema(mgr.fragment_manager.clone(), object, new_schema_id) + .await + } + MetadataManager::V2(mgr) => { + let (obj_type, id) = match object { + alter_set_schema_request::Object::TableId(id) => { + (ObjectType::Table, id as ObjectId) + } + alter_set_schema_request::Object::ViewId(id) => { + (ObjectType::View, id as ObjectId) + } + alter_set_schema_request::Object::SourceId(id) => { + (ObjectType::Source, id as ObjectId) + } + alter_set_schema_request::Object::SinkId(id) => { + (ObjectType::Sink, id as ObjectId) + } + alter_set_schema_request::Object::FunctionId(id) => { + (ObjectType::Function, id as ObjectId) + } + alter_set_schema_request::Object::ConnectionId(id) => { + (ObjectType::Connection, id as ObjectId) + } + }; + mgr.catalog_controller + .alter_schema(obj_type, id, new_schema_id as _) + .await + } + } } pub async fn wait(&self) -> MetaResult<()> { let timeout_secs = 30 * 60; for _ in 0..timeout_secs { - if self - .catalog_manager - .list_creating_background_mvs() - .await - .is_empty() - { - return Ok(()); + match &self.metadata_manager { + MetadataManager::V1(mgr) => { + if mgr + .catalog_manager + .list_creating_background_mvs() + .await + .is_empty() + { + return Ok(()); + } + } + MetadataManager::V2(mgr) => { + if mgr + .catalog_controller + .list_background_creating_mviews() + .await? + .is_empty() + { + return Ok(()); + } + } } + sleep(Duration::from_secs(1)).await; } Err(MetaError::cancelled(format!( @@ -1650,6 +1955,72 @@ impl DdlController { } async fn comment_on(&self, comment: Comment) -> MetaResult { - self.catalog_manager.comment_on(comment).await + match &self.metadata_manager { + MetadataManager::V1(mgr) => mgr.catalog_manager.comment_on(comment).await, + MetadataManager::V2(mgr) => mgr.catalog_controller.comment_on(comment).await, + } + } +} + +/// Fill in necessary information for table stream graph. +pub fn fill_table_stream_graph_info( + source: &mut PbSource, + table: &mut PbTable, + table_job_type: TableJobType, + fragment_graph: &mut PbStreamFragmentGraph, +) { + let mut source_count = 0; + // Fill in the correct table id for source. + source.optional_associated_table_id = + Some(OptionalAssociatedTableId::AssociatedTableId(table.id)); + // Fill in the correct source id for mview. + table.optional_associated_source_id = + Some(OptionalAssociatedSourceId::AssociatedSourceId(source.id)); + + for fragment in fragment_graph.fragments.values_mut() { + visit_fragment(fragment, |node_body| { + if let NodeBody::Source(source_node) = node_body { + if source_node.source_inner.is_none() { + // skip empty source for dml node + return; + } + + // If we're creating a table with connector, we should additionally fill its ID first. + source_node.source_inner.as_mut().unwrap().source_id = source.id; + source_count += 1; + + // Generate a random server id for mysql cdc source if needed + // `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication + // group (that is, different from any other server id being used by any master or slave) + if let Some(connector) = source.with_properties.get(UPSTREAM_SOURCE_KEY) + && matches!( + CdcSourceType::from(connector.as_str()), + CdcSourceType::Mysql + ) + { + let props = &mut source_node.source_inner.as_mut().unwrap().with_properties; + let rand_server_id = rand::thread_rng().gen_range(1..u32::MAX); + props + .entry("server.id".to_string()) + .or_insert(rand_server_id.to_string()); + + // make these two `Source` consistent + props.clone_into(&mut source.with_properties); + } + } + + // fill table id for cdc backfill + if let NodeBody::StreamCdcScan(node) = node_body + && table_job_type == TableJobType::SharedCdcSource + { + if let Some(table_desc) = node.cdc_table_desc.as_mut() { + table_desc.table_id = table.id; + } + } + }); } + assert_eq!( + source_count, 1, + "require exactly 1 external stream source when creating table with a connector" + ); } diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 198e74c4bb794..086d8656839f2 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -34,11 +34,15 @@ use risingwave_object_store::object::object_metrics::{ }; use risingwave_pb::common::WorkerType; use risingwave_pb::stream_plan::stream_node::NodeBody::Sink; +use thiserror_ext::AsReport; use tokio::sync::oneshot::Sender; use tokio::task::JoinHandle; +use crate::controller::catalog::CatalogControllerRef; +use crate::controller::cluster::ClusterControllerRef; +use crate::controller::utils::PartialFragmentStateTables; use crate::hummock::HummockManagerRef; -use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef}; +use crate::manager::MetadataManager; use crate::rpc::ElectionClientRef; #[derive(Clone)] @@ -693,7 +697,7 @@ impl Default for MetaMetrics { } pub fn start_worker_info_monitor( - cluster_manager: ClusterManagerRef, + metadata_manager: MetadataManager, election_client: Option, interval: Duration, meta_metrics: Arc, @@ -713,7 +717,15 @@ pub fn start_worker_info_monitor( } } - for (worker_type, worker_num) in cluster_manager.count_worker_node().await { + let node_map = match metadata_manager.count_worker_node().await { + Ok(node_map) => node_map, + Err(err) => { + tracing::warn!(error = %err.as_report(), "fail to count worker node"); + continue; + } + }; + + for (worker_type, worker_num) in node_map { meta_metrics .worker_num .with_label_values(&[(worker_type.as_str_name())]) @@ -740,10 +752,130 @@ pub fn start_worker_info_monitor( (join_handle, shutdown_tx) } +pub async fn refresh_fragment_info_metrics_v2( + catalog_controller: &CatalogControllerRef, + cluster_controller: &ClusterControllerRef, + hummock_manager: &HummockManagerRef, + meta_metrics: Arc, +) { + let worker_nodes = match cluster_controller + .list_workers(Some(WorkerType::ComputeNode.into()), None) + .await + { + Ok(worker_nodes) => worker_nodes, + Err(err) => { + tracing::warn!(error=%err.as_report(), "fail to list worker node"); + return; + } + }; + let actor_locations = match catalog_controller.list_actor_locations().await { + Ok(actor_locations) => actor_locations, + Err(err) => { + tracing::warn!(error=%err.as_report(), "fail to get actor locations"); + return; + } + }; + let sink_actor_mapping = match catalog_controller.list_sink_actor_mapping().await { + Ok(sink_actor_mapping) => sink_actor_mapping, + Err(err) => { + tracing::warn!(error=%err.as_report(), "fail to get sink actor mapping"); + return; + } + }; + let fragment_state_tables = match catalog_controller.list_fragment_state_tables().await { + Ok(fragment_state_tables) => fragment_state_tables, + Err(err) => { + tracing::warn!(error=%err.as_report(), "fail to get fragment state tables"); + return; + } + }; + let table_name_and_type_mapping = match catalog_controller.get_table_name_type_mapping().await { + Ok(mapping) => mapping, + Err(err) => { + tracing::warn!(error=%err.as_report(), "fail to get table name mapping"); + return; + } + }; + + let pu_addr_mapping: HashMap = worker_nodes + .into_iter() + .flat_map(|worker_node| { + let addr = match worker_node.host { + Some(host) => format!("{}:{}", host.host, host.port), + None => "".to_owned(), + }; + worker_node + .parallel_units + .into_iter() + .map(move |pu| (pu.id, addr.clone())) + }) + .collect(); + let table_compaction_group_id_mapping = hummock_manager + .get_table_compaction_group_id_mapping() + .await; + + // Start fresh with a reset to clear all outdated labels. This is safe since we always + // report full info on each interval. + meta_metrics.actor_info.reset(); + meta_metrics.table_info.reset(); + meta_metrics.sink_info.reset(); + for actor_location in actor_locations { + let actor_id_str = actor_location.actor_id.to_string(); + let fragment_id_str = actor_location.fragment_id.to_string(); + // Report a dummy gauge metrics with (fragment id, actor id, node + // address) as its label + if let Some(address) = pu_addr_mapping.get(&(actor_location.parallel_unit_id as u32)) { + meta_metrics + .actor_info + .with_label_values(&[&actor_id_str, &fragment_id_str, address]) + .set(1); + } + } + for (sink_id, (sink_name, actor_ids)) in sink_actor_mapping { + let sink_id_str = sink_id.to_string(); + for actor_id in actor_ids { + let actor_id_str = actor_id.to_string(); + meta_metrics + .sink_info + .with_label_values(&[&actor_id_str, &sink_id_str, &sink_name]) + .set(1); + } + } + for PartialFragmentStateTables { + fragment_id, + job_id, + state_table_ids, + } in fragment_state_tables + { + let fragment_id_str = fragment_id.to_string(); + let job_id_str = job_id.to_string(); + for table_id in state_table_ids.into_inner() { + let table_id_str = table_id.to_string(); + let (table_name, table_type) = table_name_and_type_mapping + .get(&table_id) + .cloned() + .unwrap_or_else(|| ("unknown".to_string(), "unknown".to_string())); + let compaction_group_id = table_compaction_group_id_mapping + .get(&(table_id as u32)) + .map(|cg_id| cg_id.to_string()) + .unwrap_or_else(|| "unknown".to_string()); + meta_metrics + .table_info + .with_label_values(&[ + &job_id_str, + &table_id_str, + &fragment_id_str, + &table_name, + &table_type, + &compaction_group_id, + ]) + .set(1); + } + } +} + pub fn start_fragment_info_monitor( - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manager: HummockManagerRef, meta_metrics: Arc, ) -> (JoinHandle<()>, Sender<()>) { @@ -765,10 +897,29 @@ pub fn start_fragment_info_monitor( } } + let (cluster_manager, catalog_manager, fragment_manager) = match &metadata_manager { + MetadataManager::V1(mgr) => ( + &mgr.cluster_manager, + &mgr.catalog_manager, + &mgr.fragment_manager, + ), + MetadataManager::V2(mgr) => { + refresh_fragment_info_metrics_v2( + &mgr.catalog_controller, + &mgr.cluster_controller, + &hummock_manager, + meta_metrics.clone(), + ) + .await; + continue; + } + }; + // Start fresh with a reset to clear all outdated labels. This is safe since we always // report full info on each interval. meta_metrics.actor_info.reset(); meta_metrics.table_info.reset(); + meta_metrics.sink_info.reset(); let workers: HashMap = cluster_manager .list_worker_node(Some(WorkerType::ComputeNode), None) .await diff --git a/src/meta/src/serving/mod.rs b/src/meta/src/serving/mod.rs index 02bfa6cbc07cf..7448f6f9496a9 100644 --- a/src/meta/src/serving/mod.rs +++ b/src/meta/src/serving/mod.rs @@ -24,9 +24,7 @@ use risingwave_pb::meta::{FragmentParallelUnitMapping, FragmentParallelUnitMappi use tokio::sync::oneshot::Sender; use tokio::task::JoinHandle; -use crate::manager::{ - ClusterManagerRef, FragmentManagerRef, LocalNotification, NotificationManagerRef, -}; +use crate::manager::{LocalNotification, MetadataManager, NotificationManagerRef}; use crate::model::FragmentId; pub type ServingVnodeMappingRef = Arc; @@ -105,15 +103,13 @@ pub(crate) fn to_deleted_fragment_parallel_unit_mapping( pub async fn on_meta_start( notification_manager: NotificationManagerRef, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: &MetadataManager, serving_vnode_mapping: ServingVnodeMappingRef, ) { - let streaming_parallelisms = fragment_manager.running_fragment_parallelisms(None).await; - let (mappings, _) = serving_vnode_mapping.upsert( - streaming_parallelisms, - &cluster_manager.list_active_serving_compute_nodes().await, - ); + let (serving_compute_nodes, streaming_parallelisms) = + fetch_serving_infos(metadata_manager).await; + let (mappings, _) = + serving_vnode_mapping.upsert(streaming_parallelisms, &serving_compute_nodes); tracing::debug!( "Initialize serving vnode mapping snapshot for fragments {:?}.", mappings.keys() @@ -126,10 +122,44 @@ pub async fn on_meta_start( ); } +async fn fetch_serving_infos( + metadata_manager: &MetadataManager, +) -> (Vec, HashMap) { + match metadata_manager { + MetadataManager::V1(mgr) => ( + mgr.cluster_manager + .list_active_serving_compute_nodes() + .await, + mgr.fragment_manager + .running_fragment_parallelisms(None) + .await, + ), + MetadataManager::V2(mgr) => { + // TODO: need another mechanism to refresh serving info instead of panic. + let parallelisms = mgr + .catalog_controller + .running_fragment_parallelisms(None) + .await + .expect("fail to fetch running parallelisms"); + let serving_compute_nodes = mgr + .cluster_controller + .list_active_serving_workers() + .await + .expect("fail to list serving compute nodes"); + ( + serving_compute_nodes, + parallelisms + .into_iter() + .map(|(fragment_id, cnt)| (fragment_id as FragmentId, cnt)) + .collect(), + ) + } + } +} + pub async fn start_serving_vnode_mapping_worker( notification_manager: NotificationManagerRef, - cluster_manager: ClusterManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, serving_vnode_mapping: ServingVnodeMappingRef, ) -> (JoinHandle<()>, Sender<()>) { let (local_notification_tx, mut local_notification_rx) = tokio::sync::mpsc::unbounded_channel(); @@ -148,8 +178,7 @@ pub async fn start_serving_vnode_mapping_worker( if w.r#type() != WorkerType::ComputeNode || !w.property.as_ref().map_or(false, |p| p.is_serving) { continue; } - let workers = cluster_manager.list_active_serving_compute_nodes().await; - let streaming_parallelisms = fragment_manager.running_fragment_parallelisms(None).await; + let (workers, streaming_parallelisms) = fetch_serving_infos(&metadata_manager).await; let (mappings, _) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); tracing::debug!("Update serving vnode mapping snapshot for fragments {:?}.", mappings.keys()); notification_manager.notify_frontend_without_version(Operation::Snapshot, Info::ServingParallelUnitMappings(FragmentParallelUnitMappings{ mappings: to_fragment_parallel_unit_mapping(&mappings) })); @@ -158,8 +187,7 @@ pub async fn start_serving_vnode_mapping_worker( if fragment_ids.is_empty() { continue; } - let workers = cluster_manager.list_active_serving_compute_nodes().await; - let streaming_parallelisms = fragment_manager.running_fragment_parallelisms(Some(fragment_ids.into_iter().collect())).await; + let (workers, streaming_parallelisms) = fetch_serving_infos(&metadata_manager).await; let (upserted, failed) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); if !upserted.is_empty() { tracing::debug!("Update serving vnode mapping for fragments {:?}.", upserted.keys()); diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index e17419ea23acb..d4792c0ba8e2c 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -50,7 +50,8 @@ use uuid::Uuid; use crate::barrier::{Command, Reschedule}; use crate::manager::{ - ClusterManagerRef, FragmentManagerRef, IdCategory, LocalNotification, MetaSrvEnv, WorkerId, + ClusterManagerRef, FragmentManagerRef, IdCategory, LocalNotification, MetaSrvEnv, + MetadataManager, WorkerId, }; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::serving::{ @@ -378,16 +379,18 @@ pub struct ScaleController { impl ScaleController { pub fn new( - fragment_manager: FragmentManagerRef, - cluster_manager: ClusterManagerRef, + metadata_manager: &MetadataManager, source_manager: SourceManagerRef, env: MetaSrvEnv, ) -> Self { - Self { - fragment_manager, - cluster_manager, - source_manager, - env, + match metadata_manager { + MetadataManager::V1(mgr) => Self { + fragment_manager: mgr.fragment_manager.clone(), + cluster_manager: mgr.cluster_manager.clone(), + source_manager, + env, + }, + MetadataManager::V2(_) => unimplemented!("support v2 in scale controller"), } } @@ -2393,10 +2396,16 @@ impl GlobalStreamManager { options: RescheduleOptions, table_parallelism: Option>, ) -> MetaResult<()> { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support reschedule in v2"); + }; + let mut table_parallelism = table_parallelism; let (reschedule_fragment, applied_reschedules) = self .scale_controller + .as_ref() + .unwrap() .prepare_reschedule_command(reschedules, options, table_parallelism.as_mut()) .await?; @@ -2407,7 +2416,7 @@ impl GlobalStreamManager { table_parallelism: table_parallelism.unwrap_or_default(), }; - let fragment_manager_ref = self.fragment_manager.clone(); + let fragment_manager_ref = mgr.fragment_manager.clone(); revert_funcs.push(Box::pin(async move { fragment_manager_ref @@ -2427,8 +2436,12 @@ impl GlobalStreamManager { async fn trigger_parallelism_control(&self) -> MetaResult<()> { let _reschedule_job_lock = self.reschedule_lock.write().await; + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support reschedule in v2"); + }; + let table_parallelisms = { - let guard = self.fragment_manager.get_fragment_read_guard().await; + let guard = mgr.fragment_manager.get_fragment_read_guard().await; guard .table_fragments() @@ -2437,7 +2450,7 @@ impl GlobalStreamManager { .collect() }; - let workers = self + let workers = mgr .cluster_manager .list_active_streaming_compute_nodes() .await; @@ -2456,6 +2469,8 @@ impl GlobalStreamManager { let reschedules = self .scale_controller + .as_ref() + .unwrap() .generate_table_resize_plan(TableResizePolicy { worker_ids: schedulable_worker_ids, table_parallelisms, @@ -2493,9 +2508,10 @@ impl GlobalStreamManager { ticker.reset(); let worker_nodes = self - .cluster_manager + .metadata_manager .list_active_streaming_compute_nodes() - .await; + .await + .expect("list active streaming compute nodes"); let mut worker_cache: BTreeMap<_, _> = worker_nodes .into_iter() diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 4d225e53bba3a..b5a1286804408 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -38,8 +38,8 @@ use tokio::time::MissedTickBehavior; use tokio::{select, time}; use crate::barrier::{BarrierScheduler, Command}; -use crate::manager::{CatalogManagerRef, FragmentManagerRef, MetaSrvEnv, SourceId}; -use crate::model::{ActorId, FragmentId, TableFragments}; +use crate::manager::{MetaSrvEnv, MetadataManager, SourceId}; +use crate::model::{ActorId, FragmentId, MetadataModel, TableFragments}; use crate::rpc::metrics::MetaMetrics; use crate::MetaResult; @@ -220,7 +220,7 @@ impl ConnectorSourceWorkerHandle { } pub struct SourceManagerCore { - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, /// Managed source loops managed_sources: HashMap, @@ -235,7 +235,7 @@ pub struct SourceManagerCore { impl SourceManagerCore { fn new( - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, managed_sources: HashMap, source_fragments: HashMap>, actor_splits: HashMap>, @@ -248,7 +248,7 @@ impl SourceManagerCore { } Self { - fragment_manager, + metadata_manager, managed_sources, source_fragments, fragment_sources, @@ -281,7 +281,7 @@ impl SourceManagerCore { for fragment_id in fragment_ids { let actor_ids = match self - .fragment_manager + .metadata_manager .get_running_actors_of_fragment(*fragment_id) .await { @@ -528,13 +528,12 @@ impl SourceManager { pub async fn new( env: MetaSrvEnv, barrier_scheduler: BarrierScheduler, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, metrics: Arc, ) -> MetaResult { let mut managed_sources = HashMap::new(); { - let sources = catalog_manager.list_sources().await; + let sources = metadata_manager.list_sources().await?; for source in sources { Self::create_source_worker_async( env.connector_client(), @@ -547,18 +546,32 @@ impl SourceManager { let mut actor_splits = HashMap::new(); let mut source_fragments = HashMap::new(); - for table_fragments in fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - { - source_fragments.extend(table_fragments.stream_source_fragments()); - actor_splits.extend(table_fragments.actor_splits.clone()); + + match &metadata_manager { + MetadataManager::V1(mgr) => { + for table_fragments in mgr + .fragment_manager + .get_fragment_read_guard() + .await + .table_fragments() + .values() + { + source_fragments.extend(table_fragments.stream_source_fragments()); + actor_splits.extend(table_fragments.actor_splits.clone()); + } + } + MetadataManager::V2(mgr) => { + // TODO: optimize it. + for (_, pb_table_fragments) in mgr.catalog_controller.table_fragments().await? { + let table_fragments = TableFragments::from_protobuf(pb_table_fragments); + source_fragments.extend(table_fragments.stream_source_fragments()); + actor_splits.extend(table_fragments.actor_splits); + } + } } let core = Mutex::new(SourceManagerCore::new( - fragment_manager, + metadata_manager, managed_sources, source_fragments, actor_splits, @@ -650,8 +663,8 @@ impl SourceManager { pub async fn allocate_splits(&self, table_id: &TableId) -> MetaResult { let core = self.core.lock().await; let table_fragments = core - .fragment_manager - .select_table_fragments_by_table_id(table_id) + .metadata_manager + .get_job_fragments_by_id(table_id) .await?; let source_fragments = table_fragments.stream_source_fragments(); diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 5c76675447ba8..801be6a4ab628 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -32,7 +32,7 @@ use risingwave_pb::stream_plan::{ use super::id::GlobalFragmentIdsExt; use super::Locations; -use crate::manager::{IdGeneratorManagerRef, StreamingClusterInfo, StreamingJob}; +use crate::manager::{MetaSrvEnv, StreamingClusterInfo, StreamingJob}; use crate::model::{DispatcherId, FragmentId}; use crate::stream::stream_graph::fragment::{ CompleteStreamFragmentGraph, EdgeId, EitherFragment, StreamFragmentEdge, @@ -706,7 +706,7 @@ impl ActorGraphBuilder { /// [`ActorGraphBuildResult`] that will be further used to build actors on the compute nodes. pub async fn generate_graph( self, - id_gen_manager: IdGeneratorManagerRef, + env: &MetaSrvEnv, job: &StreamingJob, expr_context: ExprContext, ) -> MetaResult { @@ -716,7 +716,9 @@ impl ActorGraphBuilder { .values() .map(|d| d.parallelism()) .sum::() as u64; - let id_gen = GlobalActorIdGen::new(&id_gen_manager, actor_len).await?; + + // TODO: use sql_id_gen that is not implemented yet. + let id_gen = GlobalActorIdGen::new(env.id_gen_manager(), actor_len).await?; // Build the actor graph and get the final state. let ActorGraphBuildStateInner { diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 7bd0ee47008ca..e4b2b03004fe8 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -38,7 +38,7 @@ use risingwave_pb::stream_plan::{ StreamFragmentGraph as StreamFragmentGraphProto, }; -use crate::manager::{IdGeneratorManagerRef, StreamingJob}; +use crate::manager::{MetaSrvEnv, StreamingJob}; use crate::model::FragmentId; use crate::stream::stream_graph::id::{GlobalFragmentId, GlobalFragmentIdGen, GlobalTableIdGen}; use crate::stream::stream_graph::schedule::Distribution; @@ -284,13 +284,15 @@ impl StreamFragmentGraph { /// Create a new [`StreamFragmentGraph`] from the given [`StreamFragmentGraphProto`], with all /// global IDs correctly filled. pub async fn new( + env: &MetaSrvEnv, proto: StreamFragmentGraphProto, - id_gen: IdGeneratorManagerRef, job: &StreamingJob, ) -> MetaResult { + // TODO: use sql_id_gen that is not implemented yet. let fragment_id_gen = - GlobalFragmentIdGen::new(&id_gen, proto.fragments.len() as u64).await?; - let table_id_gen = GlobalTableIdGen::new(&id_gen, proto.table_ids_cnt as u64).await?; + GlobalFragmentIdGen::new(env.id_gen_manager(), proto.fragments.len() as u64).await?; + let table_id_gen = + GlobalTableIdGen::new(env.id_gen_manager(), proto.table_ids_cnt as u64).await?; // Create nodes. let fragments: HashMap<_, _> = proto @@ -369,6 +371,18 @@ impl StreamFragmentGraph { tables } + pub fn refill_internal_table_ids(&mut self, table_id_map: HashMap) { + for fragment in self.fragments.values_mut() { + stream_graph_visitor::visit_internal_tables( + &mut fragment.inner, + |table, _table_type_name| { + let target = table_id_map.get(&table.id).cloned().unwrap(); + table.id = target; + }, + ); + } + } + /// Set internal tables' `table_id`s according to a list of internal tables pub fn fit_internal_table_ids( &mut self, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 270b90e4deeed..ad3da73d1115f 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeSet, HashMap, HashSet}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::sync::Arc; use futures::future::{join_all, try_join_all, BoxFuture}; @@ -20,6 +20,7 @@ use futures::stream::FuturesUnordered; use futures::TryStreamExt; use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_pb::catalog::{CreateType, Table}; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::Dispatcher; @@ -34,9 +35,7 @@ use uuid::Uuid; use super::{Locations, RescheduleOptions, ScaleController, ScaleControllerRef, TableResizePolicy}; use crate::barrier::{BarrierScheduler, Command, ReplaceTablePlan}; use crate::hummock::HummockManagerRef; -use crate::manager::{ - CatalogManagerRef, ClusterManagerRef, DdlType, FragmentManagerRef, MetaSrvEnv, StreamingJob, -}; +use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, StreamingJob, WorkerId}; use crate::model::{ActorId, TableFragments, TableParallelism}; use crate::stream::SourceManagerRef; use crate::{MetaError, MetaResult}; @@ -186,21 +185,14 @@ pub struct ReplaceTableContext { pub struct GlobalStreamManager { pub env: MetaSrvEnv, - /// Manages definition and status of fragments and actors - pub(super) fragment_manager: FragmentManagerRef, + pub metadata_manager: MetadataManager, /// Broadcasts and collect barriers pub barrier_scheduler: BarrierScheduler, - /// Maintains information of the cluster - pub cluster_manager: ClusterManagerRef, - /// Maintains streaming sources from external system like kafka pub source_manager: SourceManagerRef, - /// Catalog manager for cleaning up state from deleted stream jobs - pub catalog_manager: CatalogManagerRef, - /// Creating streaming job info. creating_job_info: CreatingStreamingJobInfoRef, @@ -208,36 +200,34 @@ pub struct GlobalStreamManager { pub reschedule_lock: RwLock<()>, - pub(crate) scale_controller: ScaleControllerRef, + pub(crate) scale_controller: Option, } impl GlobalStreamManager { pub fn new( env: MetaSrvEnv, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, barrier_scheduler: BarrierScheduler, - cluster_manager: ClusterManagerRef, source_manager: SourceManagerRef, hummock_manager: HummockManagerRef, - catalog_manager: CatalogManagerRef, ) -> MetaResult { - let scale_controller = Arc::new(ScaleController::new( - fragment_manager.clone(), - cluster_manager.clone(), - source_manager.clone(), - env.clone(), - )); + let scale_controller = match &metadata_manager { + MetadataManager::V1(_) => { + let scale_controller = + ScaleController::new(&metadata_manager, source_manager.clone(), env.clone()); + Some(Arc::new(scale_controller)) + } + MetadataManager::V2(_) => None, + }; Ok(Self { env, - fragment_manager, + metadata_manager, barrier_scheduler, - cluster_manager, source_manager, hummock_manager, creating_job_info: Arc::new(CreatingStreamingJobInfo::default()), reschedule_lock: RwLock::new(()), scale_controller, - catalog_manager, }) } @@ -301,8 +291,8 @@ impl GlobalStreamManager { CreatingState::Canceling { finish_tx } => { tracing::debug!(id=?table_id, "cancelling streaming job"); if let Ok(table_fragments) = self - .fragment_manager - .select_table_fragments_by_table_id(&table_id) + .metadata_manager + .get_job_fragments_by_id(&table_id) .await { // try to cancel buffered creating command. @@ -316,7 +306,7 @@ impl GlobalStreamManager { ); let node_actors = table_fragments.worker_actor_ids(); let cluster_info = - self.cluster_manager.get_streaming_cluster_info().await; + self.metadata_manager.get_streaming_cluster_info().await?; let node_actors = node_actors .into_iter() .map(|(id, actor_ids)| { @@ -340,19 +330,21 @@ impl GlobalStreamManager { }); try_join_all(futures).await?; - self.fragment_manager - .drop_table_fragments_vec(&HashSet::from_iter(std::iter::once( - table_id, - ))) + self.metadata_manager + .drop_streaming_job_by_ids(&HashSet::from_iter( + std::iter::once(table_id), + )) .await?; - } - if !table_fragments.is_created() { + } else if !table_fragments.is_created() { tracing::debug!( "cancelling streaming job {table_id} by issue cancel command." ); self.barrier_scheduler .run_command(Command::CancelStreamingJob(table_fragments)) .await?; + } else { + // streaming job is already completed. + continue; } let _ = finish_tx.send(()).inspect_err(|_| { tracing::warn!("failed to notify cancelled: {table_id}") @@ -499,6 +491,9 @@ impl GlobalStreamManager { .await?; if let Some((_, context, table_fragments)) = replace_table_job_info { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support create sink into table in v2"); + }; self.build_actors( &table_fragments, &context.building_locations, @@ -507,7 +502,7 @@ impl GlobalStreamManager { .await?; // Add table fragments to meta store with state: `State::Initial`. - self.fragment_manager + mgr.fragment_manager .start_create_table_fragments(table_fragments.clone()) .await?; @@ -527,11 +522,6 @@ impl GlobalStreamManager { replace_table_id = Some(dummy_table_id); } - // Add table fragments to meta store with state: `State::Initial`. - self.fragment_manager - .start_create_table_fragments(table_fragments.clone()) - .await?; - let table_id = table_fragments.table_id(); let init_split_assignment = self.source_manager.allocate_splits(&table_id).await?; @@ -552,8 +542,8 @@ impl GlobalStreamManager { if let Some(dummy_table_id) = replace_table_id { table_ids.insert(dummy_table_id); } - self.fragment_manager - .drop_table_fragments_vec(&table_ids) + self.metadata_manager + .drop_streaming_job_by_ids(&table_ids) .await?; } @@ -578,11 +568,6 @@ impl GlobalStreamManager { self.build_actors(&table_fragments, &building_locations, &existing_locations) .await?; - // Add table fragments to meta store with state: `State::Initial`. - self.fragment_manager - .start_create_table_fragments(table_fragments.clone()) - .await?; - let dummy_table_id = table_fragments.table_id(); let init_split_assignment = self.source_manager.allocate_splits(&dummy_table_id).await?; @@ -598,8 +583,8 @@ impl GlobalStreamManager { })) .await { - self.fragment_manager - .drop_table_fragments_vec(&HashSet::from_iter(std::iter::once(dummy_table_id))) + self.metadata_manager + .drop_streaming_job_by_ids(&HashSet::from_iter(std::iter::once(dummy_table_id))) .await?; return Err(err); } @@ -622,7 +607,10 @@ impl GlobalStreamManager { } pub async fn drop_streaming_jobs_impl(&self, table_ids: Vec) -> MetaResult<()> { - let table_fragments_vec = self + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("call drop_streaming_jobs_impl_v2 instead.") + }; + let table_fragments_vec = mgr .fragment_manager .select_table_fragments_by_ids(&table_ids) .await?; @@ -643,6 +631,24 @@ impl GlobalStreamManager { Ok(()) } + pub fn drop_streaming_jobs_impl_v2( + &self, + _job_info: HashMap>>, + _state_table_ids: Vec, + ) -> MetaResult<()> { + // self.barrier_scheduler.run_command(Command::DropStreamingJobsV2(job_info)).await?; + // + // // TODO: need some refactoring on source manager. + // + // // Unregister from compaction group afterwards. + // self.hummock_manager + // .unregister_table_ids_fail_fast( + // &state_table_ids + // ) + // .await; + unimplemented!("drop_streaming_jobs_impl_v2") + } + /// Cancel streaming jobs and return the canceled table ids. /// 1. Send cancel message to stream jobs (via `cancel_jobs`). /// 2. Send cancel message to recovered stream jobs (via `barrier_scheduler`). @@ -674,8 +680,7 @@ impl GlobalStreamManager { tracing::debug!(?id, "cancelling recovered streaming job"); let result: MetaResult<()> = try { let fragment = self - .fragment_manager - .select_table_fragments_by_table_id(&id) + .metadata_manager.get_job_fragments_by_id(&id) .await?; if fragment.is_created() { Err(MetaError::invalid_parameter(format!( @@ -683,7 +688,10 @@ impl GlobalStreamManager { id )))?; } - self.catalog_manager.cancel_create_table_procedure(id.into(), fragment.internal_table_ids()).await?; + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support cancel streaming job in v2"); + }; + mgr.catalog_manager.cancel_create_table_procedure(id.into(), fragment.internal_table_ids()).await?; self.barrier_scheduler .run_command(Command::CancelStreamingJob(fragment)) @@ -711,9 +719,12 @@ impl GlobalStreamManager { table_id: u32, parallelism: TableParallelism, ) -> MetaResult<()> { + let MetadataManager::V1(mgr) = &self.metadata_manager else { + unimplemented!("support alter table parallelism in v2"); + }; let _reschedule_job_lock = self.reschedule_lock.write().await; - let worker_nodes = self + let worker_nodes = mgr .cluster_manager .list_active_streaming_compute_nodes() .await; @@ -726,6 +737,8 @@ impl GlobalStreamManager { let reschedules = self .scale_controller + .as_ref() + .unwrap() .generate_table_resize_plan(TableResizePolicy { worker_ids, table_parallelisms: vec![(table_id, parallelism)].into_iter().collect(), @@ -779,8 +792,8 @@ mod tests { use crate::hummock::{CompactorManager, HummockManager}; use crate::manager::sink_coordination::SinkCoordinatorManager; use crate::manager::{ - CatalogManager, CatalogManagerRef, ClusterManager, FragmentManager, MetaSrvEnv, - RelationIdEnum, StreamingClusterInfo, + CatalogManager, CatalogManagerRef, ClusterManager, FragmentManager, FragmentManagerRef, + MetaSrvEnv, RelationIdEnum, StreamingClusterInfo, }; use crate::model::{ActorId, FragmentId}; use crate::rpc::ddl_controller::DropMode; @@ -918,7 +931,7 @@ mod tests { sleep(Duration::from_secs(1)).await; let env = MetaSrvEnv::for_test_opts(Arc::new(MetaOpts::test(enable_recovery))).await; - let system_params = env.system_params_manager().get_params().await; + let system_params = env.system_params_reader().await; let meta_metrics = Arc::new(MetaMetrics::default()); let cluster_manager = Arc::new(ClusterManager::new(env.clone(), Duration::from_secs(3600)).await?); @@ -950,13 +963,17 @@ mod tests { let (tx, _rx) = tokio::sync::mpsc::unbounded_channel(); - let hummock_manager = HummockManager::new( - env.clone(), + let metadata_manager = MetadataManager::new_v1( cluster_manager.clone(), + catalog_manager.clone(), fragment_manager.clone(), + ); + + let hummock_manager = HummockManager::new( + env.clone(), + metadata_manager.clone(), meta_metrics.clone(), compactor_manager.clone(), - catalog_manager.clone(), tx, ) .await?; @@ -971,8 +988,7 @@ mod tests { SourceManager::new( env.clone(), barrier_scheduler.clone(), - catalog_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), meta_metrics.clone(), ) .await?, @@ -983,9 +999,7 @@ mod tests { let barrier_manager = Arc::new(GlobalBarrierManager::new( scheduled_barriers, env.clone(), - cluster_manager.clone(), - catalog_manager.clone(), - fragment_manager.clone(), + metadata_manager.clone(), hummock_manager.clone(), source_manager.clone(), sink_manager, @@ -994,12 +1008,10 @@ mod tests { let stream_manager = GlobalStreamManager::new( env.clone(), - fragment_manager.clone(), + metadata_manager, barrier_scheduler.clone(), - cluster_manager.clone(), source_manager.clone(), hummock_manager, - catalog_manager.clone(), )?; let (join_handle_2, shutdown_tx_2) = GlobalBarrierManager::start(barrier_manager); @@ -1037,9 +1049,9 @@ mod tests { unschedulable_parallel_units: _, }: StreamingClusterInfo = self .global_stream_manager - .cluster_manager + .metadata_manager .get_streaming_cluster_info() - .await; + .await?; let actor_locations = fragments .values() @@ -1072,6 +1084,9 @@ mod tests { self.catalog_manager .start_create_table_procedure(&table, vec![]) .await?; + self.fragment_manager + .start_create_table_fragments(table_fragments.clone()) + .await?; self.global_stream_manager .create_streaming_job(table_fragments, ctx) .await?; @@ -1167,8 +1182,8 @@ mod tests { // test get table_fragment; let select_err_1 = services .global_stream_manager - .fragment_manager - .select_table_fragments_by_table_id(&table_id) + .metadata_manager + .get_job_fragments_by_id(&table_id) .await .unwrap_err(); @@ -1239,8 +1254,8 @@ mod tests { let table_fragments = services .global_stream_manager - .fragment_manager - .select_table_fragments_by_table_id(&table_id) + .metadata_manager + .get_job_fragments_by_id(&table_id) .await .unwrap(); assert_eq!(table_fragments.actor_ids(), (0..=3).collect_vec()); @@ -1255,8 +1270,8 @@ mod tests { // test get table_fragment; let select_err_1 = services .global_stream_manager - .fragment_manager - .select_table_fragments_by_table_id(&table_fragments.table_id()) + .metadata_manager + .get_job_fragments_by_id(&table_fragments.table_id()) .await .unwrap_err(); diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 9c9f30ac113b3..3382aae93ce9b 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -459,7 +459,7 @@ async fn test_graph_builder() -> MetaResult<()> { let expr_context = ExprContext { time_zone: graph.ctx.as_ref().unwrap().timezone.clone(), }; - let fragment_graph = StreamFragmentGraph::new(graph, env.id_gen_manager_ref(), &job).await?; + let fragment_graph = StreamFragmentGraph::new(&env, graph, &job).await?; let internal_tables = fragment_graph.internal_tables(); let actor_graph_builder = ActorGraphBuilder::new( @@ -469,7 +469,7 @@ async fn test_graph_builder() -> MetaResult<()> { NonZeroUsize::new(parallel_degree).unwrap(), )?; let ActorGraphBuildResult { graph, .. } = actor_graph_builder - .generate_graph(env.id_gen_manager_ref(), &job, expr_context) + .generate_graph(&env, &job, expr_context) .await?; let table_fragments = TableFragments::for_test(TableId::default(), graph); diff --git a/src/meta/src/telemetry.rs b/src/meta/src/telemetry.rs index 02c0431135261..fdb0ed06fd769 100644 --- a/src/meta/src/telemetry.rs +++ b/src/meta/src/telemetry.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - use risingwave_common::config::MetaBackend; use risingwave_common::telemetry::report::{TelemetryInfoFetcher, TelemetryReportCreator}; use risingwave_common::telemetry::{ @@ -22,8 +20,9 @@ use risingwave_common::telemetry::{ }; use risingwave_pb::common::WorkerType; use serde::{Deserialize, Serialize}; +use thiserror_ext::AsReport; -use crate::manager::ClusterManager; +use crate::manager::MetadataManager; use crate::model::ClusterId; #[derive(Debug, Serialize, Deserialize)] @@ -64,14 +63,14 @@ impl TelemetryInfoFetcher for MetaTelemetryInfoFetcher { #[derive(Clone)] pub struct MetaReportCreator { - cluster_mgr: Arc, + metadata_manager: MetadataManager, meta_backend: MetaBackend, } impl MetaReportCreator { - pub fn new(cluster_mgr: Arc, meta_backend: MetaBackend) -> Self { + pub fn new(metadata_manager: MetadataManager, meta_backend: MetaBackend) -> Self { Self { - cluster_mgr, + metadata_manager, meta_backend, } } @@ -86,7 +85,12 @@ impl TelemetryReportCreator for MetaReportCreator { session_id: String, up_time: u64, ) -> TelemetryResult { - let node_map = self.cluster_mgr.count_worker_node().await; + let node_map = self + .metadata_manager + .count_worker_node() + .await + .map_err(|err| err.as_report().to_string())?; + Ok(MetaTelemetryReport { base: TelemetryReportBase { tracking_id, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 1a1705e0c2efd..d8318d79b63b1 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -157,7 +157,7 @@ impl MetaClient { schema_id: u32, owner_id: u32, req: create_connection_request::Payload, - ) -> Result<(ConnectionId, CatalogVersion)> { + ) -> Result { let request = CreateConnectionRequest { name: connection_name, database_id, @@ -166,7 +166,7 @@ impl MetaClient { payload: Some(req), }; let resp = self.inner.create_connection(request).await?; - Ok((resp.connection_id, resp.version)) + Ok(resp.version) } pub async fn list_connections(&self, _name: Option<&str>) -> Result> { @@ -231,12 +231,12 @@ impl MetaClient { .await; let (add_worker_resp, system_params_resp, grpc_meta_client) = init_result?; - let worker_node = add_worker_resp - .node - .expect("AddWorkerNodeResponse::node is empty"); + let worker_id = add_worker_resp + .node_id + .expect("AddWorkerNodeResponse::node_id is empty"); let meta_client = Self { - worker_id: worker_node.id, + worker_id, worker_type, host_addr: addr.clone(), inner: grpc_meta_client, @@ -260,6 +260,7 @@ impl MetaClient { pub async fn activate(&self, addr: &HostAddr) -> Result<()> { let request = ActivateWorkerNodeRequest { host: Some(addr.to_protobuf()), + node_id: self.worker_id, }; let retry_strategy = GrpcMetaClient::retry_strategy_to_bound( Duration::from_secs(self.meta_config.max_heartbeat_interval_secs as u64), @@ -293,34 +294,34 @@ impl MetaClient { Ok(()) } - pub async fn create_database(&self, db: PbDatabase) -> Result<(DatabaseId, CatalogVersion)> { + pub async fn create_database(&self, db: PbDatabase) -> Result { let request = CreateDatabaseRequest { db: Some(db) }; let resp = self.inner.create_database(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.database_id, resp.version)) + Ok(resp.version) } - pub async fn create_schema(&self, schema: PbSchema) -> Result<(SchemaId, CatalogVersion)> { + pub async fn create_schema(&self, schema: PbSchema) -> Result { let request = CreateSchemaRequest { schema: Some(schema), }; let resp = self.inner.create_schema(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.schema_id, resp.version)) + Ok(resp.version) } pub async fn create_materialized_view( &self, table: PbTable, graph: StreamFragmentGraph, - ) -> Result<(TableId, CatalogVersion)> { + ) -> Result { let request = CreateMaterializedViewRequest { materialized_view: Some(table), fragment_graph: Some(graph), }; let resp = self.inner.create_materialized_view(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.table_id.into(), resp.version)) + Ok(resp.version) } pub async fn drop_materialized_view( @@ -337,28 +338,28 @@ impl MetaClient { Ok(resp.version) } - pub async fn create_source(&self, source: PbSource) -> Result<(u32, CatalogVersion)> { + pub async fn create_source(&self, source: PbSource) -> Result { let request = CreateSourceRequest { source: Some(source), fragment_graph: None, }; let resp = self.inner.create_source(request).await?; - Ok((resp.source_id, resp.version)) + Ok(resp.version) } pub async fn create_source_with_graph( &self, source: PbSource, graph: StreamFragmentGraph, - ) -> Result<(u32, CatalogVersion)> { + ) -> Result { let request = CreateSourceRequest { source: Some(source), fragment_graph: Some(graph), }; let resp = self.inner.create_source(request).await?; - Ok((resp.source_id, resp.version)) + Ok(resp.version) } pub async fn create_sink( @@ -366,7 +367,7 @@ impl MetaClient { sink: PbSink, graph: StreamFragmentGraph, affected_table_change: Option, - ) -> Result<(u32, CatalogVersion)> { + ) -> Result { let request = CreateSinkRequest { sink: Some(sink), fragment_graph: Some(graph), @@ -374,18 +375,15 @@ impl MetaClient { }; let resp = self.inner.create_sink(request).await?; - Ok((resp.sink_id, resp.version)) + Ok(resp.version) } - pub async fn create_function( - &self, - function: PbFunction, - ) -> Result<(FunctionId, CatalogVersion)> { + pub async fn create_function(&self, function: PbFunction) -> Result { let request = CreateFunctionRequest { function: Some(function), }; let resp = self.inner.create_function(request).await?; - Ok((resp.function_id.into(), resp.version)) + Ok(resp.version) } pub async fn create_table( @@ -394,7 +392,7 @@ impl MetaClient { table: PbTable, graph: StreamFragmentGraph, job_type: PbTableJobType, - ) -> Result<(TableId, CatalogVersion)> { + ) -> Result { let request = CreateTableRequest { materialized_view: Some(table), fragment_graph: Some(graph), @@ -403,7 +401,7 @@ impl MetaClient { }; let resp = self.inner.create_table(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.table_id.into(), resp.version)) + Ok(resp.version) } pub async fn comment_on(&self, comment: PbComment) -> Result { @@ -492,11 +490,11 @@ impl MetaClient { Ok(resp.version) } - pub async fn create_view(&self, view: PbView) -> Result<(u32, CatalogVersion)> { + pub async fn create_view(&self, view: PbView) -> Result { let request = CreateViewRequest { view: Some(view) }; let resp = self.inner.create_view(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.view_id, resp.version)) + Ok(resp.version) } pub async fn create_index( @@ -504,7 +502,7 @@ impl MetaClient { index: PbIndex, table: PbTable, graph: StreamFragmentGraph, - ) -> Result<(TableId, CatalogVersion)> { + ) -> Result { let request = CreateIndexRequest { index: Some(index), index_table: Some(table), @@ -512,7 +510,7 @@ impl MetaClient { }; let resp = self.inner.create_index(request).await?; // TODO: handle error in `resp.status` here - Ok((resp.index_id.into(), resp.version)) + Ok(resp.version) } pub async fn drop_table( @@ -575,13 +573,13 @@ impl MetaClient { Ok(resp.version) } - pub async fn drop_database(&self, database_id: u32) -> Result { + pub async fn drop_database(&self, database_id: DatabaseId) -> Result { let request = DropDatabaseRequest { database_id }; let resp = self.inner.drop_database(request).await?; Ok(resp.version) } - pub async fn drop_schema(&self, schema_id: u32) -> Result { + pub async fn drop_schema(&self, schema_id: SchemaId) -> Result { let request = DropSchemaRequest { schema_id }; let resp = self.inner.drop_schema(request).await?; Ok(resp.version) diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index cd793a3dcbfba..a80d4e3dac568 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -244,8 +244,8 @@ pub(crate) mod tests { .clone() .sstable_id_remote_fetch_number, )); - let worker_node2 = hummock_manager_ref - .cluster_manager + let worker_node_id2 = hummock_manager_ref + .metadata_manager() .add_worker_node( WorkerType::ComputeNode, HostAddress::default(), @@ -255,7 +255,7 @@ pub(crate) mod tests { .await .unwrap(); let _snapshot = hummock_manager_ref - .pin_snapshot(worker_node2.id) + .pin_snapshot(worker_node_id2) .await .unwrap(); let key = key.freeze(); diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index 27d444b7600cf..44522ddfb31ff 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -12,20 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Copyright 2023 Singularity Data -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - use std::sync::Arc; use risingwave_common::array::{Op, StreamChunk}; diff --git a/src/utils/pgwire/src/lib.rs b/src/utils/pgwire/src/lib.rs index d4018373db005..96d439b8b94ff 100644 --- a/src/utils/pgwire/src/lib.rs +++ b/src/utils/pgwire/src/lib.rs @@ -18,6 +18,7 @@ #![feature(trusted_len)] #![feature(lazy_cell)] #![feature(buf_read_has_data_left)] +#![feature(round_char_boundary)] #![expect(clippy::doc_markdown, reason = "FIXME: later")] pub mod error; diff --git a/src/utils/pgwire/src/pg_protocol.rs b/src/utils/pgwire/src/pg_protocol.rs index 98ad88ffcce25..ebffb24d81118 100644 --- a/src/utils/pgwire/src/pg_protocol.rs +++ b/src/utils/pgwire/src/pg_protocol.rs @@ -1139,8 +1139,9 @@ pub mod truncated_fmt { } if self.remaining < s.len() { - self.f.write_str(&s[0..self.remaining])?; - self.remaining = 0; + let actual = s.floor_char_boundary(self.remaining); + self.f.write_str(&s[0..actual])?; + self.remaining -= actual; self.f.write_str("...(truncated)")?; self.finished = true; // so that ...(truncated) is printed exactly once } else { @@ -1180,4 +1181,17 @@ pub mod truncated_fmt { .write_fmt(format_args!("{}", self.0)) } } + + #[cfg(test)] + mod tests { + use super::*; + + #[test] + fn test_trunc_utf8() { + assert_eq!( + format!("{}", TruncatedFmt(&"select '🌊';", 10)), + "select '...(truncated)", + ); + } + } }