diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 12ea66f9f1ac4..8fa51caf38cd5 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -66,6 +66,11 @@ sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.load.slt' sleep 10 sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.check.slt' +# cdc share stream test cases +export MYSQL_HOST=mysql MYSQL_TCP_PORT=3306 MYSQL_PWD=123456 +sqllogictest -p 4566 -d dev './e2e_test/source/cdc/cdc.share_stream.slt' + + # kill cluster and the connector node cargo make kill echo "cluster killed " diff --git a/e2e_test/source/cdc/cdc.share_stream.slt b/e2e_test/source/cdc/cdc.share_stream.slt new file mode 100644 index 0000000000000..459cde90580bc --- /dev/null +++ b/e2e_test/source/cdc/cdc.share_stream.slt @@ -0,0 +1,80 @@ +control substitution on + +# create database and tables in mysql +system ok +mysql --protocol=tcp -u root -e "DROP DATABASE IF EXISTS mytest; CREATE DATABASE mytest;" + +system ok +mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_create.sql + +# enable cdc backfill in ci +statement ok +set cdc_backfill='true'; + +# create a cdc source job, which format fixed to `FORMAT PLAIN ENCODE JSON` +statement ok +create source mysql_mytest with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'dbz', + password = '${MYSQL_PWD:}', + database.name = 'mytest', + server.id = '5601' +); + +statement ok +create table products_test ( id INT, + name STRING, + description STRING, + PRIMARY KEY (id) +) from mysql_mytest table 'mytest.products'; + +statement ok +create table orders_test ( + order_id int, + order_date timestamp, + customer_name string, + price decimal, + product_id int, + order_status smallint, + PRIMARY KEY (order_id) +) from mysql_mytest table 'mytest.orders'; + +statement ok +create materialized view products_test_cnt as select count(*) as cnt from products_test; + +statement ok +create materialized view orders_test_cnt as select count(*) as cnt from orders_test; + + +# generate data to mysql +system ok +mysql --protocol=tcp -u root mytest < e2e_test/source/cdc/mysql_init_data.sql + +sleep 5s + +# check ingestion results +query I +SELECT * from products_test_cnt +---- +9 + +query I +SELECT * from orders_test_cnt +---- +3 + +query ITT +SELECT * FROM products_test order by id limit 3 +---- +101 scooter Small 2-wheel scooter +102 car battery 12V car battery +103 12-pack drill bits 12-pack of drill bits with sizes ranging from #40 to #3 + +query ITTT +SELECT order_id,order_date,customer_name,product_id FROM orders_test order by order_id limit 3 +---- +10001 2020-07-30 10:08:22 Jark 102 +10002 2020-07-30 10:11:09 Sally 105 +10003 2020-07-30 12:00:30 Edward 106 diff --git a/e2e_test/source/cdc/cdc.validate.mysql.slt b/e2e_test/source/cdc/cdc.validate.mysql.slt index 6fe2bf86e5f40..180714c4f803e 100644 --- a/e2e_test/source/cdc/cdc.validate.mysql.slt +++ b/e2e_test/source/cdc/cdc.validate.mysql.slt @@ -68,6 +68,22 @@ create table products ( id INT, server.id = '5085' ); +# empty table name +statement error +create table products ( id INT, + name STRING, + description STRING, + PRIMARY KEY (id) +) with ( + connector = 'mysql-cdc', + hostname = 'mysql', + port = '3306', + username = 'root', + password = '123456', + database.name = 'my@db', + server.id = '5085' +); + # invalid primary key statement error create table orders ( diff --git a/e2e_test/source/cdc/mysql_create.sql b/e2e_test/source/cdc/mysql_create.sql new file mode 100644 index 0000000000000..2a742fccd3aac --- /dev/null +++ b/e2e_test/source/cdc/mysql_create.sql @@ -0,0 +1,26 @@ +CREATE TABLE products ( + id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512) +) AUTO_INCREMENT = 101; + +CREATE TABLE orders ( + order_id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, + order_date DATETIME NOT NULL, + customer_name VARCHAR(255) NOT NULL, + price DECIMAL(10, 5) NOT NULL, + product_id INTEGER NOT NULL, + order_status BOOLEAN NOT NULL -- Whether order has been placed +) AUTO_INCREMENT = 10001; + +CREATE TABLE mytable ( + v1 INTEGER NOT NULL PRIMARY KEY, + v2 INTEGER NOT NULL, + v3 VARCHAR(255) NOT NULL +); + +DROP USER IF EXISTS 'dbz'@'%'; +CREATE USER 'dbz'@'%' IDENTIFIED BY '123456'; +GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT ON *.* TO 'dbz'@'%'; + +CREATE TABLE tt3 (v1 int primary key, v2 timestamp); diff --git a/e2e_test/source/cdc/mysql_init_data.sql b/e2e_test/source/cdc/mysql_init_data.sql new file mode 100644 index 0000000000000..e954b74aaf500 --- /dev/null +++ b/e2e_test/source/cdc/mysql_init_data.sql @@ -0,0 +1,28 @@ +-- USE `my@db`; + +INSERT INTO products +VALUES (default,"scooter","Small 2-wheel scooter"), + (default,"car battery","12V car battery"), + (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3"), + (default,"hammer","12oz carpenter's hammer"), + (default,"hammer","14oz carpenter's hammer"), + (default,"hammer","16oz carpenter's hammer"), + (default,"rocks","box of assorted rocks"), + (default,"jacket","water resistent black wind breaker"), + (default,"spare tire","24 inch spare tire"); + + +INSERT INTO orders +VALUES (default, '2020-07-30 10:08:22', 'Jark', 50.50, 102, false), + (default, '2020-07-30 10:11:09', 'Sally', 15.00, 105, false), + (default, '2020-07-30 12:00:30', 'Edward', 25.25, 106, false); + + +INSERT INTO mytable +VALUES (1,1,'no'), + (2,2,'no'), + (3,3,'no'), + (4,4,'no'); + +INSERT INTO tt3 VALUES (1, '2020-07-30 10:08:22'); +INSERT INTO tt3 VALUES (2, '2020-07-31 10:09:22'); diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java index 18517ebb6dbf3..72b361e04bc0c 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/SourceValidateHandler.java @@ -76,13 +76,18 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re ensurePropNotNull(props, DbzConnectorConfig.HOST); ensurePropNotNull(props, DbzConnectorConfig.PORT); ensurePropNotNull(props, DbzConnectorConfig.DB_NAME); - ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); ensurePropNotNull(props, DbzConnectorConfig.USER); ensurePropNotNull(props, DbzConnectorConfig.PASSWORD); + // ensure table name is passed by user in single mode + if (Utils.getCdcSourceMode(props) == CdcSourceMode.SINGLE_MODE) { + ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); + } + TableSchema tableSchema = TableSchema.fromProto(request.getTableSchema()); switch (request.getSourceType()) { case POSTGRES: + ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); ensurePropNotNull(props, DbzConnectorConfig.PG_SCHEMA_NAME); ensurePropNotNull(props, DbzConnectorConfig.PG_SLOT_NAME); ensurePropNotNull(props, DbzConnectorConfig.PG_PUB_NAME); @@ -93,6 +98,7 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re break; case CITUS: + ensurePropNotNull(props, DbzConnectorConfig.TABLE_NAME); ensurePropNotNull(props, DbzConnectorConfig.PG_SCHEMA_NAME); try (var coordinatorValidator = new CitusValidator(props, tableSchema)) { coordinatorValidator.validateDistributedTable(); diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/CdcSourceMode.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/CdcSourceMode.java new file mode 100644 index 0000000000000..4c3b472dd6b53 --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/CdcSourceMode.java @@ -0,0 +1,22 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.risingwave.connector.source.common; + +public enum CdcSourceMode { + // The source is dedicated to a single CDC table + SINGLE_MODE, + // The source is sharing by multiple CDC tables + SHARING_MODE, +} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java index 5466eca2aaa5b..a77da9b6d2ed0 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/DbzConnectorConfig.java @@ -45,15 +45,18 @@ public class DbzConnectorConfig { public static final String DB_SERVERS = "database.servers"; - /* MySQL specified configs */ + /* MySQL configs */ public static final String MYSQL_SERVER_ID = "server.id"; - /* Postgres specified configs */ + /* Postgres configs */ public static final String PG_SLOT_NAME = "slot.name"; public static final String PG_PUB_NAME = "publication.name"; public static final String PG_PUB_CREATE = "publication.create.enable"; public static final String PG_SCHEMA_NAME = "schema.name"; + /* RisingWave configs */ + public static final String CDC_SHARING_MODE = "rw.sharing.mode.enable"; + private static final String DBZ_CONFIG_FILE = "debezium.properties"; private static final String MYSQL_CONFIG_FILE = "mysql.properties"; private static final String POSTGRES_CONFIG_FILE = "postgres.properties"; @@ -128,6 +131,7 @@ public DbzConnectorConfig( } dbzProps.putAll(mysqlProps); + } else if (source == SourceTypeE.POSTGRES || source == SourceTypeE.CITUS) { var postgresProps = initiateDbConfig(POSTGRES_CONFIG_FILE, substitutor); @@ -161,11 +165,21 @@ public DbzConnectorConfig( dbzProps.putIfAbsent(entry.getKey(), entry.getValue()); } + if (Utils.getCdcSourceMode(userProps) == CdcSourceMode.SHARING_MODE) { + adjustConfigForSharedCdcStream(dbzProps); + } + this.sourceId = sourceId; this.sourceType = source; this.resolvedDbzProps = dbzProps; } + private void adjustConfigForSharedCdcStream(Properties dbzProps) { + // disable table filtering for the shared cdc stream + LOG.info("Disable table filtering for the shared cdc stream"); + dbzProps.remove("table.include.list"); + } + private Properties initiateDbConfig(String fileName, StringSubstitutor substitutor) { var dbProps = new Properties(); try (var input = getClass().getClassLoader().getResourceAsStream(fileName)) { diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java index 54094bc21862d..57186bb7e9bac 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MySqlValidator.java @@ -104,7 +104,10 @@ public void validateUserPrivilege() { @Override public void validateTable() { try { - validateTableSchema(); + // validate table schema only when not running in cdc streaming mode + if (Utils.getCdcSourceMode(userProps) == CdcSourceMode.SINGLE_MODE) { + validateTableSchema(); + } } catch (SQLException e) { throw ValidatorUtils.internalError(e.getMessage()); } diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/Utils.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/Utils.java new file mode 100644 index 0000000000000..77f8d31dff9df --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/Utils.java @@ -0,0 +1,27 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.risingwave.connector.source.common; + +import java.util.Map; + +public class Utils { + + public static CdcSourceMode getCdcSourceMode(Map props) { + var isSharing = + Boolean.parseBoolean( + props.getOrDefault(DbzConnectorConfig.CDC_SHARING_MODE, "false")); + return isSharing ? CdcSourceMode.SHARING_MODE : CdcSourceMode.SINGLE_MODE; + } +} diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java index 9f3fc3d17e39b..d1d46d6924204 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEventConsumer.java @@ -55,7 +55,7 @@ public class DbzCdcEventConsumer // only serialize the value part configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName()); // include record schema - configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, false); + configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, true); jsonConverter.configure(configs); this.converter = jsonConverter; } @@ -99,6 +99,13 @@ var record = event.value(); LOG.debug("heartbeat => {}", message.getOffset()); respBuilder.addEvents(message); } else { + + // Topic naming conventions + // - PG: serverName.schemaName.tableName + // - MySQL: serverName.databaseName.tableName + // We can extract the full table name from the topic + var fullTableName = record.topic().substring(record.topic().indexOf('.') + 1); + // ignore null record if (record.value() == null) { committer.markProcessed(event); @@ -108,7 +115,10 @@ var record = event.value(); converter.fromConnectData( record.topic(), record.valueSchema(), record.value()); - msgBuilder.setPayload(new String(payload, StandardCharsets.UTF_8)).build(); + msgBuilder + .setFullTableName(fullTableName) + .setPayload(new String(payload, StandardCharsets.UTF_8)) + .build(); var message = msgBuilder.build(); LOG.debug("record => {}", message.getPayload()); diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties index cc64723b66b60..04ee9a2dad821 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mysql.properties @@ -4,25 +4,21 @@ offset.storage=com.risingwave.connector.cdc.debezium.internal.ConfigurableOffset database.history=io.debezium.relational.history.MemoryDatabaseHistory # default snapshot mode to initial snapshot.mode=${debezium.snapshot.mode:-initial} - database.hostname=${hostname} database.port=${port} database.user=${username} database.password=${password} - database.include.list=${database.name} table.include.list=${database.name}.${table.name} - # default to disable schema change events include.schema.changes=${debezium.include.schema.changes:-false} database.server.id=${server.id} - # set connector timezone to UTC(+00:00) database.connectionTimeZone=+00:00 - # default heartbeat interval 60 seconds heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-60000} heartbeat.topics.prefix=${debezium.heartbeat.topics.prefix:-RW_CDC_HeartBeat_} -name=${hostname}:${port}:${database.name}.${table.name} - +# In sharing cdc mode, we will subscribe to multiple tables in the given database, +# so here we set ${table.name} to a default value `RW_CDC_Sharing` just for display. +name=${hostname}:${port}:${database.name}.${table.name:-RW_CDC_Sharing} provide.transaction.metadata=${transactional:-false} diff --git a/proto/catalog.proto b/proto/catalog.proto index 4f421305c4eea..1372e93c2d2e1 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -61,7 +61,10 @@ message StreamSourceInfo { plan_common.EncodeType row_encode = 9; SchemaRegistryNameStrategy name_strategy = 10; optional string key_message_name = 11; - plan_common.StorageTableDesc upstream_table = 12; + plan_common.ExternalTableDesc external_table = 12; + // Whether the stream source is a cdc source streaming job. + // We need this field to differentiate the cdc source job until we fully implement risingwavelabs/rfcs#72. + bool cdc_source_job = 13; } message Source { diff --git a/proto/connector_service.proto b/proto/connector_service.proto index e750c8ce96e9d..56ce3c252bd2f 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -148,6 +148,7 @@ message CdcMessage { string payload = 1; string partition = 2; string offset = 3; + string full_table_name = 4; } enum SourceType { diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index b3c7f17509f8c..6d00225f32ec3 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -49,6 +49,7 @@ message DropSchemaResponse { message CreateSourceRequest { catalog.Source source = 1; + stream_plan.StreamFragmentGraph fragment_graph = 2; } message CreateSourceResponse { @@ -138,12 +139,25 @@ message DropViewResponse { uint64 version = 2; } +// An enum to distinguish different types of the Table streaming job. +// - GENERAL: Table streaming jobs w/ or w/o a connector +// - SHARED_CDC_SOURCE: The table streaming job is created based on a shared CDC source job (risingwavelabs/rfcs#73). +// And one may add other types to support Table jobs that based on other backfill-able sources (risingwavelabs/rfcs#72). +enum TableJobType { + UNSPECIFIED = 0; + // table streaming jobs excepts the `SHARED_CDC_SOURCE` type + GENERAL = 1; + // table streaming job sharing a CDC source job + SHARED_CDC_SOURCE = 2; +} + message CreateTableRequest { // An optional field and will be `Some` for tables with an external connector. If so, the table // will subscribe to the changes of the external connector and materialize the data. catalog.Source source = 1; catalog.Table materialized_view = 2; stream_plan.StreamFragmentGraph fragment_graph = 3; + TableJobType job_type = 4; } message CreateTableResponse { diff --git a/proto/plan_common.proto b/proto/plan_common.proto index afea3aff14bc7..ad9d58e3b2cc1 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -75,6 +75,17 @@ message StorageTableDesc { repeated uint32 stream_key = 9; } +// Represents a table in external database for CDC scenario +message ExternalTableDesc { + uint32 table_id = 1; + repeated ColumnDesc columns = 2; + // TODO: may refactor primary key representations + repeated common.ColumnOrder pk = 3; + string table_name = 4; + repeated uint32 stream_key = 5; + map connect_properties = 6; +} + enum JoinType { // Note that it comes from Calcite's JoinRelType. // DO NOT HAVE direction for SEMI and ANTI now. diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 97f0c72937cda..bebf85898a265 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -461,6 +461,8 @@ enum ChainType { // UPSTREAM_ONLY is corresponding to the chain executor, but doesn't consume the snapshot. UPSTREAM_ONLY = 4; + + CDC_BACKFILL = 5; } // ChainNode is used for mv on mv. @@ -496,6 +498,9 @@ message ChainNode { // Snapshot read every N barriers uint32 snapshot_read_barrier_interval = 9 [deprecated = true]; + + // The external table that will be backfilled for CDC. + plan_common.ExternalTableDesc cdc_table_desc = 10; } // BatchPlanNode is used for mv on mv snapshot read. @@ -719,6 +724,11 @@ enum DispatcherType { // piped into the downstream actor, if there are the same number of actors. If number of actors // are not the same, should use hash instead. Should be only used when distribution is the same. NO_SHUFFLE = 4; + + // Dispatch by table name from upstream DB, used in CDC scenario which should has only one downstream actor. + // From the optimizer's point of view, it can be treated as a specialized version of HASH distribution + // that the hash key is the upstream table name. + CDC_TABLENAME = 5; } // The property of an edge in the fragment graph. @@ -727,6 +737,8 @@ message DispatchStrategy { DispatcherType type = 1; repeated uint32 dist_key_indices = 2; repeated uint32 output_indices = 3; + // The full table name of the downstream CDC table. + optional string downstream_table_name = 4; } // A dispatcher redistribute messages. @@ -748,6 +760,8 @@ message Dispatcher { uint64 dispatcher_id = 4; // Number of downstreams decides how many endpoints a dispatcher should dispatch. repeated uint32 downstream_actor_id = 5; + // The full table name of the downstream CDC table. + optional string downstream_table_name = 7; } // A StreamActor is a running fragment of the overall stream graph, diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index b70084fbf864a..3a90e77e2e37b 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -20,7 +20,7 @@ use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{PbColumnCatalog, PbColumnDesc}; use super::row_id_column_desc; -use crate::catalog::{offset_column_desc, Field, ROW_ID_COLUMN_ID}; +use crate::catalog::{cdc_table_name_column_desc, offset_column_desc, Field, ROW_ID_COLUMN_ID}; use crate::error::ErrorCode; use crate::types::DataType; @@ -117,6 +117,18 @@ impl ColumnDesc { } } + pub fn named(name: String, column_id: ColumnId, data_type: DataType) -> ColumnDesc { + ColumnDesc { + data_type, + column_id, + name, + field_descs: vec![], + type_name: String::new(), + generated_or_default_column: None, + description: None, + } + } + /// Convert to proto pub fn to_protobuf(&self) -> PbColumnDesc { PbColumnDesc { @@ -344,6 +356,13 @@ impl ColumnCatalog { is_hidden: true, } } + + pub fn cdc_table_name_column() -> Self { + Self { + column_desc: cdc_table_name_column_desc(), + is_hidden: true, + } + } } impl From for ColumnCatalog { diff --git a/src/common/src/catalog/external_table.rs b/src/common/src/catalog/external_table.rs new file mode 100644 index 0000000000000..8c4ed3f6d1bd5 --- /dev/null +++ b/src/common/src/catalog/external_table.rs @@ -0,0 +1,77 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{BTreeMap, HashMap}; + +use risingwave_pb::plan_common::ExternalTableDesc; + +use super::{ColumnDesc, ColumnId, TableId}; +use crate::util::sort_util::ColumnOrder; + +/// Necessary information for compute node to access data in the external database. +/// Compute node will use this information to connect to the external database and scan the table. +#[derive(Debug, Clone, Default, PartialEq, Eq, Hash)] +pub struct CdcTableDesc { + /// Id of the upstream source in sharing cdc mode + pub table_id: TableId, + + /// The full name of the table in external database, e.g. `database_name.table.name` in MySQL + /// and `schema_name.table_name` in the Postgres. + pub external_table_name: String, + /// The key used to sort in storage. + pub pk: Vec, + /// All columns in the table, noticed it is NOT sorted by columnId in the vec. + pub columns: Vec, + + /// Column indices for primary keys. + pub stream_key: Vec, + + pub value_indices: Vec, + + /// properties will be passed into the ChainNode + pub connect_properties: BTreeMap, +} + +impl CdcTableDesc { + pub fn order_column_indices(&self) -> Vec { + self.pk.iter().map(|col| (col.column_index)).collect() + } + + pub fn order_column_ids(&self) -> Vec { + self.pk + .iter() + .map(|col| self.columns[col.column_index].column_id) + .collect() + } + + pub fn to_protobuf(&self) -> ExternalTableDesc { + ExternalTableDesc { + table_id: self.table_id.into(), + columns: self.columns.iter().map(Into::into).collect(), + pk: self.pk.iter().map(|v| v.to_protobuf()).collect(), + table_name: self.external_table_name.clone(), + stream_key: self.stream_key.iter().map(|k| *k as _).collect(), + connect_properties: self.connect_properties.clone(), + } + } + + /// Helper function to create a mapping from `column id` to `column index` + pub fn get_id_to_op_idx_mapping(&self) -> HashMap { + let mut id_to_idx = HashMap::new(); + self.columns.iter().enumerate().for_each(|(idx, c)| { + id_to_idx.insert(c.column_id, idx); + }); + id_to_idx + } +} diff --git a/src/common/src/catalog/mod.rs b/src/common/src/catalog/mod.rs index 1a46cdcf4057a..888ca5c2d4067 100644 --- a/src/common/src/catalog/mod.rs +++ b/src/common/src/catalog/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. mod column; +mod external_table; mod internal_table; mod physical_table; mod schema; @@ -23,6 +24,7 @@ use std::sync::Arc; use async_trait::async_trait; pub use column::*; +pub use external_table::*; pub use internal_table::*; use parse_display::Display; pub use physical_table::*; @@ -120,10 +122,17 @@ pub fn offset_column_name() -> String { OFFSET_COLUMN_NAME.to_string() } +pub const CDC_SOURCE_COLUMN_NUM: u32 = 4; +pub const TABLE_NAME_COLUMN_NAME: &str = "_rw_table_name"; +pub fn cdc_table_name_column_name() -> String { + TABLE_NAME_COLUMN_NAME.to_string() +} + pub fn is_offset_column_name(name: &str) -> bool { name.starts_with(OFFSET_COLUMN_NAME) } /// Creates a offset column for storing upstream offset +/// Used in cdc source currently pub fn offset_column_desc() -> ColumnDesc { ColumnDesc { data_type: DataType::Varchar, @@ -136,6 +145,19 @@ pub fn offset_column_desc() -> ColumnDesc { } } +/// A column to store the upstream table name of the cdc table +pub fn cdc_table_name_column_desc() -> ColumnDesc { + ColumnDesc { + data_type: DataType::Varchar, + column_id: ColumnId::placeholder(), + name: cdc_table_name_column_name(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + } +} + /// The local system catalog reader in the frontend node. #[async_trait] pub trait SysCatalogReader: Sync + Send + 'static { diff --git a/src/common/src/util/column_index_mapping.rs b/src/common/src/util/column_index_mapping.rs index 212c07df1e285..49aac42d0815d 100644 --- a/src/common/src/util/column_index_mapping.rs +++ b/src/common/src/util/column_index_mapping.rs @@ -335,6 +335,7 @@ impl ColIndexMapping { r#type: strategy.r#type, dist_key_indices: map(&strategy.dist_key_indices)?, output_indices: map(&strategy.output_indices)?, + downstream_table_name: strategy.downstream_table_name.clone(), }) } } diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index fff56a17d9117..1cb35691f8e15 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -15,6 +15,8 @@ #![feature(let_chains)] #![feature(coroutines)] +use std::collections::{HashMap, HashSet}; +use std::marker::PhantomData; use std::sync::atomic::AtomicU64; use std::sync::Arc; use std::time::Duration; @@ -28,10 +30,11 @@ use risingwave_common::array::{ }; use risingwave_common::catalog::{ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; +use risingwave_connector::source::cdc::{CdcSplitBase, DebeziumCdcSplit, MySqlCdcSplit}; use risingwave_connector::source::external::{ DebeziumOffset, DebeziumSourceOffset, ExternalTableReaderImpl, MySqlOffset, SchemaTableName, }; -use risingwave_connector::source::MockExternalTableReader; +use risingwave_connector::source::{MockExternalTableReader, SplitImpl}; use risingwave_hummock_sdk::to_committed_batch_query_epoch; use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::table::batch_table::storage_table::StorageTable; @@ -199,8 +202,9 @@ async fn test_cdc_backfill() -> StreamResult<()> { ) .await; + let actor_id = 0x1a; let cdc_backfill = CdcBackfillExecutor::new( - ActorContext::create(0x1a), + ActorContext::create(actor_id), external_table, Box::new(mock_offset_executor), vec![0, 1, 2], @@ -209,6 +213,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { vec![0], Arc::new(StreamingMetrics::unused()), source_state_handler, + false, 4, // 4 rows in a snapshot chunk ); @@ -247,14 +252,36 @@ async fn test_cdc_backfill() -> StreamResult<()> { // The first barrier let curr_epoch = 11; - tx.push_barrier(curr_epoch, false); + let mut splits = HashMap::new(); + splits.insert( + actor_id, + vec![SplitImpl::MysqlCdc(DebeziumCdcSplit { + mysql_split: Some(MySqlCdcSplit { + inner: CdcSplitBase { + split_id: 0, + start_offset: None, + snapshot_done: false, + }, + }), + pg_split: None, + _phantom: PhantomData, + })], + ); + let init_barrier = Barrier::new_test_barrier(curr_epoch).with_mutation(Mutation::Add { + adds: HashMap::new(), + added_actors: HashSet::new(), + splits, + pause: false, + }); + + tx.send_barrier(init_barrier); // assert barrier is forwarded to mview assert!(matches!( materialize.next().await.unwrap()?, Message::Barrier(Barrier { epoch, - mutation: None, + mutation: Some(_), .. }) if epoch.curr == curr_epoch )); diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index bdbb110daf7fc..cdb3ef7f51620 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -25,7 +25,7 @@ use futures_async_stream::try_stream; pub use json_parser::*; pub use protobuf::*; use risingwave_common::array::{ArrayBuilderImpl, Op, StreamChunk}; -use risingwave_common::catalog::KAFKA_TIMESTAMP_COLUMN_NAME; +use risingwave_common::catalog::{KAFKA_TIMESTAMP_COLUMN_NAME, TABLE_NAME_COLUMN_NAME}; use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::{Datum, Scalar}; @@ -172,7 +172,7 @@ impl MessageMeta<'_> { assert_eq!( desc.name.as_str(), KAFKA_TIMESTAMP_COLUMN_NAME, - "unexpected meta column name" + "unexpected kafka meta column name" ); kafka_meta .timestamp @@ -182,6 +182,10 @@ impl MessageMeta<'_> { .to_scalar_value() }) .into() + }, + SourceColumnType::Meta if let SourceMeta::DebeziumCdc(cdc_meta) = self.meta => { + assert_eq!(desc.name.as_str(), TABLE_NAME_COLUMN_NAME, "unexpected cdc meta column name"); + Datum::Some(cdc_meta.full_table_name.as_str().into()).into() } // For other cases, return `None`. diff --git a/src/connector/src/parser/mysql.rs b/src/connector/src/parser/mysql.rs index 0a0f8f52e90b2..e4a311555d008 100644 --- a/src/connector/src/parser/mysql.rs +++ b/src/connector/src/parser/mysql.rs @@ -14,7 +14,7 @@ use chrono::NaiveDate; use mysql_async::Row as MysqlRow; -use risingwave_common::catalog::{Schema, OFFSET_COLUMN_NAME}; +use risingwave_common::catalog::Schema; use risingwave_common::types::{ DataType, Date, Datum, Decimal, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, }; @@ -55,13 +55,8 @@ pub fn mysql_row_to_datums(mysql_row: &mut MysqlRow, schema: &Schema) -> Vec { - // snapshot data doesn't contain offset, just fill None - if rw_field.name.as_str() == OFFSET_COLUMN_NAME { - None - } else { - let v = mysql_row.take::(i); - v.map(ScalarImpl::from) - } + let v = mysql_row.take::(i); + v.map(ScalarImpl::from) } DataType::Date => { let v = mysql_row.take::(i); @@ -93,7 +88,7 @@ pub fn mysql_row_to_datums(mysql_row: &mut MysqlRow, schema: &Schema) -> Vec { // Interval, Struct, List, Int256 are not supported - tracing::warn!(rw_field.name, ?rw_field.data_type, "unsupported data type, set to Null"); + tracing::warn!(rw_field.name, ?rw_field.data_type, "unsupported data type, set to null"); None } } diff --git a/src/connector/src/sink/utils.rs b/src/connector/src/sink/utils.rs index 967c3fc43ba30..648414e9ac7f4 100644 --- a/src/connector/src/sink/utils.rs +++ b/src/connector/src/sink/utils.rs @@ -22,6 +22,7 @@ pub fn chunk_to_json(chunk: StreamChunk, encoder: &JsonEncoder) -> Result = Vec::with_capacity(chunk.capacity()); for (_, row) in chunk.rows() { let record = Value::Object(encoder.encode(row)?); + records.push(record.to_string()); } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index e5193291a6d14..f8e029c36110b 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -33,6 +33,7 @@ use risingwave_pb::source::ConnectorSplit; use risingwave_rpc_client::ConnectorClient; use serde::de::DeserializeOwned; +use super::cdc::DebeziumCdcMeta; use super::datagen::DatagenMeta; use super::filesystem::FsSplit; use super::google_pubsub::GooglePubsubMeta; @@ -572,6 +573,7 @@ pub enum SourceMeta { Nexmark(NexmarkMeta), GooglePubsub(GooglePubsubMeta), Datagen(DatagenMeta), + DebeziumCdc(DebeziumCdcMeta), // For the source that doesn't have meta data. Empty, } diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index d55273bf725db..8f5a2ee83b2ee 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -30,6 +30,9 @@ use crate::source::{SourceProperties, SplitImpl, TryFromHashmap}; use crate::{for_all_classified_sources, impl_cdc_source_type}; pub const CDC_CONNECTOR_NAME_SUFFIX: &str = "-cdc"; +pub const CDC_SNAPSHOT_MODE_KEY: &str = "debezium.snapshot.mode"; +pub const CDC_SNAPSHOT_BACKFILL: &str = "rw_cdc_backfill"; +pub const CDC_SHARING_MODE_KEY: &str = "rw.sharing.mode.enable"; pub const MYSQL_CDC_CONNECTOR: &str = Mysql::CDC_CONNECTOR_NAME; pub const POSTGRES_CDC_CONNECTOR: &str = Postgres::CDC_CONNECTOR_NAME; diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index f3890377e0bc6..e5acda608fba5 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -17,6 +17,11 @@ use risingwave_pb::connector_service::CdcMessage; use crate::source::base::SourceMessage; use crate::source::SourceMeta; +#[derive(Debug, Clone)] +pub struct DebeziumCdcMeta { + pub full_table_name: String, +} + impl From for SourceMessage { fn from(message: CdcMessage) -> Self { SourceMessage { @@ -28,7 +33,9 @@ impl From for SourceMessage { }, offset: message.offset, split_id: message.partition.into(), - meta: SourceMeta::Empty, + meta: SourceMeta::DebeziumCdc(DebeziumCdcMeta { + full_table_name: message.full_table_name, + }), } } } diff --git a/src/connector/src/source/cdc/source/mod.rs b/src/connector/src/source/cdc/source/mod.rs index 4a2141fa0fe7c..d73258c8f53d5 100644 --- a/src/connector/src/source/cdc/source/mod.rs +++ b/src/connector/src/source/cdc/source/mod.rs @@ -15,6 +15,7 @@ mod message; mod reader; +pub use message::*; pub use reader::*; pub use crate::source::cdc::split::*; diff --git a/src/connector/src/source/external.rs b/src/connector/src/source/external.rs index 953277ba36106..ea17a0a2272a0 100644 --- a/src/connector/src/source/external.rs +++ b/src/connector/src/source/external.rs @@ -37,14 +37,14 @@ use crate::source::MockExternalTableReader; pub type ConnectorResult = std::result::Result; #[derive(Debug)] -pub enum ExternalTableType { +pub enum CdcTableType { Undefined, MySql, Postgres, Citus, } -impl ExternalTableType { +impl CdcTableType { pub fn from_properties(properties: &HashMap) -> Self { let connector = properties .get("connector") @@ -81,12 +81,14 @@ impl ExternalTableType { #[derive(Debug, Clone)] pub struct SchemaTableName { + // namespace of the table, e.g. database in mysql, schema in postgres pub schema_name: String, pub table_name: String, } -const TABLE_NAME_KEY: &str = "table.name"; -const SCHEMA_NAME_KEY: &str = "schema.name"; +pub const TABLE_NAME_KEY: &str = "table.name"; +pub const SCHEMA_NAME_KEY: &str = "schema.name"; +pub const DATABASE_NAME_KEY: &str = "database.name"; impl SchemaTableName { pub fn new(schema_name: String, table_name: String) -> Self { @@ -97,15 +99,21 @@ impl SchemaTableName { } pub fn from_properties(properties: &HashMap) -> Self { - let table_name = properties - .get(TABLE_NAME_KEY) - .map(|c| c.to_ascii_lowercase()) - .unwrap_or_default(); - - let schema_name = properties - .get(SCHEMA_NAME_KEY) - .map(|c| c.to_ascii_lowercase()) - .unwrap_or_default(); + let table_type = CdcTableType::from_properties(properties); + let table_name = properties.get(TABLE_NAME_KEY).cloned().unwrap_or_default(); + + let schema_name = match table_type { + CdcTableType::MySql => properties + .get(DATABASE_NAME_KEY) + .cloned() + .unwrap_or_default(), + CdcTableType::Postgres | CdcTableType::Citus => { + properties.get(SCHEMA_NAME_KEY).cloned().unwrap_or_default() + } + _ => { + unreachable!("invalid external table type: {:?}", table_type); + } + }; Self { schema_name, @@ -247,7 +255,8 @@ pub struct ExternalTableConfig { impl ExternalTableReader for MySqlExternalTableReader { fn get_normalized_table_name(&self, table_name: &SchemaTableName) -> String { - format!("`{}`", table_name.table_name) + // schema name is the database name in mysql + format!("`{}`.`{}`", table_name.schema_name, table_name.table_name) } async fn current_cdc_offset(&self) -> ConnectorResult { @@ -286,13 +295,7 @@ impl ExternalTableReader for MySqlExternalTableReader { impl MySqlExternalTableReader { pub fn new(properties: HashMap, rw_schema: Schema) -> ConnectorResult { - if let Some(field) = rw_schema.fields.last() - && field.name.as_str() != OFFSET_COLUMN_NAME - { - return Err(ConnectorError::Config(anyhow!( - "last column of schema must be `_rw_offset`" - ))); - } + tracing::debug!(?rw_schema, "create mysql external table reader"); let config = serde_json::from_value::( serde_json::to_value(properties).unwrap(), diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index 8624550274299..c766f05099810 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -16,6 +16,7 @@ use std::fmt::Debug; use risingwave_common::catalog::{ ColumnDesc, ColumnId, KAFKA_TIMESTAMP_COLUMN_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, + TABLE_NAME_COLUMN_NAME, }; use risingwave_common::types::DataType; @@ -51,7 +52,8 @@ pub enum SourceColumnType { impl SourceColumnType { pub fn from_name(name: &str) -> Self { - if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) { + if name.starts_with(KAFKA_TIMESTAMP_COLUMN_NAME) || name.starts_with(TABLE_NAME_COLUMN_NAME) + { Self::Meta } else if name == (ROWID_PREFIX) { Self::RowId diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index da6b776cc1db6..9c8db9dde51ee 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -416,6 +416,7 @@ impl TestCase { source_schema, source_watermarks, append_only, + cdc_table_info, .. } => { // TODO(st1page): refacor it @@ -433,6 +434,7 @@ impl TestCase { source_watermarks, append_only, notice, + cdc_table_info, ) .await?; } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index be85293acd27f..657f02b260f9c 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -25,7 +25,7 @@ use risingwave_pb::catalog::{ PbView, }; use risingwave_pb::ddl_service::alter_relation_name_request::Relation; -use risingwave_pb::ddl_service::create_connection_request; +use risingwave_pb::ddl_service::{create_connection_request, PbTableJobType}; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_rpc_client::MetaClient; use tokio::sync::watch::Receiver; @@ -78,6 +78,7 @@ pub trait CatalogWriter: Send + Sync { source: Option, table: PbTable, graph: StreamFragmentGraph, + job_type: PbTableJobType, ) -> Result<()>; async fn replace_table( @@ -99,6 +100,12 @@ pub trait CatalogWriter: Send + Sync { async fn create_source(&self, source: PbSource) -> Result<()>; + async fn create_source_with_graph( + &self, + source: PbSource, + graph: StreamFragmentGraph, + ) -> Result<()>; + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()>; async fn create_function(&self, function: PbFunction) -> Result<()>; @@ -225,8 +232,12 @@ impl CatalogWriter for CatalogWriterImpl { source: Option, table: PbTable, graph: StreamFragmentGraph, + job_type: PbTableJobType, ) -> Result<()> { - let (_, version) = self.meta_client.create_table(source, table, graph).await?; + let (_, version) = self + .meta_client + .create_table(source, table, graph, job_type) + .await?; self.wait_version(version).await } @@ -254,6 +265,18 @@ impl CatalogWriter for CatalogWriterImpl { self.wait_version(version).await } + async fn create_source_with_graph( + &self, + source: PbSource, + graph: StreamFragmentGraph, + ) -> Result<()> { + let (_id, version) = self + .meta_client + .create_source_with_graph(source, graph) + .await?; + self.wait_version(version).await + } + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()> { let (_id, version) = self.meta_client.create_sink(sink, graph).await?; self.wait_version(version).await diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index bed35eadec9ae..38975afd6fe52 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -35,6 +35,7 @@ use crate::catalog::CatalogError; use crate::expr::{Expr, ExprImpl, InputRef}; use crate::handler::privilege::ObjectCheckItem; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{Explain, LogicalProject, LogicalScan, StreamMaterialize}; use crate::optimizer::property::{Cardinality, Distribution, Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; @@ -324,7 +325,7 @@ fn assemble_materialize( let logical_scan = LogicalScan::create( table_name, - false, + ScanTableType::default(), table_desc.clone(), // Index table has no indexes. vec![], diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 8bca367351641..637b2b544f67b 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, HashMap}; +use std::rc::Rc; use std::sync::LazyLock; use itertools::Itertools; @@ -31,7 +32,8 @@ use risingwave_connector::parser::{ }; use risingwave_connector::schema::schema_registry::name_strategy_from_str; use risingwave_connector::source::cdc::{ - CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, + CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CITUS_CDC_CONNECTOR, + MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, }; use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; @@ -44,6 +46,7 @@ use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, }; use risingwave_pb::plan_common::{EncodeType, FormatType}; +use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ get_delimiter, AstString, AvroSchema, ColumnDef, ConnectorSchema, CreateSourceStatement, DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceWatermark, @@ -51,17 +54,19 @@ use risingwave_sqlparser::ast::{ use super::RwPgResponse; use crate::binder::Binder; +use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::ColumnId; use crate::expr::Expr; use crate::handler::create_table::{ bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns, bind_sql_pk_names, ensure_table_constraints_supported, ColumnIdGenerator, }; -use crate::handler::util::{get_connector, is_kafka_connector}; +use crate::handler::util::{get_connector, is_cdc_connector, is_kafka_connector}; use crate::handler::HandlerArgs; +use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext}; use crate::session::SessionImpl; use crate::utils::resolve_privatelink_in_with_option; -use crate::{bind_data_type, WithOptions}; +use crate::{bind_data_type, build_graph, OptimizerContext, WithOptions}; pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector"; pub(crate) const CONNECTION_NAME_KEY: &str = "connection.name"; @@ -86,6 +91,26 @@ async fn extract_json_table_schema( } } +pub fn debezium_cdc_source_schema() -> Vec { + let columns = vec![ + ColumnCatalog { + column_desc: ColumnDesc { + data_type: DataType::Jsonb, + column_id: ColumnId::placeholder(), + name: "payload".to_string(), + field_descs: vec![], + type_name: "".to_string(), + generated_or_default_column: None, + description: None, + }, + is_hidden: false, + }, + ColumnCatalog::offset_column(), + ColumnCatalog::cdc_table_name_column(), + ]; + columns +} + fn json_schema_infer_use_schema_registry(schema_config: &Option<(AstString, bool)>) -> bool { match schema_config { None => false, @@ -265,6 +290,7 @@ fn get_name_strategy_or_default(name_strategy: Option) -> Result, + create_cdc_source_job: bool, ) -> Result<(Option>, StreamSourceInfo)> { const MESSAGE_NAME_KEY: &str = "message"; const KEY_MESSAGE_NAME_KEY: &str = "key.message"; @@ -333,12 +359,19 @@ pub(crate) async fn bind_columns_from_source( } (Format::Plain, Encode::Json) => { let schema_config = get_json_schema_location(&mut options)?; + let columns = if create_cdc_source_job { + Some(debezium_cdc_source_schema()) + } else { + extract_json_table_schema(&schema_config, with_properties).await? + }; + ( - extract_json_table_schema(&schema_config, with_properties).await?, + columns, StreamSourceInfo { format: FormatType::Plain as i32, row_encode: EncodeType::Json as i32, use_schema_registry: json_schema_infer_use_schema_registry(&schema_config), + cdc_source_job: create_cdc_source_job, ..Default::default() }, ) @@ -888,6 +921,8 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Bytes], Format::Debezium => vec![Encode::Json], + // support source stream job + Format::Plain => vec![Encode::Json], ), POSTGRES_CDC_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Bytes], @@ -1070,14 +1105,23 @@ pub async fn handle_create_source( session.notice_to_user(notice) }; - let mut with_properties = handler_args.with_options.into_inner().into_iter().collect(); + let mut with_properties = handler_args + .with_options + .clone() + .into_inner() + .into_iter() + .collect(); validate_compatibility(&source_schema, &mut with_properties)?; ensure_table_constraints_supported(&stmt.constraints)?; let sql_pk_names = bind_sql_pk_names(&stmt.columns, &stmt.constraints)?; + // gated the feature with a session variable + let create_cdc_source_job = + is_cdc_connector(&with_properties) && session.config().get_cdc_backfill(); + let (columns_from_resolve_source, source_info) = - bind_columns_from_source(&source_schema, &with_properties).await?; + bind_columns_from_source(&source_schema, &with_properties, create_cdc_source_job).await?; let columns_from_sql = bind_sql_columns(&stmt.columns)?; let mut columns = bind_all_columns( @@ -1095,6 +1139,13 @@ pub async fn handle_create_source( ) .await?; + if create_cdc_source_job { + // set connector to backfill mode + with_properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into()); + // enable cdc sharing mode, which will capture all tables in the given `database.name` + with_properties.insert(CDC_SHARING_MODE_KEY.into(), "true".into()); + } + check_and_add_timestamp_column(&with_properties, &mut columns); let mut col_id_gen = ColumnIdGenerator::new_initial(); @@ -1129,24 +1180,21 @@ pub async fn handle_create_source( check_source_schema(&with_properties, row_id_index, &columns)?; - let row_id_index = row_id_index.map(|index| index as _); let pk_column_ids = pk_column_ids.into_iter().map(Into::into).collect(); - let columns = columns.into_iter().map(|c| c.to_protobuf()).collect_vec(); - let mut with_options = WithOptions::new(with_properties); // resolve privatelink connection for Kafka source let connection_id = resolve_privatelink_in_with_option(&mut with_options, &schema_name, &session)?; - let definition = handler_args.normalized_sql; + let definition = handler_args.normalized_sql.clone(); let source = PbSource { id: TableId::placeholder().table_id, schema_id, database_id, name, - row_id_index, - columns, + row_id_index: row_id_index.map(|idx| idx as u32), + columns: columns.iter().map(|c| c.to_protobuf()).collect_vec(), pk_column_ids, properties: with_options.into_inner().into_iter().collect(), info: Some(source_info), @@ -1161,7 +1209,37 @@ pub async fn handle_create_source( }; let catalog_writer = session.catalog_writer()?; - catalog_writer.create_source(source).await?; + + if create_cdc_source_job { + // create a streaming job for the cdc source, which will mark as *singleton* in the Fragmenter + let graph = { + let context = OptimizerContext::from_handler_args(handler_args); + // cdc source is an append-only source in plain json format + let source_node = LogicalSource::new( + Some(Rc::new(SourceCatalog::from(&source))), + columns.clone(), + row_id_index, + false, + false, + context.into(), + )?; + + // generate stream graph for cdc source job + let stream_plan = source_node.to_stream(&mut ToStreamContext::new(false))?; + let mut graph = build_graph(stream_plan); + graph.parallelism = session + .config() + .get_streaming_parallelism() + .map(|parallelism| Parallelism { parallelism }); + graph + }; + catalog_writer + .create_source_with_graph(source, graph) + .await?; + } else { + // For other sources we don't create a streaming job + catalog_writer.create_source(source).await?; + } Ok(PgResponse::empty_result(StatementType::CREATE_SOURCE)) } @@ -1171,7 +1249,8 @@ pub mod tests { use std::collections::HashMap; use risingwave_common::catalog::{ - row_id_column_name, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, + cdc_table_name_column_name, offset_column_name, row_id_column_name, DEFAULT_DATABASE_NAME, + DEFAULT_SCHEMA_NAME, }; use risingwave_common::types::DataType; @@ -1223,4 +1302,45 @@ pub mod tests { }; assert_eq!(columns, expected_columns); } + + #[tokio::test] + async fn test_multi_table_cdc_create_source_handler() { + let sql = + "CREATE SOURCE t2 WITH (connector = 'mysql-cdc') FORMAT PLAIN ENCODE JSON".to_string(); + let frontend = LocalFrontend::new(Default::default()).await; + let session = frontend.session_ref(); + session + .set_config("cdc_backfill", vec!["true".to_string()]) + .unwrap(); + + frontend + .run_sql_with_session(session.clone(), sql) + .await + .unwrap(); + let catalog_reader = session.env().catalog_reader().read_guard(); + let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME); + + // Check source exists. + let (source, _) = catalog_reader + .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t2") + .unwrap(); + assert_eq!(source.name, "t2"); + + let columns = source + .columns + .iter() + .map(|col| (col.name(), col.data_type().clone())) + .collect::>(); + + let row_id_col_name = row_id_column_name(); + let offset_col_name = offset_column_name(); + let table_name_col_name = cdc_table_name_column_name(); + let expected_columns = maplit::hashmap! { + row_id_col_name.as_str() => DataType::Serial, + "payload" => DataType::Jsonb, + offset_col_name.as_str() => DataType::Varchar, + table_name_col_name.as_str() => DataType::Varchar, + }; + assert_eq!(columns, expected_columns); + } } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index efa4278bcbc99..805931d6c3835 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -12,33 +12,40 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use std::rc::Rc; +use anyhow::anyhow; use fixedbitset::FixedBitSet; use itertools::Itertools; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, TableDesc, TableId, TableVersionId, INITIAL_SOURCE_VERSION_ID, - INITIAL_TABLE_VERSION_ID, USER_COLUMN_ID_OFFSET, + CdcTableDesc, ColumnCatalog, ColumnDesc, TableId, TableVersionId, DEFAULT_SCHEMA_NAME, + INITIAL_SOURCE_VERSION_ID, INITIAL_TABLE_VERSION_ID, USER_COLUMN_ID_OFFSET, }; -use risingwave_common::constants::hummock::TABLE_OPTION_DUMMY_RETENTION_SECOND; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; use risingwave_common::util::value_encoding::DatumToProtoExt; -use risingwave_connector::source::external::ExternalTableType; +use risingwave_connector::source; +use risingwave_connector::source::cdc::{CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY}; +use risingwave_connector::source::external::{ + CdcTableType, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY, +}; use risingwave_pb::catalog::source::OptionalAssociatedTableId; use risingwave_pb::catalog::{PbSource, PbTable, StreamSourceInfo, WatermarkDesc}; +use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{DefaultColumnDesc, GeneratedColumnDesc}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ - ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, ObjectName, - SourceWatermark, TableConstraint, + CdcTableInfo, ColumnDef, ColumnOption, ConnectorSchema, DataType as AstDataType, Format, + ObjectName, SourceWatermark, TableConstraint, }; use super::RwPgResponse; use crate::binder::{bind_data_type, bind_struct_field, Clause}; +use crate::catalog::root_catalog::SchemaPath; +use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; use crate::catalog::{check_valid_column_name, CatalogError, ColumnId}; use crate::expr::{Expr, ExprImpl, ExprRewriter, InlineNowProcTime}; @@ -47,7 +54,7 @@ use crate::handler::create_source::{ check_source_schema, validate_compatibility, UPSTREAM_SOURCE_KEY, }; use crate::handler::HandlerArgs; -use crate::optimizer::plan_node::LogicalSource; +use crate::optimizer::plan_node::{LogicalScan, LogicalSource}; use crate::optimizer::property::{Order, RequiredDist}; use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot}; use crate::session::{CheckRelationError, SessionImpl}; @@ -461,7 +468,7 @@ pub(crate) async fn gen_create_table_plan_with_source( let sql_pk_names = bind_sql_pk_names(&column_defs, &constraints)?; let (columns_from_resolve_source, mut source_info) = - bind_columns_from_source(&source_schema, &properties).await?; + bind_columns_from_source(&source_schema, &properties, false).await?; let columns_from_sql = bind_sql_columns(&column_defs)?; let mut columns = bind_all_columns( @@ -514,16 +521,10 @@ pub(crate) async fn gen_create_table_plan_with_source( .into()); } - let table_type = ExternalTableType::from_properties(&properties); - if table_type.can_backfill() && context.session_ctx().config().get_cdc_backfill() { - // Add a column for storing the event offset - let offset_column = ColumnCatalog::offset_column(); - let _offset_index = columns.len(); - columns.push(offset_column); - - const CDC_SNAPSHOT_MODE_KEY: &str = "debezium.snapshot.mode"; + let cdc_table_type = CdcTableType::from_properties(&properties); + if cdc_table_type.can_backfill() && context.session_ctx().config().get_cdc_backfill() { // debezium connector will only consume changelogs from latest offset on this mode - properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), "rw_cdc_backfill".into()); + properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into()); let pk_column_indices = { let mut id_to_idx = HashMap::new(); @@ -541,22 +542,19 @@ pub(crate) async fn gen_create_table_plan_with_source( .map(|idx| ColumnOrder::new(*idx, OrderType::ascending())) .collect(); - let upstream_table_desc = TableDesc { + let cdc_table_desc = CdcTableDesc { table_id: TableId::placeholder(), + external_table_name: "".to_string(), pk: table_pk, columns: columns.iter().map(|c| c.column_desc.clone()).collect(), - distribution_key: pk_column_indices.clone(), stream_key: pk_column_indices, - append_only, - retention_seconds: TABLE_OPTION_DUMMY_RETENTION_SECOND, value_indices: (0..columns.len()).collect_vec(), - read_prefix_len_hint: 0, - watermark_columns: Default::default(), - versioned: false, + connect_properties: Default::default(), }; - tracing::debug!("upstream table desc: {:?}", upstream_table_desc); + + tracing::debug!(?cdc_table_desc, "create table with source w/ backfill"); // save external table info to `source_info` - source_info.upstream_table = Some(upstream_table_desc.to_protobuf()); + source_info.external_table = Some(cdc_table_desc.to_protobuf()); } gen_table_plan_inner( @@ -569,6 +567,7 @@ pub(crate) async fn gen_create_table_plan_with_source( Some(source_info), definition, watermark_descs, + Some(cdc_table_type), append_only, Some(col_id_gen.into_version()), ) @@ -647,6 +646,7 @@ pub(crate) fn gen_create_table_plan_without_bind( None, definition, watermark_descs, + None, append_only, version, ) @@ -663,6 +663,7 @@ fn gen_table_plan_inner( source_info: Option, definition: String, watermark_descs: Vec, + cdc_table_type: Option, append_only: bool, version: Option, /* TODO: this should always be `Some` if we support `ALTER * TABLE` for `CREATE TABLE AS`. */ @@ -684,10 +685,18 @@ fn gen_table_plan_inner( database_id, name: name.clone(), row_id_index: row_id_index.map(|i| i as _), - columns: columns - .iter() - .map(|column| column.to_protobuf()) - .collect_vec(), + columns: { + let mut source_columns = columns.clone(); + if let Some(t) = cdc_table_type && t.can_backfill() { + // Append the offset column to be used in the cdc backfill + let offset_column = ColumnCatalog::offset_column(); + source_columns.push(offset_column); + } + source_columns + .iter() + .map(|column| column.to_protobuf()) + .collect_vec() + }, pk_column_ids: pk_column_ids.iter().map(Into::into).collect_vec(), properties: with_options.into_inner().into_iter().collect(), info: Some(source_info), @@ -756,17 +765,167 @@ fn gen_table_plan_inner( Ok((materialize.into(), source, table)) } +#[allow(clippy::too_many_arguments)] +fn gen_create_table_plan_for_cdc_source( + context: OptimizerContextRef, + source_name: ObjectName, + table_name: ObjectName, + external_table_name: String, + column_defs: Vec, + constraints: Vec, + mut col_id_gen: ColumnIdGenerator, +) -> Result<(PlanRef, PbTable)> { + let session = context.session_ctx().clone(); + let db_name = session.database(); + let (schema_name, name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema_name.clone())?; + + // cdc table cannot be append-only + let append_only = false; + let source_name = source_name.real_value(); + + let source = { + let catalog_reader = session.env().catalog_reader().read_guard(); + let schema_name = schema_name + .clone() + .unwrap_or(DEFAULT_SCHEMA_NAME.to_string()); + let (source, _) = catalog_reader.get_source_by_name( + db_name, + SchemaPath::Name(schema_name.as_str()), + source_name.as_str(), + )?; + source.clone() + }; + + let mut columns = bind_sql_columns(&column_defs)?; + + for c in &mut columns { + c.column_desc.column_id = col_id_gen.generate(c.name()) + } + + let pk_names = bind_sql_pk_names(&column_defs, &constraints)?; + let (columns, pk_column_ids, _) = bind_pk_on_relation(columns, pk_names)?; + + let definition = context.normalized_sql().to_owned(); + + let pk_column_indices = { + let mut id_to_idx = HashMap::new(); + columns.iter().enumerate().for_each(|(idx, c)| { + id_to_idx.insert(c.column_id(), idx); + }); + // pk column id must exist in table columns. + pk_column_ids + .iter() + .map(|c| id_to_idx.get(c).copied().unwrap()) + .collect_vec() + }; + let table_pk = pk_column_indices + .iter() + .map(|idx| ColumnOrder::new(*idx, OrderType::ascending())) + .collect(); + + let connect_properties = + derive_connect_properties(source.as_ref(), external_table_name.clone())?; + + let cdc_table_desc = CdcTableDesc { + table_id: source.id.into(), // source can be considered as an external table + external_table_name: external_table_name.clone(), + pk: table_pk, + columns: columns.iter().map(|c| c.column_desc.clone()).collect(), + stream_key: pk_column_indices, + value_indices: (0..columns.len()).collect_vec(), + connect_properties, + }; + + tracing::debug!(?cdc_table_desc, "create cdc table"); + + let logical_scan = LogicalScan::create_for_cdc( + external_table_name, + Rc::new(cdc_table_desc), + context.clone(), + ); + + let scan_node: PlanRef = logical_scan.into(); + let required_cols = FixedBitSet::with_capacity(columns.len()); + let mut plan_root = PlanRoot::new( + scan_node, + RequiredDist::Any, + Order::any(), + required_cols, + vec![], + ); + + let materialize = plan_root.gen_table_plan( + context, + name, + columns, + definition, + pk_column_ids, + None, + append_only, + vec![], // no watermarks + Some(col_id_gen.into_version()), + )?; + + let mut table = materialize.table().to_prost(schema_id, database_id); + table.owner = session.user_id(); + Ok((materialize.into(), table)) +} + +fn derive_connect_properties( + source: &SourceCatalog, + external_table_name: String, +) -> Result> { + use source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR}; + // we should remove the prefix from `full_table_name` + let mut connect_properties = source.properties.clone(); + if let Some(connector) = source.properties.get(UPSTREAM_SOURCE_KEY) { + let table_name = match connector.as_str() { + MYSQL_CDC_CONNECTOR => { + let db_name = connect_properties.get(DATABASE_NAME_KEY).ok_or_else(|| { + anyhow!("{} not found in source properties", DATABASE_NAME_KEY) + })?; + + let prefix = format!("{}.", db_name.as_str()); + external_table_name + .strip_prefix(prefix.as_str()) + .ok_or_else(|| anyhow!("external table name must contain database prefix"))? + } + POSTGRES_CDC_CONNECTOR => { + let schema_name = connect_properties + .get(SCHEMA_NAME_KEY) + .ok_or_else(|| anyhow!("{} not found in source properties", SCHEMA_NAME_KEY))?; + + let prefix = format!("{}.", schema_name.as_str()); + external_table_name + .strip_prefix(prefix.as_str()) + .ok_or_else(|| anyhow!("external table name must contain schema prefix"))? + } + _ => { + return Err(RwError::from(anyhow!( + "connector {} is not supported for cdc table", + connector + ))); + } + }; + connect_properties.insert(TABLE_NAME_KEY.into(), table_name.into()); + } + Ok(connect_properties.into_iter().collect()) +} + #[allow(clippy::too_many_arguments)] pub async fn handle_create_table( handler_args: HandlerArgs, table_name: ObjectName, - columns: Vec, + column_defs: Vec, constraints: Vec, if_not_exists: bool, source_schema: Option, source_watermarks: Vec, append_only: bool, notice: Option, + cdc_table_info: Option, ) -> Result { let session = handler_args.session.clone(); // TODO(st1page): refactor it @@ -788,42 +947,65 @@ pub async fn handle_create_table( Ok(_) => {} }; - let (graph, source, table) = { + let (graph, source, table, job_type) = { let context = OptimizerContext::from_handler_args(handler_args); let source_schema = check_create_table_with_source(context.with_options(), source_schema)?; let col_id_gen = ColumnIdGenerator::new_initial(); - let (plan, source, table) = match source_schema { - Some(source_schema) => { + let ((plan, source, table), job_type) = match (source_schema, cdc_table_info.as_ref()) { + (Some(source_schema), None) => ( gen_create_table_plan_with_source( context, table_name.clone(), - columns, + column_defs, constraints, source_schema, source_watermarks, col_id_gen, append_only, ) - .await? + .await?, + TableJobType::General, + ), + (None, None) => ( + gen_create_table_plan( + context, + table_name.clone(), + column_defs, + constraints, + col_id_gen, + source_watermarks, + append_only, + )?, + TableJobType::General, + ), + + (None, Some(cdc_table)) => { + let (plan, table) = gen_create_table_plan_for_cdc_source( + context.into(), + cdc_table.source_name.clone(), + table_name.clone(), + cdc_table.external_table_name.clone(), + column_defs, + constraints, + col_id_gen, + )?; + + ((plan, None, table), TableJobType::SharedCdcSource) } - None => gen_create_table_plan( - context, - table_name.clone(), - columns, - constraints, - col_id_gen, - source_watermarks, - append_only, - )?, + (Some(_), Some(_)) => return Err(ErrorCode::NotSupported( + "Data format and encoding format doesn't apply to table created from a CDC source" + .into(), + "Remove the FORMAT and ENCODE specification".into(), + ) + .into()), }; - let mut graph = build_graph(plan); graph.parallelism = session .config() .get_streaming_parallelism() .map(|parallelism| Parallelism { parallelism }); - (graph, source, table) + (graph, source, table, job_type) }; tracing::trace!( @@ -833,7 +1015,9 @@ pub async fn handle_create_table( ); let catalog_writer = session.catalog_writer()?; - catalog_writer.create_table(source, table, graph).await?; + catalog_writer + .create_table(source, table, graph, job_type) + .await?; Ok(PgResponse::empty_result(StatementType::CREATE_TABLE)) } diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index 14b8390a902fa..accea93ca2837 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -15,6 +15,7 @@ use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_common::catalog::{ColumnCatalog, ColumnDesc}; use risingwave_common::error::{ErrorCode, Result}; +use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; use risingwave_sqlparser::ast::{ColumnDef, ObjectName, Query, Statement}; @@ -124,7 +125,9 @@ pub async fn handle_create_as( ); let catalog_writer = session.catalog_writer()?; - catalog_writer.create_table(source, table, graph).await?; + catalog_writer + .create_table(source, table, graph, TableJobType::Unspecified) + .await?; // Generate insert let insert = Statement::Insert { diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 8275551fbc4a6..fadac730fdbb7 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -224,6 +224,7 @@ pub async fn handle( source_schema, source_watermarks, append_only, + cdc_table_info, } => { if or_replace { return Err(ErrorCode::NotImplemented( @@ -267,6 +268,7 @@ pub async fn handle( source_watermarks, append_only, notice, + cdc_table_info, ) .await } diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index 7c1eca3fa9bbf..4ddb5419357ff 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -254,6 +254,14 @@ pub fn is_kafka_connector(with_properties: &HashMap) -> bool { connector == KAFKA_CONNECTOR } +#[inline(always)] +pub fn is_cdc_connector(with_properties: &HashMap) -> bool { + let Some(connector) = get_connector(with_properties) else { + return false; + }; + connector.contains("-cdc") +} + #[inline(always)] pub fn get_connection_name(with_properties: &BTreeMap) -> Option { with_properties diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 20da006dcc992..dd38a2f6f089a 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -618,7 +618,7 @@ fn exist_and_no_exchange_before(plan: &PlanRef, is_candidate: fn(&PlanRef) -> bo fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table) + .map(|node| !node.core().is_sys_table()) .unwrap_or(false) } @@ -651,7 +651,7 @@ fn require_additional_exchange_on_root_in_distributed_mode(plan: PlanRef) -> boo fn require_additional_exchange_on_root_in_local_mode(plan: PlanRef) -> bool { fn is_user_table(plan: &PlanRef) -> bool { plan.as_batch_seq_scan() - .map(|node| !node.core().is_sys_table) + .map(|node| !node.core().is_sys_table()) .unwrap_or(false) } diff --git a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs index 7a2d97c266b36..65487e537420c 100644 --- a/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs +++ b/src/frontend/src/optimizer/plan_node/batch_seq_scan.rs @@ -78,7 +78,7 @@ impl BatchSeqScan { fn clone_with_dist(&self) -> Self { Self::new_inner( self.core.clone(), - if self.core.is_sys_table { + if self.core.is_sys_table() { Distribution::Single } else { match self.core.distribution_key() { @@ -221,7 +221,7 @@ impl ToBatchPb for BatchSeqScan { .map(PbColumnDesc::from) .collect(); - if self.core.is_sys_table { + if self.core.is_sys_table() { NodeBody::SysRowSeqScan(SysRowSeqScanNode { table_id: self.core.table_desc.table_id.table_id, column_descs, @@ -250,7 +250,7 @@ impl ToBatchPb for BatchSeqScan { impl ToLocalBatch for BatchSeqScan { fn to_local(&self) -> Result { - let dist = if self.core.is_sys_table { + let dist = if self.core.is_sys_table() { Distribution::Single } else if let Some(distribution_key) = self.core.distribution_key() && !distribution_key.is_empty() diff --git a/src/frontend/src/optimizer/plan_node/generic/scan.rs b/src/frontend/src/optimizer/plan_node/generic/scan.rs index b7de99d11096b..7d83ac0ba7e4f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/scan.rs +++ b/src/frontend/src/optimizer/plan_node/generic/scan.rs @@ -18,7 +18,7 @@ use std::rc::Rc; use educe::Educe; use fixedbitset::FixedBitSet; use pretty_xmlish::Pretty; -use risingwave_common::catalog::{ColumnDesc, Field, Schema, TableDesc}; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, Field, Schema, TableDesc}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::sort_util::ColumnOrder; @@ -29,18 +29,29 @@ use crate::optimizer::optimizer_context::OptimizerContextRef; use crate::optimizer::property::{Cardinality, FunctionalDependencySet, Order}; use crate::utils::{ColIndexMappingRewriteExt, Condition}; +#[derive(Debug, Default, Clone, Educe)] +#[educe(PartialEq, Eq, Hash)] +pub enum ScanTableType { + #[default] + General, + SysTable, + CdcTable, +} + /// [`Scan`] returns contents of a table or other equivalent object #[derive(Debug, Clone, Educe)] #[educe(PartialEq, Eq, Hash)] pub struct Scan { pub table_name: String, - pub is_sys_table: bool, + pub scan_table_type: ScanTableType, /// Include `output_col_idx` and columns required in `predicate` pub required_col_idx: Vec, pub output_col_idx: Vec, - // Descriptor of the table + /// Descriptor of the table pub table_desc: Rc, - // Descriptors of all indexes on this table + /// Descriptor of the external table for CDC + pub cdc_table_desc: Rc, + /// Descriptors of all indexes on this table pub indexes: Vec>, /// The pushed down predicates. It refers to column indexes of the table. pub predicate: Condition, @@ -67,6 +78,10 @@ impl Scan { /// /// Return `None` if the table's distribution key are not all in the `output_col_idx`. pub fn distribution_key(&self) -> Option> { + if self.is_cdc_table() { + return None; + } + let tb_idx_to_op_idx = self .output_col_idx .iter() @@ -84,30 +99,38 @@ impl Scan { pub fn output_column_ids(&self) -> Vec { self.output_col_idx .iter() - .map(|i| self.table_desc.columns[*i].column_id) + .map(|i| self.get_table_columns()[*i].column_id) .collect() } pub fn primary_key(&self) -> &[ColumnOrder] { - &self.table_desc.pk + if self.is_cdc_table() { + &self.cdc_table_desc.pk + } else { + &self.table_desc.pk + } } pub fn watermark_columns(&self) -> FixedBitSet { - let watermark_columns = &self.table_desc.watermark_columns; - self.i2o_col_mapping().rewrite_bitset(watermark_columns) + if self.is_cdc_table() { + FixedBitSet::with_capacity(self.get_table_columns().len()) + } else { + let watermark_columns = &self.table_desc.watermark_columns; + self.i2o_col_mapping().rewrite_bitset(watermark_columns) + } } pub(crate) fn column_names_with_table_prefix(&self) -> Vec { self.output_col_idx .iter() - .map(|&i| format!("{}.{}", self.table_name, self.table_desc.columns[i].name)) + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) .collect() } pub(crate) fn column_names(&self) -> Vec { self.output_col_idx .iter() - .map(|&i| self.table_desc.columns[i].name.clone()) + .map(|&i| self.get_table_columns()[i].name.clone()) .collect() } @@ -115,7 +138,7 @@ impl Scan { self.table_desc .order_column_indices() .iter() - .map(|&i| self.table_desc.columns[i].name.clone()) + .map(|&i| self.get_table_columns()[i].name.clone()) .collect() } @@ -123,7 +146,7 @@ impl Scan { self.table_desc .order_column_indices() .iter() - .map(|&i| format!("{}.{}", self.table_name, self.table_desc.columns[i].name)) + .map(|&i| format!("{}.{}", self.table_name, self.get_table_columns()[i].name)) .collect() } @@ -148,14 +171,17 @@ impl Scan { /// get the Mapping of columnIndex from internal column index to output column index pub fn i2o_col_mapping(&self) -> ColIndexMapping { - ColIndexMapping::with_remaining_columns(&self.output_col_idx, self.table_desc.columns.len()) + ColIndexMapping::with_remaining_columns( + &self.output_col_idx, + self.get_table_columns().len(), + ) } /// Get the ids of the output columns and primary key columns. pub fn output_and_pk_column_ids(&self) -> Vec { let mut ids = self.output_column_ids(); for column_order in self.primary_key() { - let id = self.table_desc.columns[column_order.column_index].column_id; + let id = self.get_table_columns()[column_order.column_index].column_id; if !ids.contains(&id) { ids.push(id); } @@ -216,7 +242,7 @@ impl Scan { Self::new( index_name.to_string(), - false, + ScanTableType::default(), new_output_col_idx, index_table_desc, vec![], @@ -231,7 +257,7 @@ impl Scan { #[allow(clippy::too_many_arguments)] pub(crate) fn new( table_name: String, - is_sys_table: bool, + scan_table_type: ScanTableType, output_col_idx: Vec, // the column index in the table table_desc: Rc, indexes: Vec>, @@ -239,6 +265,54 @@ impl Scan { predicate: Condition, // refers to column indexes of the table for_system_time_as_of_proctime: bool, table_cardinality: Cardinality, + ) -> Self { + Self::new_inner( + table_name, + scan_table_type, + output_col_idx, + table_desc, + Rc::new(CdcTableDesc::default()), + indexes, + ctx, + predicate, + for_system_time_as_of_proctime, + table_cardinality, + ) + } + + /// Create a logical scan node for CDC backfill + pub(crate) fn new_for_cdc( + table_name: String, + output_col_idx: Vec, // the column index in the table + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + Self::new_inner( + table_name, + ScanTableType::CdcTable, + output_col_idx, + Rc::new(TableDesc::default()), + cdc_table_desc, + vec![], + ctx, + Condition::true_cond(), + false, + Cardinality::unknown(), + ) + } + + #[allow(clippy::too_many_arguments)] + pub(crate) fn new_inner( + table_name: String, + scan_table_type: ScanTableType, + output_col_idx: Vec, // the column index in the table + table_desc: Rc, + cdc_table_desc: Rc, + indexes: Vec>, + ctx: OptimizerContextRef, + predicate: Condition, // refers to column indexes of the table + for_system_time_as_of_proctime: bool, + table_cardinality: Cardinality, ) -> Self { // here we have 3 concepts // 1. column_id: ColumnId, stored in catalog and a ID to access data from storage. @@ -258,10 +332,11 @@ impl Scan { Self { table_name, - is_sys_table, + scan_table_type, required_col_idx, output_col_idx, table_desc, + cdc_table_desc, indexes, predicate, chunk_size: None, @@ -294,13 +369,14 @@ impl Scan { } } +// TODO: extend for cdc table impl GenericPlanNode for Scan { fn schema(&self) -> Schema { let fields = self .output_col_idx .iter() .map(|tb_idx| { - let col = &self.table_desc.columns[*tb_idx]; + let col = &self.get_table_columns()[*tb_idx]; Field::from_with_table_name_prefix(col, &self.table_name) }) .collect(); @@ -308,16 +384,21 @@ impl GenericPlanNode for Scan { } fn stream_key(&self) -> Option> { - let id_to_op_idx = Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); - self.table_desc - .stream_key - .iter() - .map(|&c| { - id_to_op_idx - .get(&self.table_desc.columns[c].column_id) - .copied() - }) - .collect::>>() + if self.is_cdc_table() { + Some(self.cdc_table_desc.stream_key.clone()) + } else { + let id_to_op_idx = + Self::get_id_to_op_idx_mapping(&self.output_col_idx, &self.table_desc); + self.table_desc + .stream_key + .iter() + .map(|&c| { + id_to_op_idx + .get(&self.table_desc.columns[c].column_id) + .copied() + }) + .collect::>>() + } } fn ctx(&self) -> OptimizerContextRef { @@ -335,11 +416,34 @@ impl GenericPlanNode for Scan { } impl Scan { + pub fn get_table_columns(&self) -> &[ColumnDesc] { + if self.is_cdc_table() { + &self.cdc_table_desc.columns + } else { + &self.table_desc.columns + } + } + + pub fn is_sys_table(&self) -> bool { + matches!(self.scan_table_type, ScanTableType::SysTable) + } + + pub fn is_cdc_table(&self) -> bool { + matches!(self.scan_table_type, ScanTableType::CdcTable) + } + + pub fn append_only(&self) -> bool { + if self.is_cdc_table() { + return false; + } + self.table_desc.append_only + } + /// Get the descs of the output columns. pub fn column_descs(&self) -> Vec { self.output_col_idx .iter() - .map(|&i| self.table_desc.columns[i].clone()) + .map(|&i| self.get_table_columns()[i].clone()) .collect() } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index c9f5494b86be6..bde0eeba44524 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -18,9 +18,10 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{ColumnDesc, TableDesc}; +use risingwave_common::catalog::{CdcTableDesc, ColumnDesc, TableDesc}; use risingwave_common::error::{ErrorCode, Result, RwError}; use risingwave_common::util::sort_util::ColumnOrder; +use risingwave_pb::stream_plan::ChainType; use super::generic::{GenericPlanNode, GenericPlanRef}; use super::utils::{childless_record, Distill}; @@ -31,6 +32,7 @@ use super::{ use crate::catalog::{ColumnId, IndexCatalog}; use crate::expr::{CorrelatedInputRef, ExprImpl, ExprRewriter, ExprVisitor, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ BatchSeqScan, ColumnPruningContext, LogicalFilter, LogicalProject, LogicalValues, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, @@ -63,7 +65,7 @@ impl LogicalScan { /// Create a [`LogicalScan`] node. Used by planner. pub fn create( table_name: String, // explain-only - is_sys_table: bool, + scan_table_type: ScanTableType, table_desc: Rc, indexes: Vec>, ctx: OptimizerContextRef, @@ -72,7 +74,7 @@ impl LogicalScan { ) -> Self { generic::Scan::new( table_name, - is_sys_table, + scan_table_type, (0..table_desc.columns.len()).collect(), table_desc, indexes, @@ -84,12 +86,34 @@ impl LogicalScan { .into() } + pub fn create_for_cdc( + table_name: String, // explain-only + cdc_table_desc: Rc, + ctx: OptimizerContextRef, + ) -> Self { + generic::Scan::new_for_cdc( + table_name, + (0..cdc_table_desc.columns.len()).collect(), + cdc_table_desc, + ctx, + ) + .into() + } + pub fn table_name(&self) -> &str { &self.core.table_name } + pub fn scan_table_type(&self) -> &ScanTableType { + &self.core.scan_table_type + } + pub fn is_sys_table(&self) -> bool { - self.core.is_sys_table + self.core.is_sys_table() + } + + pub fn is_cdc_table(&self) -> bool { + matches!(self.core.scan_table_type, ScanTableType::CdcTable) } pub fn for_system_time_as_of_proctime(&self) -> bool { @@ -106,6 +130,10 @@ impl LogicalScan { self.core.table_desc.as_ref() } + pub fn cdc_table_desc(&self) -> &CdcTableDesc { + self.core.cdc_table_desc.as_ref() + } + /// Get the descs of the output columns. pub fn column_descs(&self) -> Vec { self.core.column_descs() @@ -249,7 +277,7 @@ impl LogicalScan { let scan_without_predicate = generic::Scan::new( self.table_name().to_string(), - self.is_sys_table(), + self.scan_table_type().clone(), self.required_col_idx().to_vec(), self.core.table_desc.clone(), self.indexes().to_vec(), @@ -267,11 +295,12 @@ impl LogicalScan { } fn clone_with_predicate(&self, predicate: Condition) -> Self { - generic::Scan::new( + generic::Scan::new_inner( self.table_name().to_string(), - self.is_sys_table(), + self.scan_table_type().clone(), self.output_col_idx().to_vec(), self.core.table_desc.clone(), + self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), predicate, @@ -282,11 +311,12 @@ impl LogicalScan { } pub fn clone_with_output_indices(&self, output_col_idx: Vec) -> Self { - generic::Scan::new( + generic::Scan::new_inner( self.table_name().to_string(), - self.is_sys_table(), + self.scan_table_type().clone(), output_col_idx, self.core.table_desc.clone(), + self.core.cdc_table_desc.clone(), self.indexes().to_vec(), self.base.ctx().clone(), self.predicate().clone(), @@ -394,6 +424,11 @@ impl PredicatePushdown for LogicalScan { mut predicate: Condition, _ctx: &mut PredicatePushdownContext, ) -> PlanRef { + // skip pushdown if the table is cdc table + if self.is_cdc_table() { + return self.clone().into(); + } + // If the predicate contains `CorrelatedInputRef` or `now()`. We don't push down. // This case could come from the predicate push down before the subquery unnesting. struct HasCorrelated {} @@ -530,7 +565,14 @@ impl ToStream for LogicalScan { ))); } if self.predicate().always_true() { - Ok(StreamTableScan::new(self.core.clone()).into()) + if self.is_cdc_table() { + Ok( + StreamTableScan::new_with_chain_type(self.core.clone(), ChainType::CdcBackfill) + .into(), + ) + } else { + Ok(StreamTableScan::new(self.core.clone()).into()) + } } else { let (scan, predicate, project_expr) = self.predicate_pull_up(); let mut plan = LogicalFilter::create(scan.into(), predicate); @@ -551,6 +593,14 @@ impl ToStream for LogicalScan { None.into(), ))); } + + if self.is_cdc_table() { + return Ok(( + self.clone().into(), + ColIndexMapping::identity(self.schema().len()), + )); + } + match self.base.stream_key().is_none() { true => { let mut col_ids = HashSet::new(); diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index d9d28fce298ad..f2f25a5352d65 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -115,6 +115,7 @@ impl StreamNode for StreamExchange { r#type: DispatcherType::NoShuffle as i32, dist_key_indices: vec![], output_indices: (0..self.schema().len() as u32).collect(), + downstream_table_name: None, }) } else { Some(DispatchStrategy { @@ -131,6 +132,7 @@ impl StreamNode for StreamExchange { _ => vec![], }, output_indices: (0..self.schema().len() as u32).collect(), + downstream_table_name: None, }) }, }) diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 882410e5c9010..3979aceae757e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -17,7 +17,7 @@ use std::rc::Rc; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{Field, TableDesc}; +use risingwave_common::catalog::{ColumnCatalog, Field, TableDesc}; use risingwave_common::hash::VirtualNode; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; @@ -29,6 +29,7 @@ use super::utils::{childless_record, Distill}; use super::{generic, ExprRewritable, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; use crate::expr::{ExprRewriter, FunctionCall}; +use crate::handler::create_source::debezium_cdc_source_schema; use crate::optimizer::plan_node::utils::{IndicesDisplay, TableCatalogBuilder}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -53,6 +54,7 @@ impl StreamTableScan { pub fn new_with_chain_type(core: generic::Scan, chain_type: ChainType) -> Self { let batch_plan_id = core.ctx.next_plan_node_id(); + // TODO: correctly derive the distribution for cdc backfill let distribution = { match core.distribution_key() { Some(distribution_key) => { @@ -66,10 +68,11 @@ impl StreamTableScan { None => Distribution::SomeShard, } }; + let base = PlanBase::new_stream_with_core( &core, distribution, - core.table_desc.append_only, + core.append_only(), false, core.watermark_columns(), ); @@ -150,7 +153,7 @@ impl StreamTableScan { ) -> TableCatalog { let properties = self.ctx().with_options().internal_table_subset(); let mut catalog_builder = TableCatalogBuilder::new(properties); - let upstream_schema = &self.core.table_desc.columns; + let upstream_schema = &self.core.get_table_columns(); // We use vnode as primary key in state table. // If `Distribution::Single`, vnode will just be `VirtualNode::default()`. @@ -236,10 +239,13 @@ impl StreamTableScan { .map(|x| *x as u32) .collect_vec(); + // A flag to mark whether the upstream is a cdc source job + let cdc_upstream = matches!(self.chain_type, ChainType::CdcBackfill); + // The required columns from the table (both scan and upstream). let upstream_column_ids = match self.chain_type { // For backfill, we additionally need the primary key columns. - ChainType::Backfill => self.core.output_and_pk_column_ids(), + ChainType::Backfill | ChainType::CdcBackfill => self.core.output_and_pk_column_ids(), ChainType::Chain | ChainType::Rearrange | ChainType::UpstreamOnly => { self.core.output_column_ids() } @@ -249,14 +255,13 @@ impl StreamTableScan { .map(ColumnId::get_id) .collect_vec(); - // The schema of the upstream table (both scan and upstream). - let upstream_schema = upstream_column_ids + // The schema of the snapshot read stream + let snapshot_schema = upstream_column_ids .iter() .map(|&id| { let col = self .core - .table_desc - .columns + .get_table_columns() .iter() .find(|c| c.column_id.get_id() == id) .unwrap(); @@ -264,6 +269,19 @@ impl StreamTableScan { }) .collect_vec(); + // The schema of the shared cdc source upstream is different from snapshot, + // refer to `debezium_cdc_source_schema()` for details. + let upstream_schema = if cdc_upstream { + let mut columns = debezium_cdc_source_schema(); + columns.push(ColumnCatalog::row_id_column()); + columns + .into_iter() + .map(|c| Field::from(c.column_desc).to_prost()) + .collect_vec() + } else { + snapshot_schema.clone() + }; + let output_indices = self .core .output_column_ids() @@ -276,9 +294,12 @@ impl StreamTableScan { }) .collect_vec(); - // TODO: snapshot read of upstream mview let batch_plan_node = BatchPlanNode { - table_desc: Some(self.core.table_desc.to_protobuf()), + table_desc: if cdc_upstream { + None + } else { + Some(self.core.table_desc.to_protobuf()) + }, column_ids: upstream_column_ids.clone(), }; @@ -286,6 +307,40 @@ impl StreamTableScan { .build_backfill_state_catalog(state) .to_internal_table_prost(); + let node_body = if cdc_upstream { + // don't need batch plan for cdc source + PbNodeBody::Chain(ChainNode { + table_id: self.core.cdc_table_desc.table_id.table_id, + chain_type: self.chain_type as i32, + // The column indices need to be forwarded to the downstream + output_indices, + upstream_column_ids, + // The table desc used by backfill executor + state_table: Some(catalog), + rate_limit: None, + cdc_table_desc: Some(self.core.cdc_table_desc.to_protobuf()), + ..Default::default() + }) + } else { + PbNodeBody::Chain(ChainNode { + table_id: self.core.table_desc.table_id.table_id, + chain_type: self.chain_type as i32, + // The column indices need to be forwarded to the downstream + output_indices, + upstream_column_ids, + // The table desc used by backfill executor + table_desc: Some(self.core.table_desc.to_protobuf()), + state_table: Some(catalog), + rate_limit: self + .base + .ctx() + .session_ctx() + .config() + .get_streaming_rate_limit(), + ..Default::default() + }) + }; + PbStreamNode { fields: self.schema().to_prost(), input: vec![ @@ -301,29 +356,14 @@ impl StreamTableScan { node_body: Some(PbNodeBody::BatchPlan(batch_plan_node)), operator_id: self.batch_plan_id.0 as u64, identity: "BatchPlanNode".into(), - fields: upstream_schema, + fields: snapshot_schema, stream_key: vec![], // not used input: vec![], append_only: true, }, ], - node_body: Some(PbNodeBody::Chain(ChainNode { - table_id: self.core.table_desc.table_id.table_id, - chain_type: self.chain_type as i32, - // The column indices need to be forwarded to the downstream - output_indices, - upstream_column_ids, - // The table desc used by backfill executor - table_desc: Some(self.core.table_desc.to_protobuf()), - state_table: Some(catalog), - rate_limit: self - .base - .ctx() - .session_ctx() - .config() - .get_streaming_rate_limit(), - ..Default::default() - })), + + node_body: Some(node_body), stream_key, operator_id: self.base.id().0 as u64, identity: { diff --git a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs index 770f099aab529..e899021b8bdef 100644 --- a/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/relation_collector_visitor.rs @@ -52,7 +52,7 @@ impl PlanVisitor for RelationCollectorVisitor { } fn visit_batch_seq_scan(&mut self, plan: &crate::optimizer::plan_node::BatchSeqScan) { - if !plan.core().is_sys_table { + if !plan.core().is_sys_table() { self.relations.insert(plan.core().table_desc.table_id); } } @@ -65,7 +65,7 @@ impl PlanVisitor for RelationCollectorVisitor { fn visit_stream_table_scan(&mut self, plan: &StreamTableScan) { let logical = plan.core(); - if !logical.is_sys_table { + if !logical.is_sys_table() { self.relations.insert(logical.table_desc.table_id); } } diff --git a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs index dcbfb2d93d3f5..e93a5c4e0bb1c 100644 --- a/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs +++ b/src/frontend/src/optimizer/plan_visitor/sys_table_visitor.rs @@ -37,7 +37,7 @@ impl PlanVisitor for SysTableVisitor { } fn visit_batch_seq_scan(&mut self, batch_seq_scan: &BatchSeqScan) -> bool { - batch_seq_scan.core().is_sys_table + batch_seq_scan.core().is_sys_table() } fn visit_logical_scan(&mut self, logical_scan: &LogicalScan) -> bool { @@ -45,6 +45,6 @@ impl PlanVisitor for SysTableVisitor { } fn visit_stream_table_scan(&mut self, stream_table_scan: &StreamTableScan) -> bool { - stream_table_scan.core().is_sys_table + stream_table_scan.core().is_sys_table() } } diff --git a/src/frontend/src/optimizer/rule/index_selection_rule.rs b/src/frontend/src/optimizer/rule/index_selection_rule.rs index 323cc59ef3558..7aeefe4f9154e 100644 --- a/src/frontend/src/optimizer/rule/index_selection_rule.rs +++ b/src/frontend/src/optimizer/rule/index_selection_rule.rs @@ -66,7 +66,7 @@ use crate::expr::{ FunctionCall, InputRef, }; use crate::optimizer::optimizer_context::OptimizerContextRef; -use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::generic::{GenericPlanRef, ScanTableType}; use crate::optimizer::plan_node::{ generic, ColumnPruningContext, LogicalJoin, LogicalScan, LogicalUnion, PlanTreeNode, PlanTreeNodeBinary, PredicatePushdown, PredicatePushdownContext, @@ -222,7 +222,7 @@ impl IndexSelectionRule { let index_scan = LogicalScan::create( index.index_table.name.clone(), - false, + ScanTableType::default(), index.index_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -232,7 +232,7 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( index.primary_table.name.clone(), - false, + ScanTableType::default(), index.primary_table.table_desc().into(), vec![], logical_scan.ctx(), @@ -332,7 +332,7 @@ impl IndexSelectionRule { let primary_table_scan = LogicalScan::create( logical_scan.table_name().to_string(), - false, + ScanTableType::default(), primary_table_desc.clone().into(), vec![], logical_scan.ctx(), @@ -560,7 +560,7 @@ impl IndexSelectionRule { let primary_access = generic::Scan::new( logical_scan.table_name().to_string(), - false, + ScanTableType::default(), primary_table_desc .pk .iter() @@ -603,7 +603,7 @@ impl IndexSelectionRule { Some( generic::Scan::new( index.index_table.name.to_string(), - false, + ScanTableType::default(), index .primary_table_pk_ref_to_index_table() .iter() diff --git a/src/frontend/src/planner/relation.rs b/src/frontend/src/planner/relation.rs index f4085f6ffa42e..d4686bbf15c28 100644 --- a/src/frontend/src/planner/relation.rs +++ b/src/frontend/src/planner/relation.rs @@ -24,6 +24,7 @@ use crate::binder::{ BoundWindowTableFunction, Relation, WindowTableFunctionKind, }; use crate::expr::{Expr, ExprImpl, ExprType, FunctionCall, InputRef}; +use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ LogicalApply, LogicalHopWindow, LogicalJoin, LogicalProject, LogicalScan, LogicalShare, LogicalSource, LogicalTableFunction, LogicalValues, PlanRef, @@ -58,7 +59,7 @@ impl Planner { pub(crate) fn plan_sys_table(&mut self, sys_table: BoundSystemTable) -> Result { Ok(LogicalScan::create( sys_table.sys_table_catalog.name().to_string(), - true, + ScanTableType::SysTable, Rc::new(sys_table.sys_table_catalog.table_desc()), vec![], self.ctx(), @@ -71,7 +72,7 @@ impl Planner { pub(super) fn plan_base_table(&mut self, base_table: &BoundBaseTable) -> Result { Ok(LogicalScan::create( base_table.table_catalog.name().to_string(), - false, + ScanTableType::default(), Rc::new(base_table.table_catalog.table_desc()), base_table .table_indexes diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 6b47e5f5582c8..f35b4771ba13b 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -449,6 +449,7 @@ pub(crate) mod tests { use crate::catalog::catalog_service::CatalogReader; use crate::catalog::root_catalog::Catalog; use crate::expr::InputRef; + use crate::optimizer::plan_node::generic::ScanTableType; use crate::optimizer::plan_node::{ generic, BatchExchange, BatchFilter, BatchHashJoin, EqJoinPredicate, LogicalScan, ToBatch, }; @@ -512,7 +513,7 @@ pub(crate) mod tests { let table_id = 0.into(); let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), - false, + ScanTableType::default(), Rc::new(TableDesc { table_id, stream_key: vec![], diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index cc23256f43687..b20197f9a491d 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -919,7 +919,7 @@ impl BatchPlanFragmenter { if let Some(scan_node) = node.as_batch_seq_scan() { let name = scan_node.core().table_name.to_owned(); - let info = if scan_node.core().is_sys_table { + let info = if scan_node.core().is_sys_table() { TableScanInfo::system_table(name) } else { let table_desc = &*scan_node.core().table_desc; diff --git a/src/frontend/src/stream_fragmenter/mod.rs b/src/frontend/src/stream_fragmenter/mod.rs index d049a7d656a7e..d762223e0b24d 100644 --- a/src/frontend/src/stream_fragmenter/mod.rs +++ b/src/frontend/src/stream_fragmenter/mod.rs @@ -257,8 +257,14 @@ fn build_fragment( current_fragment.fragment_type_mask |= FragmentTypeFlag::BarrierRecv as u32 } - NodeBody::Source(_) => { + NodeBody::Source(node) => { current_fragment.fragment_type_mask |= FragmentTypeFlag::Source as u32; + + if let Some(source) = node.source_inner.as_ref() && + let Some(source_info) = source.info.as_ref() && source_info.cdc_source_job { + tracing::debug!("mark cdc source job as singleton"); + current_fragment.requires_singleton = true; + } } NodeBody::Dml(_) => { @@ -276,6 +282,7 @@ fn build_fragment( NodeBody::Chain(node) => { current_fragment.fragment_type_mask |= FragmentTypeFlag::ChainNode as u32; // memorize table id for later use + // The table id could be a upstream CDC source state .dependent_table_ids .insert(TableId::new(node.table_id)); @@ -356,6 +363,7 @@ fn build_fragment( r#type: DispatcherType::NoShuffle as i32, dist_key_indices: vec![], output_indices: (0..ref_fragment_node.fields.len() as u32).collect(), + downstream_table_name: None, }; let no_shuffle_exchange_operator_id = state.gen_operator_id() as u64; diff --git a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs index b09dc847fc3fd..c108488b2aac7 100644 --- a/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs +++ b/src/frontend/src/stream_fragmenter/rewrite/delta_join.rs @@ -71,6 +71,7 @@ fn dispatch_no_shuffle(output_indices: Vec) -> DispatchStrategy { r#type: DispatcherType::NoShuffle.into(), dist_key_indices: vec![], output_indices, + downstream_table_name: None, } } @@ -83,6 +84,7 @@ fn dispatch_consistent_hash_shuffle( r#type: DispatcherType::Hash.into(), dist_key_indices, output_indices, + downstream_table_name: None, } } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index d78ece19b0e83..6f88ea41a1bbd 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -34,7 +34,7 @@ use risingwave_pb::catalog::table::OptionalAssociatedSourceId; use risingwave_pb::catalog::{ PbComment, PbDatabase, PbFunction, PbIndex, PbSchema, PbSink, PbSource, PbTable, PbView, Table, }; -use risingwave_pb::ddl_service::{create_connection_request, DdlProgress}; +use risingwave_pb::ddl_service::{create_connection_request, DdlProgress, PbTableJobType}; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::{ BranchedObject, CompactionGroupInfo, HummockSnapshot, HummockVersion, HummockVersionDelta, @@ -107,6 +107,15 @@ impl LocalFrontend { self.session_ref().run_statement(sql.as_str(), vec![]).await } + pub async fn run_sql_with_session( + &self, + session_ref: Arc, + sql: impl Into, + ) -> std::result::Result> { + let sql = sql.into(); + session_ref.run_statement(sql.as_str(), vec![]).await + } + pub async fn run_user_sql( &self, sql: impl Into, @@ -254,6 +263,7 @@ impl CatalogWriter for MockCatalogWriter { source: Option, mut table: PbTable, graph: StreamFragmentGraph, + _job_type: PbTableJobType, ) -> Result<()> { if let Some(source) = source { let source_id = self.create_source_inner(source)?; @@ -279,6 +289,14 @@ impl CatalogWriter for MockCatalogWriter { self.create_source_inner(source).map(|_| ()) } + async fn create_source_with_graph( + &self, + source: PbSource, + _graph: StreamFragmentGraph, + ) -> Result<()> { + self.create_source_inner(source).map(|_| ()) + } + async fn create_sink(&self, sink: PbSink, graph: StreamFragmentGraph) -> Result<()> { self.create_sink_inner(sink, graph) } diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index 500167f9380de..f4a299f1aba4a 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -148,6 +148,8 @@ impl StreamGraphFormatter { DispatcherType::Broadcast => "Broadcast".to_string(), DispatcherType::Simple => "Single".to_string(), DispatcherType::NoShuffle => "NoShuffle".to_string(), + DispatcherType::CdcTablename => + format!("CdcTableName({:?})", dist.downstream_table_name), }, upstream_fragment_id ) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 5f73ffb815117..4d996d7849924 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -172,25 +172,55 @@ impl DdlService for DdlServiceImpl { &self, request: Request, ) -> Result, Status> { - let mut source = request.into_inner().get_source()?.clone(); + let req = request.into_inner(); + let mut 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 id = self.gen_unique_id::<{ IdCategory::Table }>().await?; - source.id = id; + let source_id = self.gen_unique_id::<{ IdCategory::Table }>().await?; + source.id = source_id; - let version = self - .ddl_controller - .run_command(DdlCommand::CreateSource(source)) - .await?; - Ok(Response::new(CreateSourceResponse { - status: None, - source_id: id, - version, - })) + match req.fragment_graph { + None => { + let version = self + .ddl_controller + .run_command(DdlCommand::CreateSource(source)) + .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; + } + }); + } + + // The id of stream job has been set above + let stream_job = StreamingJob::Source(source); + let version = self + .ddl_controller + .run_command(DdlCommand::CreateStreamingJob( + stream_job, + fragment_graph, + CreateType::Foreground, + )) + .await?; + Ok(Response::new(CreateSourceResponse { + status: None, + source_id, + version, + })) + } + } } async fn drop_source( @@ -422,6 +452,7 @@ impl DdlService for DdlServiceImpl { request: Request, ) -> 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(); @@ -443,7 +474,7 @@ impl DdlService for DdlServiceImpl { } } - let mut stream_job = StreamingJob::Table(source, mview); + let mut stream_job = StreamingJob::Table(source, mview, job_type); stream_job.set_id(table_id); @@ -550,7 +581,7 @@ impl DdlService for DdlServiceImpl { } let table_col_index_mapping = ColIndexMapping::from_protobuf(&req.table_col_index_mapping.unwrap()); - let stream_job = StreamingJob::Table(source, table); + let stream_job = StreamingJob::Table(source, table, TableJobType::General); let version = self .ddl_controller diff --git a/src/meta/src/manager/catalog/database.rs b/src/meta/src/manager/catalog/database.rs index 62b5692ce82ba..648f09951d6b2 100644 --- a/src/meta/src/manager/catalog/database.rs +++ b/src/meta/src/manager/catalog/database.rs @@ -345,6 +345,17 @@ impl DatabaseManager { .copied() .chain(self.sinks.keys().copied()) .chain(self.indexes.keys().copied()) + .chain(self.sources.keys().copied()) + .chain( + // filter cdc source jobs + self.sources + .iter() + .filter(|(_, source)| { + source.info.as_ref().is_some_and(|info| info.cdc_source_job) + }) + .map(|(id, _)| id) + .copied(), + ) } pub fn check_database_duplicated(&self, database_key: &DatabaseKey) -> MetaResult<()> { diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index c3445da8cc57a..051b3492cf68f 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -23,6 +23,7 @@ use risingwave_common::catalog::TableId; use risingwave_common::hash::{ActorMapping, ParallelUnitId, ParallelUnitMapping}; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_connector::source::SplitImpl; +use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; @@ -1170,10 +1171,11 @@ impl FragmentManager { .mview_actor_ids()) } - /// Get and filter the upstream `Materialize` fragments of the specified relations. - pub async fn get_upstream_mview_fragments( + /// Get and filter the upstream `Materialize` or `Source` fragments of the specified relations. + pub async fn get_upstream_root_fragments( &self, upstream_table_ids: &HashSet, + table_job_type: Option, ) -> MetaResult> { let map = &self.core.read().await.table_fragments; let mut fragments = HashMap::new(); @@ -1182,8 +1184,18 @@ impl FragmentManager { let table_fragments = map .get(&table_id) .with_context(|| format!("table_fragment not exist: id={}", table_id))?; - if let Some(fragment) = table_fragments.mview_fragment() { - fragments.insert(table_id, fragment); + match table_job_type.as_ref() { + Some(TableJobType::SharedCdcSource) => { + if let Some(fragment) = table_fragments.source_fragment() { + fragments.insert(table_id, fragment); + } + } + // MV on MV, and other kinds of table job + None | Some(TableJobType::General) | Some(TableJobType::Unspecified) => { + if let Some(fragment) = table_fragments.mview_fragment() { + fragments.insert(table_id, fragment); + } + } } } @@ -1211,6 +1223,7 @@ impl FragmentManager { r#type: d.r#type, dist_key_indices: d.dist_key_indices.clone(), output_indices: d.output_indices.clone(), + downstream_table_name: d.downstream_table_name.clone(), }; (fragment_id, strategy) }) diff --git a/src/meta/src/manager/catalog/mod.rs b/src/meta/src/manager/catalog/mod.rs index 116ae756bcb9d..48cb07073d8e5 100644 --- a/src/meta/src/manager/catalog/mod.rs +++ b/src/meta/src/manager/catalog/mod.rs @@ -642,7 +642,7 @@ impl CatalogManager { StreamingJob::Index(index, index_table) => { self.start_create_index_procedure(index, index_table).await } - StreamingJob::Table(source, table) => { + StreamingJob::Table(source, table, ..) => { if let Some(source) = source { self.start_create_table_procedure_with_source(source, table) .await @@ -651,6 +651,7 @@ impl CatalogManager { .await } } + StreamingJob::Source(source) => self.start_create_source_procedure(source).await, } } @@ -1017,6 +1018,7 @@ impl CatalogManager { let mut all_sink_ids: HashSet = HashSet::default(); let mut all_source_ids: HashSet = HashSet::default(); let mut all_view_ids: HashSet = HashSet::default(); + let mut all_cdc_source_ids: HashSet = HashSet::default(); let relations_depend_on = |relation_id: RelationId| -> Vec { let tables_depend_on = tables @@ -1270,6 +1272,12 @@ impl CatalogManager { if !all_source_ids.insert(source.id) { continue; } + + // add cdc source id + if let Some(info) = source.info && info.cdc_source_job { + all_cdc_source_ids.insert(source.id); + } + if let Some(ref_count) = database_core.relation_ref_count.get(&source.id).cloned() { @@ -1491,6 +1499,7 @@ impl CatalogManager { .into_iter() .map(|id| id.into()) .chain(all_sink_ids.into_iter().map(|id| id.into())) + .chain(all_cdc_source_ids.into_iter().map(|id| id.into())) .collect_vec(); Ok((version, catalog_deleted_ids)) @@ -1844,9 +1853,11 @@ impl CatalogManager { pub async fn finish_create_source_procedure( &self, mut source: Source, + internal_tables: Vec, ) -> MetaResult { let core = &mut *self.core.lock().await; let database_core = &mut core.database; + let mut tables = BTreeMapTransaction::new(&mut database_core.tables); let mut sources = BTreeMapTransaction::new(&mut database_core.sources); let key = (source.database_id, source.schema_id, source.name.clone()); assert!( @@ -1858,11 +1869,24 @@ impl CatalogManager { source.created_at_epoch = Some(Epoch::now().0); sources.insert(source.id, source.clone()); - - commit_meta!(self, sources)?; + for table in &internal_tables { + tables.insert(table.id, table.clone()); + } + commit_meta!(self, sources, tables)?; let version = self - .notify_frontend_relation_info(Operation::Add, RelationInfo::Source(source.to_owned())) + .notify_frontend( + Operation::Add, + Info::RelationGroup(RelationGroup { + relations: std::iter::once(Relation { + relation_info: RelationInfo::Source(source.to_owned()).into(), + }) + .chain(internal_tables.into_iter().map(|internal_table| Relation { + relation_info: RelationInfo::Table(internal_table).into(), + })) + .collect_vec(), + }), + ) .await; Ok(version) @@ -2217,7 +2241,7 @@ impl CatalogManager { /// This is used for `ALTER TABLE ADD/DROP COLUMN`. pub async fn start_replace_table_procedure(&self, stream_job: &StreamingJob) -> MetaResult<()> { - let StreamingJob::Table(source, table) = stream_job else { + let StreamingJob::Table(source, table, ..) = stream_job else { unreachable!("unexpected job: {stream_job:?}") }; let core = &mut *self.core.lock().await; @@ -2347,7 +2371,7 @@ impl CatalogManager { &self, stream_job: &StreamingJob, ) -> MetaResult<()> { - let StreamingJob::Table(source, table) = stream_job else { + let StreamingJob::Table(source, table, ..) = stream_job else { unreachable!("unexpected job: {stream_job:?}") }; let core = &mut *self.core.lock().await; diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index e02388eba4f3d..51ba6bb5fbbb9 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -16,7 +16,8 @@ use std::collections::HashMap; use risingwave_common::catalog::TableVersionId; use risingwave_common::util::epoch::Epoch; -use risingwave_pb::catalog::{CreateType, Index, Sink, Source, Table}; +use risingwave_pb::catalog::{CreateType, Index, PbSource, Sink, Table}; +use risingwave_pb::ddl_service::TableJobType; use crate::model::FragmentId; @@ -26,8 +27,9 @@ use crate::model::FragmentId; pub enum StreamingJob { MaterializedView(Table), Sink(Sink), - Table(Option, Table), + Table(Option, Table, TableJobType), Index(Index, Table), + Source(PbSource), } impl StreamingJob { @@ -36,7 +38,7 @@ impl StreamingJob { match self { StreamingJob::MaterializedView(table) => table.created_at_epoch = created_at_epoch, StreamingJob::Sink(table) => table.created_at_epoch = created_at_epoch, - StreamingJob::Table(source, table) => { + StreamingJob::Table(source, table, ..) => { table.created_at_epoch = created_at_epoch; if let Some(source) = source { source.created_at_epoch = created_at_epoch; @@ -45,6 +47,9 @@ impl StreamingJob { StreamingJob::Index(index, _) => { index.created_at_epoch = created_at_epoch; } + StreamingJob::Source(source) => { + source.created_at_epoch = created_at_epoch; + } } } @@ -55,7 +60,7 @@ impl StreamingJob { table.initialized_at_epoch = initialized_at_epoch } StreamingJob::Sink(table) => table.initialized_at_epoch = initialized_at_epoch, - StreamingJob::Table(source, table) => { + StreamingJob::Table(source, table, ..) => { table.initialized_at_epoch = initialized_at_epoch; if let Some(source) = source { source.initialized_at_epoch = initialized_at_epoch; @@ -64,6 +69,9 @@ impl StreamingJob { StreamingJob::Index(index, _) => { index.initialized_at_epoch = initialized_at_epoch; } + StreamingJob::Source(source) => { + source.initialized_at_epoch = initialized_at_epoch; + } } } } @@ -73,32 +81,38 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.id = id, Self::Sink(sink) => sink.id = id, - Self::Table(_, table) => table.id = id, + Self::Table(_, table, ..) => table.id = id, Self::Index(index, index_table) => { index.id = id; 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!() + } } } /// Set the fragment id where the table is materialized. pub fn set_table_fragment_id(&mut self, id: FragmentId) { match self { - Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table) => { + Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table, ..) => { table.fragment_id = id; } - Self::Sink(_) => {} + Self::Sink(_) | Self::Source(_) => {} } } /// Set the fragment id where the table dml is received. pub fn set_dml_fragment_id(&mut self, id: Option) { match self { - Self::Table(_, table) => { + Self::Table(_, table, ..) => { table.dml_fragment_id = id; } Self::MaterializedView(_) | Self::Index(_, _) | Self::Sink(_) => {} + Self::Source(_) => {} } } @@ -106,8 +120,9 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.id, Self::Sink(sink) => sink.id, - Self::Table(_, table) => table.id, + Self::Table(_, table, ..) => table.id, Self::Index(index, _) => index.id, + Self::Source(source) => source.id, } } @@ -115,18 +130,19 @@ impl StreamingJob { match self { Self::MaterializedView(table) => Some(table.id), Self::Sink(_sink) => None, - Self::Table(_, table) => Some(table.id), + Self::Table(_, table, ..) => Some(table.id), Self::Index(_, table) => Some(table.id), + Self::Source(_) => None, } } /// Returns the reference to the [`Table`] of the job if it exists. pub fn table(&self) -> Option<&Table> { match self { - Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table) => { + Self::MaterializedView(table) | Self::Index(_, table) | Self::Table(_, table, ..) => { Some(table) } - Self::Sink(_) => None, + Self::Sink(_) | Self::Source(_) => None, } } @@ -134,8 +150,9 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.schema_id, Self::Sink(sink) => sink.schema_id, - Self::Table(_, table) => table.schema_id, + Self::Table(_, table, ..) => table.schema_id, Self::Index(index, _) => index.schema_id, + Self::Source(source) => source.schema_id, } } @@ -143,8 +160,9 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.database_id, Self::Sink(sink) => sink.database_id, - Self::Table(_, table) => table.database_id, + Self::Table(_, table, ..) => table.database_id, Self::Index(index, _) => index.database_id, + Self::Source(source) => source.database_id, } } @@ -152,8 +170,9 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.name.clone(), Self::Sink(sink) => sink.name.clone(), - Self::Table(_, table) => table.name.clone(), + Self::Table(_, table, ..) => table.name.clone(), Self::Index(index, _) => index.name.clone(), + Self::Source(source) => source.name.clone(), } } @@ -161,17 +180,19 @@ impl StreamingJob { match self { StreamingJob::MaterializedView(mv) => mv.owner, StreamingJob::Sink(sink) => sink.owner, - StreamingJob::Table(_, table) => table.owner, + StreamingJob::Table(_, table, ..) => table.owner, StreamingJob::Index(index, _) => index.owner, + StreamingJob::Source(source) => source.owner, } } pub fn definition(&self) -> String { match self { Self::MaterializedView(table) => table.definition.clone(), - Self::Table(_, table) => table.definition.clone(), + Self::Table(_, table, ..) => table.definition.clone(), Self::Index(_, table) => table.definition.clone(), Self::Sink(sink) => sink.definition.clone(), + Self::Source(source) => source.definition.clone(), } } @@ -179,14 +200,15 @@ impl StreamingJob { match self { Self::MaterializedView(table) => table.properties.clone(), Self::Sink(sink) => sink.properties.clone(), - Self::Table(_, table) => table.properties.clone(), + Self::Table(_, table, ..) => table.properties.clone(), Self::Index(_, index_table) => index_table.properties.clone(), + Self::Source(source) => source.properties.clone(), } } /// Returns the [`TableVersionId`] if this job is `Table`. pub fn table_version_id(&self) -> Option { - if let Self::Table(_, table) = self { + if let Self::Table(_, table, ..) = self { Some( table .get_version() @@ -206,4 +228,16 @@ impl StreamingJob { _ => CreateType::Foreground, } } + + pub fn table_job_type(&self) -> Option { + if let Self::Table(.., sub_type) = self { + Some(*sub_type) + } else { + None + } + } + + 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 726bd7fcd8e73..3026880b1da09 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -304,6 +304,15 @@ impl TableFragments { .cloned() } + pub fn source_fragment(&self) -> Option { + self.fragments + .values() + .find(|fragment| { + (fragment.get_fragment_type_mask() & FragmentTypeFlag::Source as u32) != 0 + }) + .cloned() + } + /// Returns actors that contains Chain node. pub fn chain_actor_ids(&self) -> HashSet { Self::filter_actor_ids(self, |fragment_type_mask| { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index c08281a2f59ed..780f96552115a 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::cmp::Ordering; +use std::collections::HashMap; use std::num::NonZeroUsize; use std::sync::Arc; use std::time::Duration; @@ -318,7 +319,7 @@ impl DdlController { } self.catalog_manager - .finish_create_source_procedure(source) + .finish_create_source_procedure(source, vec![]) .await } @@ -456,7 +457,7 @@ impl DdlController { internal_tables = ctx.internal_tables(); match stream_job { - StreamingJob::Table(Some(ref source), _) => { + StreamingJob::Table(Some(ref source), ..) => { // Register the source on the connector node. self.source_manager.register_source(source).await?; } @@ -464,6 +465,10 @@ impl DdlController { // Validate the sink on the connector node. validate_sink(sink).await?; } + StreamingJob::Source(ref source) => { + // Register the source on the connector node. + self.source_manager.register_source(source).await?; + } _ => {} } (ctx, table_fragments) @@ -671,11 +676,16 @@ impl DdlController { // 1. Resolve the upstream fragments, extend the fragment graph to a complete graph that // contains all information needed for building the actor graph. - let upstream_mview_fragments = self + + let upstream_root_fragments = self .fragment_manager - .get_upstream_mview_fragments(fragment_graph.dependent_table_ids()) + .get_upstream_root_fragments( + fragment_graph.dependent_table_ids(), + stream_job.table_job_type(), + ) .await?; - let upstream_mview_actors = upstream_mview_fragments + + let upstream_actors: HashMap<_, _> = upstream_root_fragments .iter() .map(|(&table_id, fragment)| { ( @@ -685,8 +695,11 @@ impl DdlController { }) .collect(); - let complete_graph = - CompleteStreamFragmentGraph::with_upstreams(fragment_graph, upstream_mview_fragments)?; + let complete_graph = CompleteStreamFragmentGraph::with_upstreams( + fragment_graph, + upstream_root_fragments, + stream_job.table_job_type(), + )?; // 2. Build the actor graph. let cluster_info = self.cluster_manager.get_streaming_cluster_info().await; @@ -715,7 +728,7 @@ impl DdlController { let ctx = CreateStreamingJobContext { dispatchers, - upstream_mview_actors, + upstream_mview_actors: upstream_actors, internal_tables, building_locations, existing_locations, @@ -766,7 +779,7 @@ impl DdlController { .cancel_create_sink_procedure(sink) .await; } - StreamingJob::Table(source, table) => { + StreamingJob::Table(source, table, ..) => { if let Some(source) = source { self.catalog_manager .cancel_create_table_procedure_with_source(source, table) @@ -791,6 +804,11 @@ impl DdlController { .cancel_create_index_procedure(index, table) .await; } + StreamingJob::Source(source) => { + self.catalog_manager + .cancel_create_source_procedure(source) + .await?; + } } // 2. unmark creating tables. self.catalog_manager @@ -823,7 +841,7 @@ impl DdlController { .finish_create_sink_procedure(internal_tables, sink) .await? } - StreamingJob::Table(source, table) => { + StreamingJob::Table(source, table, ..) => { creating_internal_table_ids.push(table.id); if let Some(source) = source { self.catalog_manager @@ -841,6 +859,11 @@ impl DdlController { .finish_create_index_procedure(internal_tables, index, table) .await? } + StreamingJob::Source(source) => { + self.catalog_manager + .finish_create_source_procedure(source, internal_tables) + .await? + } }; // 2. unmark creating tables. @@ -1055,7 +1078,7 @@ impl DdlController { stream_job: &StreamingJob, table_col_index_mapping: ColIndexMapping, ) -> MetaResult { - let StreamingJob::Table(source, table) = stream_job else { + let StreamingJob::Table(source, table, ..) = stream_job else { unreachable!("unexpected job: {stream_job:?}") }; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index afe6186165e22..582c6585f5ba8 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1462,7 +1462,10 @@ impl GlobalStreamManager { match upstream_dispatch_type { DispatcherType::Unspecified => unreachable!(), - DispatcherType::Hash | DispatcherType::Broadcast | DispatcherType::Simple => { + DispatcherType::Hash + | DispatcherType::Broadcast + | DispatcherType::Simple + | DispatcherType::CdcTablename => { let upstream_fragment = &ctx.fragment_map.get(upstream_fragment_id).unwrap(); let mut upstream_actor_ids = upstream_fragment .actors @@ -1544,7 +1547,10 @@ impl GlobalStreamManager { fragment_actors_to_create.get(&downstream_fragment_id); match dispatcher.r#type() { - d @ (DispatcherType::Hash | DispatcherType::Simple | DispatcherType::Broadcast) => { + d @ (DispatcherType::Hash + | DispatcherType::Simple + | DispatcherType::Broadcast + | DispatcherType::CdcTablename) => { if let Some(downstream_actors_to_remove) = downstream_fragment_actors_to_remove { dispatcher diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 8851e2e880da6..7cd208496954d 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -26,7 +26,7 @@ use risingwave_pb::meta::table_fragments::Fragment; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use risingwave_pb::stream_plan::{ - DispatchStrategy, Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode, + ChainType, DispatchStrategy, Dispatcher, DispatcherType, MergeNode, StreamActor, StreamNode, }; use super::id::GlobalFragmentIdsExt; @@ -159,6 +159,8 @@ impl ActorBuilder { // "Leaf" node `Chain`. NodeBody::Chain(chain_node) => { + let cdc_backfill = chain_node.chain_type == ChainType::CdcBackfill as i32; + let input = stream_node.get_input(); assert_eq!(input.len(), 2); @@ -184,7 +186,11 @@ impl ActorBuilder { node_body: Some(NodeBody::Merge(MergeNode { upstream_actor_id, upstream_fragment_id: upstreams.fragment_id.as_global_id(), - upstream_dispatcher_type: DispatcherType::NoShuffle as _, + upstream_dispatcher_type: if cdc_backfill { + DispatcherType::CdcTablename as _ + } else { + DispatcherType::NoShuffle as _ + }, fields: merge_node.fields.clone(), })), ..merge_node.clone() @@ -352,6 +358,7 @@ impl ActorGraphBuildStateInner { hash_mapping: Some(downstream_actor_mapping.to_protobuf()), dispatcher_id: downstream_fragment_id.as_global_id() as u64, downstream_actor_id: downstream_actors.as_global_ids(), + downstream_table_name: strategy.downstream_table_name.clone(), } } @@ -371,6 +378,28 @@ impl ActorGraphBuildStateInner { hash_mapping: None, dispatcher_id: downstream_fragment_id.as_global_id() as u64, downstream_actor_id: downstream_actors.as_global_ids(), + downstream_table_name: None, + } + } + + /// Create a new dispatcher for cdc event dispatch. + fn new_cdc_dispatcher( + strategy: &DispatchStrategy, + downstream_fragment_id: GlobalFragmentId, + downstream_actors: &[GlobalActorId], + ) -> Dispatcher { + // dist key is the index to `_rw_table_name` column + assert_eq!(strategy.dist_key_indices.len(), 1); + assert!(strategy.downstream_table_name.is_some()); + + Dispatcher { + r#type: strategy.r#type, + dist_key_indices: strategy.dist_key_indices.clone(), + output_indices: strategy.output_indices.clone(), + hash_mapping: None, + dispatcher_id: downstream_fragment_id.as_global_id() as u64, + downstream_actor_id: downstream_actors.as_global_ids(), + downstream_table_name: strategy.downstream_table_name.clone(), } } @@ -471,7 +500,10 @@ impl ActorGraphBuildStateInner { } // Otherwise, make m * n links between the actors. - DispatcherType::Hash | DispatcherType::Broadcast | DispatcherType::Simple => { + DispatcherType::Hash + | DispatcherType::Broadcast + | DispatcherType::Simple + | DispatcherType::CdcTablename => { // Add dispatchers for the upstream actors. let dispatcher = if let DispatcherType::Hash = dt { // Transform the `ParallelUnitMapping` from the downstream distribution to the @@ -493,6 +525,12 @@ impl ActorGraphBuildStateInner { downstream.actor_ids, actor_mapping, ) + } else if let DispatcherType::CdcTablename = dt { + Self::new_cdc_dispatcher( + &edge.dispatch_strategy, + downstream.fragment_id, + downstream.actor_ids, + ) } else { Self::new_normal_dispatcher( &edge.dispatch_strategy, diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 5f022330f261c..19641a06e5d85 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -21,10 +21,14 @@ use anyhow::Context; use enum_as_inner::EnumAsInner; use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::catalog::{generate_internal_table_name_with_type, TableId}; +use risingwave_common::catalog::{ + generate_internal_table_name_with_type, TableId, CDC_SOURCE_COLUMN_NUM, TABLE_NAME_COLUMN_NAME, +}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::stream_graph_visitor; +use risingwave_common::util::stream_graph_visitor::visit_fragment; use risingwave_pb::catalog::Table; +use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::meta::table_fragments::Fragment; use risingwave_pb::stream_plan::stream_fragment_graph::{ Parallelism, StreamFragment, StreamFragmentEdge as StreamFragmentEdgeProto, @@ -146,6 +150,12 @@ impl BuildingFragment { dml_node.table_id = table_id; dml_node.table_version_id = job.table_version_id().unwrap(); } + NodeBody::Source(_) => { + // Notice: Table job has a dumb Source node, we should be careful that `has_table` should not be overwrite to `false` + if !has_table { + has_table = job.is_source_job(); + } + } _ => {} }); @@ -472,7 +482,9 @@ pub struct CompleteStreamFragmentGraph { } pub struct FragmentGraphUpstreamContext { - upstream_mview_fragments: HashMap, + /// Root fragment is the root of upstream stream graph, which can be a + /// mview fragment or source fragment for cdc source job + upstream_root_fragments: HashMap, } pub struct FragmentGraphDownstreamContext { @@ -493,18 +505,20 @@ impl CompleteStreamFragmentGraph { } } - /// Create a new [`CompleteStreamFragmentGraph`] for MV on MV, with the upstream existing - /// `Materialize` fragments. + /// Create a new [`CompleteStreamFragmentGraph`] for MV on MV or Table on CDC Source, with the upstream existing + /// `Materialize` or `Source` fragments. pub fn with_upstreams( graph: StreamFragmentGraph, - upstream_mview_fragments: HashMap, + upstream_root_fragments: HashMap, + table_job_type: Option, ) -> MetaResult { Self::build_helper( graph, Some(FragmentGraphUpstreamContext { - upstream_mview_fragments, + upstream_root_fragments, }), None, + table_job_type, ) } @@ -522,85 +536,147 @@ impl CompleteStreamFragmentGraph { original_table_fragment_id, downstream_fragments, }), + None, ) } fn build_helper( - graph: StreamFragmentGraph, + mut graph: StreamFragmentGraph, upstream_ctx: Option, downstream_ctx: Option, + table_job_type: Option, ) -> MetaResult { let mut extra_downstreams = HashMap::new(); let mut extra_upstreams = HashMap::new(); let mut existing_fragments = HashMap::new(); if let Some(FragmentGraphUpstreamContext { - upstream_mview_fragments, + upstream_root_fragments, }) = upstream_ctx { // Build the extra edges between the upstream `Materialize` and the downstream `Chain` // of the new materialized view. - for (&id, fragment) in &graph.fragments { + for (&id, fragment) in &mut graph.fragments { for (&upstream_table_id, output_columns) in &fragment.upstream_table_columns { - let mview_fragment = upstream_mview_fragments - .get(&upstream_table_id) - .context("upstream materialized view fragment not found")?; - let mview_id = GlobalFragmentId::new(mview_fragment.fragment_id); - - // Resolve the required output columns from the upstream materialized view. - let output_indices = { - let nodes = mview_fragment.actors[0].get_nodes().unwrap(); - let mview_node = nodes.get_node_body().unwrap().as_materialize().unwrap(); - let all_column_ids = mview_node - .get_table() - .unwrap() - .columns - .iter() - .map(|c| c.column_desc.as_ref().unwrap().column_id) - .collect_vec(); - - output_columns - .iter() - .map(|c| { - all_column_ids + let (up_fragment_id, edge) = match table_job_type.as_ref() { + Some(TableJobType::SharedCdcSource) => { + // extract the upstream full_table_name from the source fragment + let mut full_table_name = None; + visit_fragment(&mut fragment.inner, |node_body| { + if let NodeBody::Chain(chain_node) = node_body { + full_table_name = chain_node + .cdc_table_desc + .as_ref() + .map(|desc| desc.table_name.clone()); + } + }); + + let source_fragment = + upstream_root_fragments + .get(&upstream_table_id) + .context("upstream materialized view fragment not found")?; + let source_job_id = GlobalFragmentId::new(source_fragment.fragment_id); + // extract `_rw_table_name` column index + let rw_table_name_index = { + let node = source_fragment.actors[0].get_nodes().unwrap(); + + // may remove the expect to extend other scenarios, currently only target the CDC scenario + node.fields .iter() - .position(|&id| id == *c) - .map(|i| i as u32) - }) - .collect::>>() - .context("column not found in the upstream materialized view")? - }; + .position(|f| f.name.as_str() == TABLE_NAME_COLUMN_NAME) + .expect("table name column not found") + }; + + assert!(full_table_name.is_some()); + tracing::debug!( + ?full_table_name, + ?source_job_id, + ?rw_table_name_index, + ?output_columns, + "chain with upstream source fragment" + ); + let edge = StreamFragmentEdge { + id: EdgeId::UpstreamExternal { + upstream_table_id, + downstream_fragment_id: id, + }, + dispatch_strategy: DispatchStrategy { + r#type: DispatcherType::CdcTablename as _, /* there may have multiple downstream table jobs, so we use `Hash` here */ + dist_key_indices: vec![rw_table_name_index as _], /* index to `_rw_table_name` column */ + output_indices: (0..CDC_SOURCE_COLUMN_NUM as _).collect_vec(), /* require all columns from the cdc source */ + downstream_table_name: full_table_name, + }, + }; + + (source_job_id, edge) + } + _ => { + // handle other kinds of streaming graph, normally MV on MV + let mview_fragment = upstream_root_fragments + .get(&upstream_table_id) + .context("upstream materialized view fragment not found")?; + let mview_id = GlobalFragmentId::new(mview_fragment.fragment_id); + + // Resolve the required output columns from the upstream materialized view. + let output_indices = { + let nodes = mview_fragment.actors[0].get_nodes().unwrap(); + let mview_node = + nodes.get_node_body().unwrap().as_materialize().unwrap(); + let all_column_ids = mview_node + .get_table() + .unwrap() + .columns + .iter() + .map(|c| c.column_desc.as_ref().unwrap().column_id) + .collect_vec(); - let edge = StreamFragmentEdge { - id: EdgeId::UpstreamExternal { - upstream_table_id, - downstream_fragment_id: id, - }, - // We always use `NoShuffle` for the exchange between the upstream - // `Materialize` and the downstream `Chain` of the - // new materialized view. - dispatch_strategy: DispatchStrategy { - r#type: DispatcherType::NoShuffle as _, - dist_key_indices: vec![], // not used for `NoShuffle` - output_indices, - }, + output_columns + .iter() + .map(|c| { + all_column_ids + .iter() + .position(|&id| id == *c) + .map(|i| i as u32) + }) + .collect::>>() + .context("column not found in the upstream materialized view")? + }; + let edge = StreamFragmentEdge { + id: EdgeId::UpstreamExternal { + upstream_table_id, + downstream_fragment_id: id, + }, + // We always use `NoShuffle` for the exchange between the upstream + // `Materialize` and the downstream `Chain` of the + // new materialized view. + dispatch_strategy: DispatchStrategy { + r#type: DispatcherType::NoShuffle as _, + dist_key_indices: vec![], // not used for `NoShuffle` + output_indices, + downstream_table_name: None, + }, + }; + + (mview_id, edge) + } }; + // put the edge into the extra edges extra_downstreams - .entry(mview_id) + .entry(up_fragment_id) .or_insert_with(HashMap::new) .try_insert(id, edge.clone()) .unwrap(); extra_upstreams .entry(id) .or_insert_with(HashMap::new) - .try_insert(mview_id, edge) + .try_insert(up_fragment_id, edge) .unwrap(); } } existing_fragments.extend( - upstream_mview_fragments + upstream_root_fragments .into_values() .map(|f| (GlobalFragmentId::new(f.fragment_id), f)), ); diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index 4df57ea901331..bbde3b6ae9206 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -110,6 +110,8 @@ crepe::crepe! { // The downstream fragment of a `Simple` edge must be singleton. SingletonReq(y) <- Edge(_, y, Simple); + // The downstream fragment of a `CdcTablename` edge must be singleton. + SingletonReq(y) <- Edge(_, y, CdcTablename); // Multiple requirements conflict. Failed(x) <- Requirement(x, d1), Requirement(x, d2), (d1 != d2); @@ -326,6 +328,8 @@ impl Scheduler { }) .collect(); + tracing::debug!(?distributions, "schedule fragments"); + Ok(distributions) } } diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 68cb8125e67d0..03f787a7c38b7 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -24,6 +24,7 @@ use risingwave_pb::common::{ }; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; +use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::expr::agg_call::Type; use risingwave_pb::expr::expr_node::RexNode; use risingwave_pb::expr::expr_node::Type::{Add, GreaterThan}; @@ -227,6 +228,7 @@ fn make_stream_fragments() -> Vec { r#type: DispatcherType::Hash as i32, dist_key_indices: vec![0], output_indices: vec![0, 1, 2], + ..Default::default() }), })), fields: vec![ @@ -389,6 +391,7 @@ fn make_fragment_edges() -> Vec { r#type: DispatcherType::Simple as i32, dist_key_indices: vec![], output_indices: vec![], + ..Default::default() }), link_id: 4, upstream_id: 1, @@ -399,6 +402,7 @@ fn make_fragment_edges() -> Vec { r#type: DispatcherType::Hash as i32, dist_key_indices: vec![0], output_indices: vec![], + ..Default::default() }), link_id: 1, upstream_id: 2, @@ -452,7 +456,7 @@ fn make_cluster_info() -> StreamingClusterInfo { async fn test_graph_builder() -> MetaResult<()> { let env = MetaSrvEnv::for_test().await; let parallel_degree = 4; - let job = StreamingJob::Table(None, make_materialize_table(888)); + let job = StreamingJob::Table(None, make_materialize_table(888), TableJobType::General); let graph = make_stream_graph(); let fragment_graph = StreamFragmentGraph::new(graph, env.id_gen_manager_ref(), &job).await?; diff --git a/src/prost/build.rs b/src/prost/build.rs index 5722a04767962..e446518b7a6f1 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -59,7 +59,10 @@ fn main() -> Result<(), Box> { .collect(); // Paths to generate `BTreeMap` for protobuf maps. - let btree_map_paths = [".monitor_service.StackTraceResponse"]; + let btree_map_paths = [ + ".monitor_service.StackTraceResponse", + ".plan_common.ExternalTableDesc", + ]; // Build protobuf structs. @@ -108,7 +111,7 @@ fn main() -> Result<(), Box> { .type_attribute("plan_common.GeneratedColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.DefaultColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.Cardinality", "#[derive(Eq, Hash, Copy)]") - .type_attribute("plan_common.StorageTableDesc", "#[derive(Eq, Hash)]") + .type_attribute("plan_common.ExternalTableDesc", "#[derive(Eq, Hash)]") .type_attribute("plan_common.ColumnDesc", "#[derive(Eq, Hash)]") .type_attribute("common.ColumnOrder", "#[derive(Eq, Hash)]") .type_attribute("common.OrderType", "#[derive(Eq, Hash)]") diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index b58a13027261b..24b2d4c73e90a 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -362,6 +362,21 @@ impl MetaClient { pub async fn create_source(&self, source: PbSource) -> Result<(u32, CatalogVersion)> { let request = CreateSourceRequest { source: Some(source), + fragment_graph: None, + }; + + let resp = self.inner.create_source(request).await?; + Ok((resp.source_id, resp.version)) + } + + pub async fn create_source_with_graph( + &self, + source: PbSource, + graph: StreamFragmentGraph, + ) -> Result<(u32, CatalogVersion)> { + let request = CreateSourceRequest { + source: Some(source), + fragment_graph: Some(graph), }; let resp = self.inner.create_source(request).await?; @@ -398,11 +413,13 @@ impl MetaClient { source: Option, table: PbTable, graph: StreamFragmentGraph, + job_type: PbTableJobType, ) -> Result<(TableId, CatalogVersion)> { let request = CreateTableRequest { materialized_view: Some(table), fragment_graph: Some(graph), source, + job_type: job_type as _, }; let resp = self.inner.create_table(request).await?; // TODO: handle error in `resp.status` here diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 5d802bae99cdc..fade5cfddda7e 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1003,6 +1003,13 @@ impl fmt::Display for ExplainOptions { } } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct CdcTableInfo { + pub source_name: ObjectName, + pub external_table_name: String, +} + /// A top-level statement (SELECT, INSERT, CREATE, etc.) #[allow(clippy::large_enum_variant)] #[derive(Debug, Clone, PartialEq, Eq, Hash)] @@ -1084,6 +1091,8 @@ pub enum Statement { append_only: bool, /// `AS ( query )` query: Option>, + /// `FROM cdc_source TABLE database_name.table_name` + cdc_table_info: Option, }, /// CREATE INDEX CreateIndex { @@ -1515,6 +1524,7 @@ impl fmt::Display for Statement { source_watermarks, append_only, query, + cdc_table_info, } => { // We want to allow the following options // Empty column list, allowed by PostgreSQL: @@ -1549,6 +1559,10 @@ impl fmt::Display for Statement { if let Some(query) = query { write!(f, " AS {}", query)?; } + if let Some(info) = cdc_table_info { + write!(f, " FROM {}", info.source_name)?; + write!(f, " TABLE {}", info.external_table_name)?; + } Ok(()) } Statement::CreateIndex { diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 3ff012c81b766..3af27d5f87fc0 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -445,12 +445,17 @@ impl Parser { pub fn parse_source_schema_with_connector( &mut self, connector: &str, + cdc_source_job: bool, ) -> Result { // row format for cdc source must be debezium json // row format for nexmark source must be native // default row format for datagen source is native if connector.contains("-cdc") { - let expected = ConnectorSchema::debezium_json(); + let expected = if cdc_source_job { + ConnectorSchema::plain_json() + } else { + ConnectorSchema::debezium_json() + }; if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_source_schema_v2().0; if schema != expected { @@ -508,6 +513,14 @@ impl Parser { } impl ConnectorSchema { + pub const fn plain_json() -> Self { + ConnectorSchema { + format: Format::Plain, + row_encode: Encode::Json, + row_options: Vec::new(), + } + } + /// Create a new source schema with `Debezium` format and `Json` encoding. pub const fn debezium_json() -> Self { ConnectorSchema { @@ -719,10 +732,12 @@ impl ParseTo for CreateSourceStatement { .iter() .find(|&opt| opt.name.real_value() == UPSTREAM_SOURCE_KEY); let connector: String = option.map(|opt| opt.value.to_string()).unwrap_or_default(); - // row format for cdc source must be debezium json + // The format of cdc source job is fixed to `FORMAT PLAIN ENCODE JSON` + let cdc_source_job = + connector.contains("-cdc") && columns.is_empty() && constraints.is_empty(); // row format for nexmark source must be native // default row format for datagen source is native - let source_schema = p.parse_source_schema_with_connector(&connector)?; + let source_schema = p.parse_source_schema_with_connector(&connector, cdc_source_job)?; Ok(Self { if_not_exists, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 87af26bfb1750..190c94a72a199 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -2446,7 +2446,7 @@ impl Parser { let connector = option.map(|opt| opt.value.to_string()); let source_schema = if let Some(connector) = connector { - Some(self.parse_source_schema_with_connector(&connector)?) + Some(self.parse_source_schema_with_connector(&connector, false)?) } else { None // Table is NOT created with an external connector. }; @@ -2463,6 +2463,23 @@ impl Parser { None }; + let cdc_table_info = if self.parse_keyword(Keyword::FROM) { + let source_name = self.parse_object_name()?; + if self.parse_keyword(Keyword::TABLE) { + let external_table_name = self.parse_literal_string()?; + Some(CdcTableInfo { + source_name, + external_table_name, + }) + } else { + return Err(ParserError::ParserError( + "Expect a TABLE clause on table created by CREATE TABLE FROM".to_string(), + )); + } + } else { + None + }; + Ok(Statement::CreateTable { name: table_name, temporary, @@ -2475,6 +2492,7 @@ impl Parser { source_watermarks, append_only, query, + cdc_table_info, }) } diff --git a/src/stream/src/executor/backfill/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs similarity index 60% rename from src/stream/src/executor/backfill/cdc_backfill.rs rename to src/stream/src/executor/backfill/cdc/cdc_backfill.rs index e3121e1bd0a7b..333a1ad106340 100644 --- a/src/stream/src/executor/backfill/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::default::Default; use std::pin::{pin, Pin}; use std::sync::Arc; @@ -22,17 +21,20 @@ use futures::stream::select_with_strategy; use futures::{pin_mut, stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use maplit::hashmap; -use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; -use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{JsonbVal, ScalarRefImpl}; -use risingwave_common::util::epoch::EpochPair; -use risingwave_connector::source::external::{CdcOffset, DebeziumOffset, DebeziumSourceOffset}; -use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; +use risingwave_common::array::{DataChunk, StreamChunk}; +use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; +use risingwave_common::row::{OwnedRow, Row, RowExt}; +use risingwave_common::types::{DataType, ScalarRefImpl}; +use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_connector::parser::{ + DebeziumParser, EncodingProperties, JsonProperties, ProtocolProperties, + SourceStreamChunkBuilder, SpecificParserConfig, +}; +use risingwave_connector::source::external::CdcOffset; +use risingwave_connector::source::{SourceColumnDesc, SourceContext, SplitMetaData}; use risingwave_storage::StateStore; -use serde_json::Value; +use crate::executor::backfill::cdc::state::{CdcBackfillStateImpl, SingleTableState}; use crate::executor::backfill::upstream_table::external::ExternalStorageTable; use crate::executor::backfill::upstream_table::snapshot::{ SnapshotReadArgs, UpstreamTableRead, UpstreamTableReader, @@ -48,15 +50,13 @@ use crate::executor::{ }; use crate::task::{ActorId, CreateMviewProgress}; -pub const BACKFILL_STATE_KEY_SUFFIX: &str = "_backfill"; - pub struct CdcBackfillExecutor { actor_ctx: ActorContextRef, /// Upstream external table upstream_table: ExternalStorageTable, - /// Upstream changelog with the same schema with the external table. + /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset` upstream: BoxedExecutor, /// The column indices need to be forwarded to the downstream from the upstream and table scan. @@ -67,9 +67,13 @@ pub struct CdcBackfillExecutor { info: ExecutorInfo, - /// Stores the backfill done flag + /// State table of the Source executor source_state_handler: SourceStateTableHandler, + shared_cdc_source: bool, + + progress: Option, + metrics: Arc, chunk_size: usize, @@ -82,11 +86,12 @@ impl CdcBackfillExecutor { upstream_table: ExternalStorageTable, upstream: BoxedExecutor, output_indices: Vec, - _progress: Option, + progress: Option, schema: Schema, pk_indices: PkIndices, metrics: Arc, source_state_handler: SourceStateTableHandler, + shared_cdc_source: bool, chunk_size: usize, ) -> Self { Self { @@ -103,6 +108,8 @@ impl CdcBackfillExecutor { chunk_size, actor_ctx, source_state_handler, + shared_cdc_source, + progress, } } @@ -112,10 +119,11 @@ impl CdcBackfillExecutor { let pk_in_output_indices = self.upstream_table.pk_in_output_indices().unwrap(); let pk_order = self.upstream_table.pk_order_types().to_vec(); + let shared_cdc_source = self.shared_cdc_source; let upstream_table_id = self.upstream_table.table_id().table_id; let upstream_table_reader = UpstreamTableReader::new(self.upstream_table); - let mut upstream = self.upstream.execute().peekable(); + let mut upstream = self.upstream.execute(); // Current position of the upstream_table storage primary key. // `None` means it starts from the beginning. @@ -129,54 +137,56 @@ impl CdcBackfillExecutor { // Check whether this parallelism has been assigned splits, // if not, we should bypass the backfill directly. - let mut invalid_backfill = false; - let mut split_id: Option = None; - let mut cdc_split: Option = None; - if let Some(mutation) = first_barrier.mutation.as_ref() { - match mutation.as_ref() { - Mutation::Add { splits, .. } - | Mutation::Update { - actor_splits: splits, - .. - } => { - invalid_backfill = match splits.get(&self.actor_ctx.id) { - None => true, - Some(splits) => { - let is_empty = splits.is_empty(); - if !is_empty { - let split = splits.iter().exactly_one().map_err(|err| { - StreamExecutorError::from(anyhow!( - "fail to get cdc split {}", - err - )) - })?; - split_id = Some(split.id()); - cdc_split = Some(split.clone()); + let mut state_impl = if shared_cdc_source { + CdcBackfillStateImpl::Undefined + } else if let Some(mutation) = first_barrier.mutation.as_ref() && + let Mutation::Add{splits, ..} = mutation.as_ref() + { + tracing::info!(?mutation, ?shared_cdc_source, "got first barrier"); + // We can assume for cdc table, the parallism of the fragment must be 1 + match splits.get(&self.actor_ctx.id) { + None => { + unreachable!("expect to receive the cdc split, please check the parallelism of the fragment") + }, + Some(splits) => { + if splits.is_empty() { + tracing::info!(?splits, "got empty cdc split, bypass the backfill"); + // The first barrier message should be propagated. + yield Message::Barrier(first_barrier); + #[for_await] + for msg in upstream { + if let Some(msg) = mapping_message(msg?, &self.output_indices) { + yield msg; } - is_empty } + // exit the executor + return Ok(()); } - } - _ => {} - } - } - if invalid_backfill { - // The first barrier message should be propagated. - yield Message::Barrier(first_barrier); - #[for_await] - for msg in upstream { - if let Some(msg) = mapping_message(msg?, &self.output_indices) { - yield msg; + let split = splits.iter().exactly_one().map_err(|_err| { + StreamExecutorError::from(anyhow!( + "expect only one cdc split for table {}", + upstream_table_id + )) + })?; + CdcBackfillStateImpl::SingleTable(SingleTableState::new(self.source_state_handler, upstream_table_id, split.id(), split.clone())) } } - // exit the executor - return Ok(()); } + else { + unreachable!("backfilled cdc source init fail") + }; - tracing::debug!("start cdc backfill: actor {:?}", self.actor_ctx.id); + let mut upstream = if shared_cdc_source { + transform_upstream(upstream, &self.info.schema) + .boxed() + .peekable() + } else { + upstream.peekable() + }; - self.source_state_handler.init_epoch(first_barrier.epoch); + tracing::debug!(?upstream_table_id, ?self.actor_ctx.id, ?shared_cdc_source, "start cdc backfill"); + state_impl.init_epoch(first_barrier.epoch); // start from the beginning // TODO(siyuan): restore backfill offset from state store @@ -184,22 +194,8 @@ impl CdcBackfillExecutor { current_pk_pos = backfill_offset; - let is_finished = { - // restore backfill done flag from state store - if let Some(split_id) = split_id.as_ref() { - let mut key = split_id.to_string(); - key.push_str(BACKFILL_STATE_KEY_SUFFIX); - match self.source_state_handler.get(key.into()).await? { - Some(row) => match row.datum_at(1) { - Some(ScalarRefImpl::Jsonb(jsonb_ref)) => jsonb_ref.as_bool()?, - _ => unreachable!("invalid backfill persistent state"), - }, - None => false, - } - } else { - false - } - }; + // restore backfill done flag from state store + let is_finished = state_impl.check_finished().await?; // If the snapshot is empty, we don't need to backfill. let is_snapshot_empty: bool = { @@ -226,6 +222,7 @@ impl CdcBackfillExecutor { // Keep track of rows from the snapshot. #[allow(unused_variables)] let mut total_snapshot_processed_rows: u64 = 0; + let mut snapshot_read_epoch; // Read the current binlog offset as a low watermark let mut last_binlog_offset: Option = @@ -338,11 +335,15 @@ impl CdcBackfillExecutor { } // seal current epoch even though there is no data - Self::persist_state( - &mut self.source_state_handler, - barrier.epoch, - ) - .await?; + state_impl.commit_state(barrier.epoch).await?; + snapshot_read_epoch = barrier.epoch.prev; + if let Some(progress) = self.progress.as_mut() { + progress.update( + barrier.epoch.curr, + snapshot_read_epoch, + total_snapshot_processed_rows, + ); + } yield Message::Barrier(barrier); // Break the for loop and start a new snapshot read stream. @@ -413,14 +414,7 @@ impl CdcBackfillExecutor { )); } - Self::write_backfill_state( - &mut self.source_state_handler, - upstream_table_id, - &split_id, - &mut cdc_split, - last_binlog_offset.clone(), - ) - .await?; + state_impl.mutate_state(last_binlog_offset.clone()).await?; break 'backfill_loop; } Some(chunk) => { @@ -453,14 +447,8 @@ impl CdcBackfillExecutor { initial_binlog_offset = ?last_binlog_offset, "upstream snapshot is empty, mark backfill is done and persist current binlog offset"); - Self::write_backfill_state( - &mut self.source_state_handler, - upstream_table_id, - &split_id, - &mut cdc_split, - last_binlog_offset, - ) - .await?; + // The snapshot is empty, just set backfill to finished + state_impl.mutate_state(last_binlog_offset).await?; } tracing::info!( @@ -468,6 +456,27 @@ impl CdcBackfillExecutor { "CdcBackfill has already finished and forward messages directly to the downstream" ); + // Wait for first barrier to come after backfill is finished. + // So we can update our progress + persist the status. + while let Some(Ok(msg)) = upstream.next().await { + if let Some(msg) = mapping_message(msg, &self.output_indices) { + // If not finished then we need to update state, otherwise no need. + if let Message::Barrier(barrier) = &msg { + // persist the backfill state + state_impl.commit_state(barrier.epoch).await?; + + // mark progress as finished + if let Some(progress) = self.progress.as_mut() { + progress.finish(barrier.epoch.curr, total_snapshot_processed_rows); + } + yield msg; + // break after the state have been saved + break; + } + yield msg; + } + } + // After backfill progress finished // we can forward messages directly to the downstream, // as backfill is finished. @@ -476,87 +485,120 @@ impl CdcBackfillExecutor { // upstream offsets will be removed from the message before forwarding to // downstream if let Some(msg) = mapping_message(msg?, &self.output_indices) { - // persist the backfill state if any if let Message::Barrier(barrier) = &msg { - Self::persist_state(&mut self.source_state_handler, barrier.epoch).await?; + // commit state just to bump the epoch of state table + state_impl.commit_state(barrier.epoch).await?; } yield msg; } } } +} - /// When snapshot read stream ends, we should persist two states: - /// 1) a backfill finish flag to denote the backfill has done - /// 2) a consumed binlog offset to denote the last binlog offset - /// which will be committed to the state store upon next barrier. - async fn write_backfill_state( - source_state_handler: &mut SourceStateTableHandler, - upstream_table_id: u32, - split_id: &Option, - cdc_split: &mut Option, - last_binlog_offset: Option, - ) -> StreamExecutorResult<()> { - assert!( - last_binlog_offset.is_some(), - "last binlog offset cannot be None" - ); - - if let Some(split_id) = split_id.as_ref() { - let mut key = split_id.to_string(); - key.push_str(BACKFILL_STATE_KEY_SUFFIX); - source_state_handler - .set(key.into(), JsonbVal::from(Value::Bool(true))) - .await?; - - if let Some(SplitImpl::MysqlCdc(split)) = cdc_split.as_mut() - && let Some(s) = split.mysql_split.as_mut() - { - let start_offset = last_binlog_offset.as_ref().map(|cdc_offset| { - let source_offset = if let CdcOffset::MySql(o) = cdc_offset { - DebeziumSourceOffset { - file: Some(o.filename.clone()), - pos: Some(o.position), - ..Default::default() - } - } else { - DebeziumSourceOffset::default() - }; - - let mut server = "RW_CDC_".to_string(); - server.push_str(upstream_table_id.to_string().as_str()); - DebeziumOffset { - source_partition: hashmap! { - "server".to_string() => server - }, - source_offset, - // upstream heartbeat event would not emit to the cdc backfill executor, - // since we don't parse heartbeat event in the source parser. - is_heartbeat: false, - } - }); - - // persist the last binlog offset into split state - s.inner.start_offset = start_offset.map(|o| { - let value = serde_json::to_value(o).unwrap(); - value.to_string() - }); - s.inner.snapshot_done = true; - } - if let Some(split_impl) = cdc_split { - source_state_handler - .set(split_impl.id(), split_impl.encode_to_json()) - .await? - } +#[try_stream(ok = Message, error = StreamExecutorError)] +pub async fn transform_upstream(upstream: BoxedMessageStream, schema: &Schema) { + let props = SpecificParserConfig { + key_encoding_config: None, + encoding_config: EncodingProperties::Json(JsonProperties { + use_schema_registry: false, + }), + protocol_config: ProtocolProperties::Debezium, + }; + let mut parser = DebeziumParser::new( + props, + get_rw_columns(schema), + Arc::new(SourceContext::default()), + ) + .await + .map_err(StreamExecutorError::connector_error)?; + + pin_mut!(upstream); + #[for_await] + for msg in upstream { + let mut msg = msg?; + if let Message::Chunk(chunk) = &mut msg { + let parsed_chunk = parse_debezium_chunk(&mut parser, chunk, schema).await?; + let _ = std::mem::replace(chunk, parsed_chunk); } - Ok(()) + yield msg; + } +} + +async fn parse_debezium_chunk( + parser: &mut DebeziumParser, + chunk: &StreamChunk, + schema: &Schema, +) -> StreamExecutorResult { + // here we transform the input chunk in (payload varchar, _rw_offset varchar, _rw_table_name varchar) schema + // to chunk with downstream table schema `info.schema` of MergeNode contains the schema of the + // table job with `_rw_offset` in the end + // see `gen_create_table_plan_for_cdc_source` for details + let column_descs = get_rw_columns(schema); + let mut builder = SourceStreamChunkBuilder::with_capacity(column_descs, chunk.capacity()); + + // The schema of input chunk (payload varchar, _rw_offset varchar, _rw_table_name varchar, _row_id) + // We should use the debezium parser to parse the first column, + // then chain the parsed row with `_rw_offset` row to get a new row. + let payloads = chunk.data_chunk().project(vec![0].as_slice()); + let offset_columns = chunk.data_chunk().project(vec![1].as_slice()); + + // TODO: preserve the transaction semantics + for payload in payloads.rows() { + let ScalarRefImpl::Jsonb(jsonb_ref) = payload.datum_at(0).expect("payload must exist") + else { + unreachable!("payload must be jsonb"); + }; + + parser + .parse_inner( + None, + Some(jsonb_ref.to_string().as_bytes().to_vec()), + builder.row_writer(), + ) + .await + .unwrap(); } - async fn persist_state( - source_state_handler: &mut SourceStateTableHandler, - new_epoch: EpochPair, - ) -> StreamExecutorResult<()> { - source_state_handler.state_store.commit(new_epoch).await + let parsed_chunk = builder.finish(); + let (data_chunk, ops) = parsed_chunk.into_parts(); + + // concat the rows in the parsed chunk with the _rw_offset column, we should also retain the Op column + let mut new_rows = Vec::with_capacity(chunk.capacity()); + let offset_columns = offset_columns.compact(); + for (data_row, offset_row) in data_chunk + .rows_with_holes() + .zip_eq_fast(offset_columns.rows_with_holes()) + { + let combined = data_row.chain(offset_row); + new_rows.push(combined); } + + let data_types = schema + .fields + .iter() + .map(|field| field.data_type.clone()) + .chain(std::iter::once(DataType::Varchar)) // _rw_offset column + .collect_vec(); + + Ok(StreamChunk::from_parts( + ops, + DataChunk::from_rows(new_rows.as_slice(), data_types.as_slice()), + )) +} + +fn get_rw_columns(schema: &Schema) -> Vec { + schema + .fields + .iter() + .map(|field| { + let column_desc = ColumnDesc::named( + field.name.clone(), + ColumnId::placeholder(), + field.data_type.clone(), + ); + SourceColumnDesc::from(&column_desc) + }) + .collect_vec() } impl Executor for CdcBackfillExecutor { @@ -576,3 +618,70 @@ impl Executor for CdcBackfillExecutor { &self.info.identity } } + +#[cfg(test)] +mod tests { + use std::str::FromStr; + + use futures::{pin_mut, StreamExt}; + use risingwave_common::array::{DataChunk, Op, StreamChunk}; + use risingwave_common::catalog::{Field, Schema}; + use risingwave_common::types::{DataType, Datum, JsonbVal}; + use risingwave_common::util::iter_util::ZipEqFast; + + use crate::executor::backfill::cdc::cdc_backfill::transform_upstream; + use crate::executor::test_utils::MockSource; + use crate::executor::Executor; + + #[tokio::test] + async fn test_transform_upstream_chunk() { + let schema = Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // debezium json payload + Field::unnamed(DataType::Varchar), // _rw_offset + Field::unnamed(DataType::Varchar), // _rw_table_name + ]); + let pk_indices = vec![1]; + let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + // let payload = r#"{"before": null,"after":{"O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" },"source":{"version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null},"op":"r","ts_ms":1695277757017,"transaction":null}"#.to_string(); + let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#; + + let datums: Vec = vec![ + Some(JsonbVal::from_str(payload).unwrap().into()), + Some("file: 1.binlog, pos: 100".to_string().into()), + Some("mydb.orders".to_string().into()), + ]; + + println!("datums: {:?}", datums[1]); + + let mut builders = schema.create_array_builders(8); + for (builder, datum) in builders.iter_mut().zip_eq_fast(datums.iter()) { + builder.append(datum.clone()); + } + let columns = builders + .into_iter() + .map(|builder| builder.finish().into()) + .collect(); + + // one row chunk + let chunk = StreamChunk::from_parts(vec![Op::Insert], DataChunk::new(columns, 1)); + + tx.push_chunk(chunk); + let upstream = Box::new(source).execute(); + + // schema of the CDC table + let rw_schema = Schema::new(vec![ + Field::with_name(DataType::Int64, "O_ORDERKEY"), // orderkey + Field::with_name(DataType::Int64, "O_CUSTKEY"), // custkey + Field::with_name(DataType::Varchar, "O_ORDERSTATUS"), // orderstatus + Field::with_name(DataType::Decimal, "O_TOTALPRICE"), // totalprice + Field::with_name(DataType::Date, "O_ORDERDATE"), // orderdate + ]); + + let parsed_stream = transform_upstream(upstream, &rw_schema); + pin_mut!(parsed_stream); + // the output chunk must contain the offset column + if let Some(message) = parsed_stream.next().await { + println!("chunk: {:#?}", message.unwrap()); + } + } +} diff --git a/src/stream/src/executor/backfill/cdc/mod.rs b/src/stream/src/executor/backfill/cdc/mod.rs new file mode 100644 index 0000000000000..5061211f28c6a --- /dev/null +++ b/src/stream/src/executor/backfill/cdc/mod.rs @@ -0,0 +1,18 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod cdc_backfill; +mod state; + +pub use state::BACKFILL_STATE_KEY_SUFFIX; diff --git a/src/stream/src/executor/backfill/cdc/state.rs b/src/stream/src/executor/backfill/cdc/state.rs new file mode 100644 index 0000000000000..2d32bbd1d4353 --- /dev/null +++ b/src/stream/src/executor/backfill/cdc/state.rs @@ -0,0 +1,175 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use maplit::hashmap; +use risingwave_common::row::Row; +use risingwave_common::types::{JsonbVal, ScalarRefImpl}; +use risingwave_common::util::epoch::EpochPair; +use risingwave_connector::source::external::{CdcOffset, DebeziumOffset, DebeziumSourceOffset}; +use risingwave_connector::source::{SplitId, SplitImpl, SplitMetaData}; +use risingwave_storage::StateStore; +use serde_json::Value; + +use crate::executor::{SourceStateTableHandler, StreamExecutorResult}; + +pub enum CdcBackfillStateImpl { + Undefined, + SingleTable(SingleTableState), +} + +impl CdcBackfillStateImpl { + pub fn init_epoch(&mut self, epoch: EpochPair) { + match self { + CdcBackfillStateImpl::Undefined => {} + CdcBackfillStateImpl::SingleTable(state) => state.init_epoch(epoch), + } + } + + pub async fn check_finished(&self) -> StreamExecutorResult { + match self { + CdcBackfillStateImpl::Undefined => Ok(false), + CdcBackfillStateImpl::SingleTable(state) => state.check_finished().await, + } + } + + pub async fn mutate_state( + &mut self, + last_binlog_offset: Option, + ) -> StreamExecutorResult<()> { + match self { + CdcBackfillStateImpl::Undefined => Ok(()), + CdcBackfillStateImpl::SingleTable(state) => { + state.mutate_state(last_binlog_offset).await + } + } + } + + pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { + match self { + CdcBackfillStateImpl::Undefined => Ok(()), + CdcBackfillStateImpl::SingleTable(state) => state.commit_state(new_epoch).await, + } + } +} + +pub const BACKFILL_STATE_KEY_SUFFIX: &str = "_backfill"; + +/// The state manager for single cdc table +pub struct SingleTableState { + /// Stores the backfill done flag + source_state_handler: SourceStateTableHandler, + cdc_table_id: u32, + split_id: SplitId, + cdc_split: SplitImpl, +} + +impl SingleTableState {} + +impl SingleTableState { + pub fn new( + source_state_handler: SourceStateTableHandler, + cdc_table_id: u32, + split_id: SplitId, + cdc_split: SplitImpl, + ) -> Self { + Self { + source_state_handler, + cdc_table_id, + split_id, + cdc_split, + } + } + + pub fn init_epoch(&mut self, epoch: EpochPair) { + self.source_state_handler.init_epoch(epoch) + } + + pub async fn check_finished(&self) -> StreamExecutorResult { + let mut key = self.split_id.to_string(); + key.push_str(BACKFILL_STATE_KEY_SUFFIX); + match self.source_state_handler.get(key.into()).await? { + Some(row) => match row.datum_at(1) { + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => Ok(jsonb_ref.as_bool()?), + _ => unreachable!("invalid backfill persistent state"), + }, + None => Ok(false), + } + } + + /// When snapshot read stream ends, we should persist two states: + /// 1) a backfill finish flag to denote the backfill has done + /// 2) a consumed binlog offset to denote the last binlog offset + /// which will be committed to the state store upon next barrier. + pub async fn mutate_state( + &mut self, + last_binlog_offset: Option, + ) -> StreamExecutorResult<()> { + let mut key = self.split_id.to_string(); + key.push_str(BACKFILL_STATE_KEY_SUFFIX); + // write backfill finished flag + self.source_state_handler + .set(key.into(), JsonbVal::from(Value::Bool(true))) + .await?; + + if let SplitImpl::MysqlCdc(split) = &mut self.cdc_split + && let Some(state) = split.mysql_split.as_mut() { + let start_offset = + last_binlog_offset.as_ref().map(|cdc_offset| { + let source_offset = + if let CdcOffset::MySql(o) = cdc_offset + { + DebeziumSourceOffset { + file: Some(o.filename.clone()), + pos: Some(o.position), + ..Default::default() + } + } else { + DebeziumSourceOffset::default() + }; + + let mut server = "RW_CDC_".to_string(); + server.push_str( + self.cdc_table_id.to_string().as_str(), + ); + DebeziumOffset { + source_partition: hashmap! { + "server".to_string() => server + }, + source_offset, + // upstream heartbeat event would not emit to the cdc backfill executor, + // since we don't parse heartbeat event in the source parser. + is_heartbeat: false, + } + }); + + // persist the last binlog offset into split state + state.inner.start_offset = start_offset.map(|o| { + let value = serde_json::to_value(o).unwrap(); + value.to_string() + }); + state.inner.snapshot_done = true; + } + // write the last binlog offset that will be used upon recovery + self.source_state_handler + .set(self.split_id.clone(), self.cdc_split.encode_to_json()) + .await + } + + pub async fn commit_state(&mut self, new_epoch: EpochPair) -> StreamExecutorResult<()> { + self.source_state_handler + .state_store + .commit(new_epoch) + .await + } +} diff --git a/src/stream/src/executor/backfill/mod.rs b/src/stream/src/executor/backfill/mod.rs index 0f8e3373fd247..0c782596813ff 100644 --- a/src/stream/src/executor/backfill/mod.rs +++ b/src/stream/src/executor/backfill/mod.rs @@ -13,7 +13,7 @@ // limitations under the License. pub mod arrangement_backfill; -pub mod cdc_backfill; +pub mod cdc; pub mod no_shuffle_backfill; pub mod upstream_table; pub mod utils; diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index 414721c34efbf..b6de1ac0daa4f 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -25,6 +25,8 @@ use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; use risingwave_common::hash::{ActorMapping, ExpandedActorMapping, VirtualNode}; +use risingwave_common::row::Row; +use risingwave_common::types::ScalarRefImpl; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::stream_plan::update_mutation::PbDispatcherUpdate; use risingwave_pb::stream_plan::PbDispatcher; @@ -345,6 +347,7 @@ pub enum DispatcherImpl { Broadcast(BroadcastDispatcher), Simple(SimpleDispatcher), RoundRobin(RoundRobinDataDispatcher), + CdcTableName(CdcTableNameDispatcher), } impl DispatcherImpl { @@ -386,6 +389,29 @@ impl DispatcherImpl { dispatcher.dispatcher_id, )) } + CdcTablename => { + assert!(!outputs.is_empty()); + assert!(dispatcher.downstream_table_name.is_some()); + let dist_key_indices: Vec = dispatcher + .dist_key_indices + .iter() + .map(|i| *i as usize) + .collect_vec(); + + assert_eq!( + dist_key_indices.len(), + 1, + "expect only one table name column index" + ); + DispatcherImpl::CdcTableName(CdcTableNameDispatcher::new( + outputs, + dist_key_indices[0], + output_indices, + dispatcher.dispatcher_id, + dispatcher.downstream_table_name.clone(), + )) + } + Broadcast => DispatcherImpl::Broadcast(BroadcastDispatcher::new( outputs, output_indices, @@ -466,7 +492,8 @@ macro_rules! for_all_dispatcher_variants { { Hash }, { Broadcast }, { Simple }, - { RoundRobin } + { RoundRobin }, + { CdcTableName } } }; } @@ -818,6 +845,127 @@ impl Dispatcher for BroadcastDispatcher { } } +/// Dispatch stream chunk based on table name from upstream DB +#[derive(Debug)] +pub struct CdcTableNameDispatcher { + outputs: Vec, + // column index to the `_rw_table_name` column + table_name_col_index: usize, + output_indices: Vec, + dispatcher_id: DispatcherId, + dispatcher_id_str: String, + downstream_table_name: Option, +} + +impl CdcTableNameDispatcher { + pub fn new( + outputs: Vec, + table_name_col_index: usize, + output_indices: Vec, + dispatcher_id: DispatcherId, + downstream_table_name: Option, + ) -> Self { + Self { + outputs, + table_name_col_index, + output_indices, + dispatcher_id, + dispatcher_id_str: dispatcher_id.to_string(), + downstream_table_name, + } + } +} + +impl Dispatcher for CdcTableNameDispatcher { + async fn dispatch_data(&mut self, chunk: StreamChunk) -> StreamResult<()> { + let num_outputs = self.outputs.len(); + + let mut vis_maps = repeat_with(|| BitmapBuilder::with_capacity(chunk.capacity())) + .take(num_outputs) + .collect_vec(); + + let chunk = chunk.project(&self.output_indices); + + // TODO: use a more efficient way to filter data, e.g. add a Filter node before Chain + for (visible, row) in chunk + .visibility() + .iter() + .zip_eq_fast(chunk.data_chunk().rows_with_holes()) + { + // Build visibility map for every output chunk. + for vis_map in &mut vis_maps { + let should_emit = if let Some(row) = row && let Some(full_table_name) = self.downstream_table_name.as_ref() { + let table_name_datum = row.datum_at(self.table_name_col_index).unwrap(); + tracing::trace!(target: "events::stream::dispatch::hash::cdc", "keys: {:?}, table: {}", self.table_name_col_index, full_table_name); + // dispatch based on downstream table name + table_name_datum == ScalarRefImpl::Utf8(full_table_name) + } else { + true + }; + vis_map.append(visible && should_emit); + } + } + + for (vis_map, output) in vis_maps.into_iter().zip_eq_fast(self.outputs.iter_mut()) { + let vis_map = vis_map.finish(); + let new_stream_chunk = + StreamChunk::with_visibility(chunk.ops(), chunk.columns().into(), vis_map); + if new_stream_chunk.cardinality() > 0 { + event!( + tracing::Level::TRACE, + msg = "chunk", + downstream = output.actor_id(), + "send = \n{:#?}", + new_stream_chunk + ); + output.send(Message::Chunk(new_stream_chunk)).await?; + } + } + + Ok(()) + } + + async fn dispatch_barrier(&mut self, barrier: Barrier) -> StreamResult<()> { + // always broadcast barrier + for output in &mut self.outputs { + output.send(Message::Barrier(barrier.clone())).await?; + } + Ok(()) + } + + async fn dispatch_watermark(&mut self, watermark: Watermark) -> StreamResult<()> { + if let Some(watermark) = watermark.transform_with_indices(&self.output_indices) { + // always broadcast watermark + for output in &mut self.outputs { + output.send(Message::Watermark(watermark.clone())).await?; + } + } + Ok(()) + } + + fn add_outputs(&mut self, outputs: impl IntoIterator) { + self.outputs.extend(outputs); + } + + fn remove_outputs(&mut self, actor_ids: &HashSet) { + self.outputs + .extract_if(|output| actor_ids.contains(&output.actor_id())) + .count(); + } + + fn dispatcher_id(&self) -> DispatcherId { + self.dispatcher_id + } + + fn dispatcher_id_str(&self) -> &str { + &self.dispatcher_id_str + } + + fn is_empty(&self) -> bool { + self.outputs.is_empty() + } +} + /// `SimpleDispatcher` dispatches message to a single output. #[derive(Debug)] pub struct SimpleDispatcher { diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index c28d6ec8564d9..b01bfb42a3c09 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -100,10 +100,11 @@ mod wrapper; #[cfg(test)] mod integration_tests; pub mod test_utils; +mod utils; pub use actor::{Actor, ActorContext, ActorContextRef}; use anyhow::Context; -pub use backfill::cdc_backfill::*; +pub use backfill::cdc::cdc_backfill::CdcBackfillExecutor; pub use backfill::no_shuffle_backfill::*; pub use backfill::upstream_table::*; pub use barrier_recv::BarrierRecvExecutor; @@ -141,6 +142,7 @@ pub use top_n::{ AppendOnlyGroupTopNExecutor, AppendOnlyTopNExecutor, GroupTopNExecutor, TopNExecutor, }; pub use union::UnionExecutor; +pub use utils::DummyExecutor; pub use values::ValuesExecutor; pub use watermark_filter::WatermarkFilterExecutor; pub use wrapper::WrapperExecutor; diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 1705350426968..fdacf1100493e 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -37,8 +37,9 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use crate::common::table::state_table::StateTable; +use crate::executor::backfill::cdc::BACKFILL_STATE_KEY_SUFFIX; use crate::executor::error::StreamExecutorError; -use crate::executor::{StreamExecutorResult, BACKFILL_STATE_KEY_SUFFIX}; +use crate::executor::StreamExecutorResult; const COMPLETE_SPLIT_PREFIX: &str = "SsGLdzRDqBuKzMf9bDap"; diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index 2ae461e62351c..ec3b5f861e2d0 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -70,6 +70,10 @@ impl MessageSender { self.0.send(Message::Barrier(barrier)).unwrap(); } + pub fn send_barrier(&self, barrier: Barrier) { + self.0.send(Message::Barrier(barrier)).unwrap(); + } + #[allow(dead_code)] pub fn push_barrier_with_prev_epoch_for_test( &mut self, diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs new file mode 100644 index 0000000000000..fe5f6b044a115 --- /dev/null +++ b/src/stream/src/executor/utils.rs @@ -0,0 +1,53 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use futures::StreamExt; +use risingwave_common::catalog::Schema; + +use crate::executor::{BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef}; + +#[derive(Default)] +pub struct DummyExecutor { + pub info: ExecutorInfo, +} + +impl DummyExecutor { + pub fn new() -> Self { + Self { + info: ExecutorInfo { + schema: Schema::empty().clone(), + pk_indices: vec![], + identity: "DummyExecutor".to_string(), + }, + } + } +} + +impl Executor for DummyExecutor { + fn execute(self: Box) -> BoxedMessageStream { + futures::stream::pending().boxed() + } + + fn schema(&self) -> &Schema { + &self.info.schema + } + + fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.info.pk_indices + } + + fn identity(&self) -> &str { + &self.info.identity + } +} diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 46e58cadac66a..8b9054b9a776e 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::anyhow; use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId, TableOption}; use risingwave_common::util::sort_util::OrderType; @@ -22,7 +23,7 @@ use risingwave_storage::table::Distribution; use risingwave_storage::StateStore; use super::*; -use crate::executor::BatchQueryExecutor; +use crate::executor::{BatchQueryExecutor, DummyExecutor}; pub struct BatchQueryExecutorBuilder; @@ -36,7 +37,14 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { state_store: impl StateStore, stream: &mut LocalStreamManagerCore, ) -> StreamResult { - let table_desc: &StorageTableDesc = node.get_table_desc()?; + if node.table_desc.is_none() { + // used in sharing cdc source backfill as a dummy batch plan node + return Ok(Box::new(DummyExecutor::new())); + } + + let table_desc: &StorageTableDesc = node + .get_table_desc() + .map_err(|err| anyhow!("batch_plan: table_desc not found! {:?}", err))?; let table_id = TableId { table_id: table_desc.table_id, }; diff --git a/src/stream/src/from_proto/chain.rs b/src/stream/src/from_proto/chain.rs index ba48d2c5a25f1..e119007199068 100644 --- a/src/stream/src/from_proto/chain.rs +++ b/src/stream/src/from_proto/chain.rs @@ -12,19 +12,24 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::sync::Arc; +use anyhow::anyhow; use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema, TableId, TableOption}; use risingwave_common::util::sort_util::OrderType; -use risingwave_pb::plan_common::StorageTableDesc; +use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; +use risingwave_pb::plan_common::{ExternalTableDesc, StorageTableDesc}; use risingwave_pb::stream_plan::{ChainNode, ChainType}; use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::Distribution; use super::*; use crate::common::table::state_table::StateTable; +use crate::executor::external::ExternalStorageTable; use crate::executor::{ - BackfillExecutor, ChainExecutor, FlowControlExecutor, RearrangedChainExecutor, + BackfillExecutor, CdcBackfillExecutor, ChainExecutor, FlowControlExecutor, + RearrangedChainExecutor, SourceStateTableHandler, }; pub struct ChainExecutorBuilder; @@ -39,7 +44,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { state_store: impl StateStore, stream: &mut LocalStreamManagerCore, ) -> StreamResult { - let [mview, snapshot]: [_; 2] = params.input.try_into().unwrap(); + let [upstream, snapshot]: [_; 2] = params.input.try_into().unwrap(); // For reporting the progress. let progress = stream .context @@ -58,6 +63,11 @@ impl ExecutorBuilder for ChainExecutorBuilder { .map(|i| snapshot.schema().fields()[*i].clone()) .collect_vec(), ) + } else if matches!(node.chain_type(), ChainType::CdcBackfill) { + let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; + let schema = Schema::new(table_desc.columns.iter().map(Into::into).collect()); + assert_eq!(output_indices, (0..schema.len()).collect_vec()); + schema } else { // For `Chain`s other than `Backfill`, there should be no extra mapping required. We can // directly output the columns received from the upstream or snapshot. @@ -71,7 +81,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { let upstream_only = matches!(node.chain_type(), ChainType::UpstreamOnly); ChainExecutor::new( snapshot, - mview, + upstream, progress, schema, params.pk_indices, @@ -79,12 +89,71 @@ impl ExecutorBuilder for ChainExecutorBuilder { ) .boxed() } - ChainType::Rearrange => { - RearrangedChainExecutor::new(snapshot, mview, progress, schema, params.pk_indices) - .boxed() + ChainType::Rearrange => RearrangedChainExecutor::new( + snapshot, + upstream, + progress, + schema, + params.pk_indices, + ) + .boxed(), + ChainType::CdcBackfill => { + let table_desc: &ExternalTableDesc = node.get_cdc_table_desc()?; + let properties: HashMap = table_desc + .connect_properties + .iter() + .map(|(k, v)| (k.clone(), v.clone())) + .collect(); + let table_type = CdcTableType::from_properties(&properties); + let table_reader = + table_type.create_table_reader(properties.clone(), schema.clone())?; + + let order_types = table_desc + .pk + .iter() + .map(|desc| OrderType::from_protobuf(desc.get_order_type().unwrap())) + .collect_vec(); + + let pk_indices = table_desc + .pk + .iter() + .map(|k| k.column_index as usize) + .collect_vec(); + + let schema_table_name = SchemaTableName::from_properties(&properties); + let external_table = ExternalStorageTable::new( + TableId::new(table_desc.table_id), + schema_table_name, + table_reader, + schema.clone(), + order_types, + pk_indices.clone(), + (0..table_desc.columns.len()).collect_vec(), + ); + + let source_state_handler = SourceStateTableHandler::from_table_catalog( + node.get_state_table().as_ref().unwrap(), + state_store.clone(), + ) + .await; + CdcBackfillExecutor::new( + params.actor_context.clone(), + external_table, + upstream, + (0..table_desc.columns.len()).collect_vec(), /* eliminate the last column (_rw_offset) */ + Some(progress), + schema.clone(), + pk_indices, + params.executor_stats, + source_state_handler, + true, + params.env.config().developer.chunk_size, + ).boxed() } ChainType::Backfill => { - let table_desc: &StorageTableDesc = node.get_table_desc()?; + let table_desc: &StorageTableDesc = node + .get_table_desc() + .map_err(|err| anyhow!("chain: table_desc not found! {:?}", err))?; let table_id = TableId { table_id: table_desc.table_id, }; @@ -164,7 +233,7 @@ impl ExecutorBuilder for ChainExecutorBuilder { BackfillExecutor::new( upstream_table, - mview, + upstream, state_table, output_indices, progress, diff --git a/src/stream/src/from_proto/merge.rs b/src/stream/src/from_proto/merge.rs index 8fd401c2ab4a9..450d55d9b8641 100644 --- a/src/stream/src/from_proto/merge.rs +++ b/src/stream/src/from_proto/merge.rs @@ -59,7 +59,7 @@ impl ExecutorBuilder for MergeExecutorBuilder { // There could be arbitrary number of upstreams with simple dispatcher. DispatcherType::Simple => false, // There should be always only one upstream with no-shuffle dispatcher. - DispatcherType::NoShuffle => true, + DispatcherType::NoShuffle | DispatcherType::CdcTablename => true, }; if always_single_input { diff --git a/src/stream/src/from_proto/row_id_gen.rs b/src/stream/src/from_proto/row_id_gen.rs index 92f13d4544e9f..dca80cfa89449 100644 --- a/src/stream/src/from_proto/row_id_gen.rs +++ b/src/stream/src/from_proto/row_id_gen.rs @@ -34,6 +34,7 @@ impl ExecutorBuilder for RowIdGenExecutorBuilder { _stream: &mut LocalStreamManagerCore, ) -> StreamResult { let [upstream]: [_; 1] = params.input.try_into().unwrap(); + tracing::debug!("row id gen executor: {:?}", params.vnode_bitmap); let vnodes = params .vnode_bitmap .expect("vnodes not set for row id gen executor"); diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index b105837d9b093..ec0c2ffcb5d13 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -15,7 +15,7 @@ use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; -use risingwave_connector::source::external::{ExternalTableType, SchemaTableName}; +use risingwave_connector::source::external::{CdcTableType, SchemaTableName}; use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::SourceNode; use risingwave_source::source_desc::SourceDescBuilder; @@ -159,12 +159,12 @@ impl ExecutorBuilder for SourceExecutorBuilder { params.env.connector_params(), ); - let table_type = ExternalTableType::from_properties(&source.properties); + let table_type = CdcTableType::from_properties(&source.properties); if table_type.can_backfill() - && let Some(table_desc) = source_info.upstream_table.clone() + && let Some(table_desc) = source_info.external_table.clone() { - let upstream_table_name = - SchemaTableName::from_properties(&source.properties); + let table_schema = Schema::new(table_desc.columns.iter().map(Into::into).collect()); + let upstream_table_name = SchemaTableName::from_properties(&source.properties); let table_pk_indices = table_desc .pk .iter() @@ -177,33 +177,34 @@ impl ExecutorBuilder for SourceExecutorBuilder { .collect_vec(); let table_reader = table_type - .create_table_reader(source.properties.clone(), schema.clone())?; + .create_table_reader(source.properties.clone(), table_schema.clone())?; let external_table = ExternalStorageTable::new( TableId::new(source.source_id), upstream_table_name, table_reader, - schema.clone(), + table_schema.clone(), table_pk_order_types, table_pk_indices, - (0..table_desc.columns.len()).collect_vec(), + (0..table_schema.len()).collect_vec(), ); // use the state table from source to store the backfill state (may refactor in future) let source_state_handler = SourceStateTableHandler::from_table_catalog( source.state_table.as_ref().unwrap(), store.clone(), - ) - .await; + ).await; + // use schema from table_desc let cdc_backfill = CdcBackfillExecutor::new( params.actor_context.clone(), external_table, Box::new(source_exec), - (0..source.columns.len()).collect_vec(), /* eliminate the last column (_rw_offset) */ + (0..table_schema.len()).collect_vec(), None, - schema.clone(), + table_schema, params.pk_indices, params.executor_stats, source_state_handler, + false, source_ctrl_opts.chunk_size, ); cdc_backfill.boxed() diff --git a/src/tests/sqlsmith/src/lib.rs b/src/tests/sqlsmith/src/lib.rs index ebb0682a7aaaf..d3df482bddf48 100644 --- a/src/tests/sqlsmith/src/lib.rs +++ b/src/tests/sqlsmith/src/lib.rs @@ -277,6 +277,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, CreateTable { or_replace: false, @@ -320,6 +321,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, CreateTable { or_replace: false, @@ -374,6 +376,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, ], )"#]], @@ -504,6 +507,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, CreateTable { or_replace: false, @@ -554,6 +558,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, CreateTable { or_replace: false, @@ -611,6 +616,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, CreateTable { or_replace: false, @@ -686,6 +692,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY source_watermarks: [], append_only: false, query: None, + cdc_table_info: None, }, ], )"#]],