From 7399c7ca10c8019db748a96458d521bf333873df Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 26 Jan 2024 12:21:45 +0800 Subject: [PATCH 01/23] WIP: mongodb --- .../connector/api/source/SourceTypeE.java | 3 +++ .../connector/source/core/DbzCdcEventConsumer.java | 5 +++-- .../src/main/resources/mongodb.properties | 13 +++++++++++++ java/pom.xml | 5 +++++ proto/connector_service.proto | 1 + 5 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties diff --git a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java index 88b8351f80fb4..0c9858ab4fd5d 100644 --- a/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java +++ b/java/connector-node/connector-api/src/main/java/com/risingwave/connector/api/source/SourceTypeE.java @@ -20,6 +20,7 @@ public enum SourceTypeE { MYSQL, POSTGRES, CITUS, + MONGODB, INVALID; public static SourceTypeE valueOf(ConnectorServiceProto.SourceType type) { @@ -30,6 +31,8 @@ public static SourceTypeE valueOf(ConnectorServiceProto.SourceType type) { return SourceTypeE.POSTGRES; case CITUS: return SourceTypeE.CITUS; + case MONGODB: + return SourceTypeE.MONGODB; default: return SourceTypeE.INVALID; } 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 f0880d52c8b57..c4cb6ab72a7ba 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 @@ -152,8 +152,9 @@ var record = event.value(); case DATA: { // Topic naming conventions - // - PG: serverName.schemaName.tableName - // - MySQL: serverName.databaseName.tableName + // - PG: topicPrefix.schemaName.tableName + // - MySQL: topicPrefix.databaseName.tableName + // - Mongo: topicPrefix.databaseName.collectionName // We can extract the full table name from the topic var fullTableName = record.topic().substring(record.topic().indexOf('.') + 1); diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties new file mode 100644 index 0000000000000..cd044f45544a3 --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties @@ -0,0 +1,13 @@ +# configs for postgres conneoctor +connector.class=io.debezium.connector.mongodb.MongoDbConnector +# default snapshot mode to initial +snapshot.mode=${debezium.snapshot.mode:-initial} +mongodb.connection.string=${mongodb.url} +collection.include.list=${collection.name} +# default heartbeat interval 5 mins +heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} +# In sharing cdc source 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. +# TODO: set this field in the code +name=${collection.name:-RW_CDC_Sharing} +provide.transaction.metadata=${transactional:-false} diff --git a/java/pom.xml b/java/pom.xml index 5f168c48bd9ef..8f2979c132f62 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -161,6 +161,11 @@ debezium-connector-mysql ${debezium.version} + + io.debezium + debezium-connector-mongodb + ${debezium.version} + org.postgresql postgresql diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 49fca31d1330d..d450bcb6e9a89 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -169,6 +169,7 @@ enum SourceType { MYSQL = 1; POSTGRES = 2; CITUS = 3; + MONGODB = 4; } message SourceCommonParam { From 6a5bbb94983286897cd18ad04cb3a583747b926e Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 27 Jan 2024 22:02:43 +0800 Subject: [PATCH 02/23] mongo source unit test --- .../debezium-mongo/docker-compose.yaml | 2 +- .../mongodb-cdc/config-replica.js | 6 ++ integration_tests/mongodb-cdc/create_mv.sql | 7 ++ .../mongodb-cdc/create_source.sql | 6 ++ integration_tests/mongodb-cdc/data_check | 1 + .../mongodb-cdc/docker-compose.yaml | 30 ++++++ .../source/common/DbzConnectorConfig.java | 13 +++ .../risingwave-source-test/pom.xml | 20 +--- .../connector/source/MongoDbSourceTest.java | 92 +++++++++++++++++++ .../connector/source/SourceTestClient.java | 20 ++++ 10 files changed, 181 insertions(+), 16 deletions(-) create mode 100644 integration_tests/mongodb-cdc/config-replica.js create mode 100644 integration_tests/mongodb-cdc/create_mv.sql create mode 100644 integration_tests/mongodb-cdc/create_source.sql create mode 100644 integration_tests/mongodb-cdc/data_check create mode 100644 integration_tests/mongodb-cdc/docker-compose.yaml create mode 100644 java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java diff --git a/integration_tests/debezium-mongo/docker-compose.yaml b/integration_tests/debezium-mongo/docker-compose.yaml index afa753440f77d..4056aae12c7d3 100644 --- a/integration_tests/debezium-mongo/docker-compose.yaml +++ b/integration_tests/debezium-mongo/docker-compose.yaml @@ -31,7 +31,7 @@ services: image: mongo:4.4 container_name: mongodb ports: - - "27017:27017" + - "7017:27017" command: --replSet rs0 --oplogSize 128 restart: always healthcheck: diff --git a/integration_tests/mongodb-cdc/config-replica.js b/integration_tests/mongodb-cdc/config-replica.js new file mode 100644 index 0000000000000..c174820a6933b --- /dev/null +++ b/integration_tests/mongodb-cdc/config-replica.js @@ -0,0 +1,6 @@ +rsconf = { + _id: "rs0", + members: [{ _id: 0, host: "mongodb:27017", priority: 1.0 }], +}; +rs.initiate(rsconf); +rs.status(); diff --git a/integration_tests/mongodb-cdc/create_mv.sql b/integration_tests/mongodb-cdc/create_mv.sql new file mode 100644 index 0000000000000..17ce354009c7b --- /dev/null +++ b/integration_tests/mongodb-cdc/create_mv.sql @@ -0,0 +1,7 @@ +CREATE MATERIALIZED VIEW normalized_users AS +SELECT + payload ->> 'name' as name, + payload ->> 'email' as email, + payload ->> 'address' as address +FROM + users; \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/create_source.sql b/integration_tests/mongodb-cdc/create_source.sql new file mode 100644 index 0000000000000..9c95e87644d0c --- /dev/null +++ b/integration_tests/mongodb-cdc/create_source.sql @@ -0,0 +1,6 @@ +CREATE TABLE users (_id JSONB PRIMARY KEY, payload JSONB) WITH ( + connector = 'kafka', + kafka.topic = 'dbserver1.random_data.users', + kafka.brokers = 'message_queue:29092', + kafka.scan.startup.mode = 'earliest' +) FORMAT DEBEZIUM_MONGO ENCODE JSON; \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/data_check b/integration_tests/mongodb-cdc/data_check new file mode 100644 index 0000000000000..c57752e1fd9b2 --- /dev/null +++ b/integration_tests/mongodb-cdc/data_check @@ -0,0 +1 @@ +users,normalized_users \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/docker-compose.yaml b/integration_tests/mongodb-cdc/docker-compose.yaml new file mode 100644 index 0000000000000..75904fb558f73 --- /dev/null +++ b/integration_tests/mongodb-cdc/docker-compose.yaml @@ -0,0 +1,30 @@ +version: '3.1' + +services: + mongodb: + image: mongodb/mongodb-community-server:4.4.23-ubi8 + container_name: mongodb + ports: + - "7017:27017" + command: --replSet rs0 --oplogSize 128 + restart: always + healthcheck: + test: "echo 'db.runCommand({ping: 1})' | mongo" + interval: 5s + timeout: 10s + retries: 3 + + mongodb-setup: + image: mongodb/mongodb-community-server:4.4.23-ubi8 + container_name: mongodb-setup + depends_on: + - mongodb + entrypoint: + [ + "bash", + "-c", + "sleep 10 && mongo --host mongodb:27017 /config-replica.js && sleep 10" + ] + restart: "no" + volumes: + - ./config-replica.js:/config-replica.js 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 b796cae4ccb4a..de6587c8acc97 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 @@ -61,6 +61,7 @@ public class DbzConnectorConfig { 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"; + private static final String MONGODB_CONFIG_FILE = "mongodb.properties"; private static final String DBZ_PROPERTY_PREFIX = "debezium."; @@ -217,6 +218,18 @@ public DbzConnectorConfig( ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); } dbzProps.putAll(postgresProps); + } else if (source == SourceTypeE.MONGODB) { + var mongodbProps = initiateDbConfig(MONGODB_CONFIG_FILE, substitutor); + + // if snapshot phase is finished and offset is specified, we will continue reading + // changes from the given offset + if (snapshotDone && null != startOffset && !startOffset.isBlank()) { + mongodbProps.setProperty("snapshot.mode", "never"); + mongodbProps.setProperty( + ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); + } + dbzProps.putAll(mongodbProps); + } else { throw new RuntimeException("unsupported source type: " + source); } diff --git a/java/connector-node/risingwave-source-test/pom.xml b/java/connector-node/risingwave-source-test/pom.xml index 46956b642f15e..68a3807d95f84 100644 --- a/java/connector-node/risingwave-source-test/pom.xml +++ b/java/connector-node/risingwave-source-test/pom.xml @@ -19,21 +19,6 @@ connector-api test - - com.risingwave - s3-common - test - - - org.slf4j - slf4j-log4j12 - - - org.slf4j - slf4j-reload4j - - - io.grpc grpc-protobuf @@ -75,6 +60,11 @@ org.testcontainers postgresql + + org.mongodb + mongodb-driver-sync + 4.11.1 + com.fasterxml.jackson.core jackson-databind diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java new file mode 100644 index 0000000000000..d454dbdfcc84d --- /dev/null +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java @@ -0,0 +1,92 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.risingwave.connector.source; + +import com.risingwave.connector.ConnectorServiceImpl; +import com.risingwave.proto.ConnectorServiceProto; +import io.grpc.*; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.*; +import javax.sql.DataSource; +import org.junit.*; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MongoDbSourceTest { + private static final Logger LOG = LoggerFactory.getLogger(MongoDbSourceTest.class.getName()); + + // private static final PostgreSQLContainer pg = + // new PostgreSQLContainer<>("postgres:15-alpine") + // .withDatabaseName("test") + // .withUsername("postgres") + // .withCommand("postgres -c wal_level=logical -c max_wal_senders=10"); + + public static Server connectorServer = + ServerBuilder.forPort(SourceTestClient.DEFAULT_PORT) + .addService(new ConnectorServiceImpl()) + .build(); + + public static SourceTestClient testClient = + new SourceTestClient( + Grpc.newChannelBuilder( + "localhost:" + SourceTestClient.DEFAULT_PORT, + InsecureChannelCredentials.create()) + .build()); + + private static DataSource pgDataSource; + + @BeforeClass + public static void init() { + try { + connectorServer.start(); + } catch (Exception e) { + e.printStackTrace(); + } + } + + @AfterClass + public static void cleanup() { + connectorServer.shutdown(); + } + + // create a TPC-H orders table in postgres + // insert 10,000 rows into orders + // check if the number of changes debezium captures is 10,000 + @Test + public void testMongoDbSource() throws Exception { + ExecutorService executorService = Executors.newFixedThreadPool(1); + Map props = new HashMap<>(); + props.put("mongodb.url", "mongodb://localhost:7017/?replicaSet=rs0"); + props.put("collection.name", "dev.test"); + Iterator eventStream = + testClient.getEventStream(ConnectorServiceProto.SourceType.MONGODB, 3001, props); + int count = 0; + while (eventStream.hasNext()) { + List messages = eventStream.next().getEventsList(); + for (ConnectorServiceProto.CdcMessage msg : messages) { + if (!msg.getPayload().isBlank()) { + count++; + } + } + if (count >= 10000) { + break; + } + } + LOG.info("number of cdc messages received: {}", count); + } +} diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java index 55792b84eb809..58c72c688dfef 100644 --- a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/SourceTestClient.java @@ -29,6 +29,7 @@ import java.sql.SQLException; import java.sql.Statement; import java.util.Iterator; +import java.util.Map; import java.util.Properties; import java.util.Random; import java.util.UUID; @@ -152,6 +153,25 @@ protected Iterator getEventStreamS return responses; } + public Iterator getEventStream( + ConnectorServiceProto.SourceType sourceType, + long sourceId, + Map properties) { + ConnectorServiceProto.GetEventStreamRequest req = + ConnectorServiceProto.GetEventStreamRequest.newBuilder() + .setSourceId(sourceId) + .setSourceType(sourceType) + .putAllProperties(properties) + .build(); + Iterator responses = null; + try { + responses = blockingStub.getEventStream(req); + } catch (StatusRuntimeException e) { + fail("RPC failed: {}", e.getStatus()); + } + return responses; + } + // generates an orders.tbl in class path using random data // if file does not contain 10000 lines static void genOrdersTable(int numRows) { From fd4aa015ae44354cc01dc17c19eaf293f64bbadf Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 1 Feb 2024 19:26:21 +0800 Subject: [PATCH 03/23] Add mongodb split and refactor cdc split --- .../mongodb-cdc/config-replica.js | 2 +- .../mongodb-cdc/docker-compose.yaml | 2 +- .../source/SourceValidateHandler.java | 4 + .../source/common/DbzConnectorConfig.java | 5 + .../source/common/MongoDbValidator.java | 32 + .../source/core/DbzCdcEventConsumer.java | 25 +- .../src/main/resources/mongodb.properties | 2 + .../risingwave-source-test/pom.xml | 13 + .../connector/source/MongoDbSourceTest.java | 83 +- .../src/test/resources/log4j2.properties | 2 +- java/pom.xml | 8 +- proto/connector_service.proto | 1 + src/compute/tests/cdc_tests.rs | 2 +- src/connector/src/macros.rs | 3 +- src/connector/src/source/base.rs | 5 +- .../src/source/cdc/enumerator/mod.rs | 59 +- src/connector/src/source/cdc/mod.rs | 3 + src/connector/src/source/cdc/source/reader.rs | 9 +- src/connector/src/source/cdc/split.rs | 231 +- src/connector/src/split_expand.rs | 1913 +++++++++++++++++ 20 files changed, 2267 insertions(+), 137 deletions(-) create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java create mode 100644 src/connector/src/split_expand.rs diff --git a/integration_tests/mongodb-cdc/config-replica.js b/integration_tests/mongodb-cdc/config-replica.js index c174820a6933b..a4c29a401791c 100644 --- a/integration_tests/mongodb-cdc/config-replica.js +++ b/integration_tests/mongodb-cdc/config-replica.js @@ -1,6 +1,6 @@ rsconf = { _id: "rs0", - members: [{ _id: 0, host: "mongodb:27017", priority: 1.0 }], + members: [{ _id: 0, host: "localhost:27017", priority: 1.0 }], }; rs.initiate(rsconf); rs.status(); diff --git a/integration_tests/mongodb-cdc/docker-compose.yaml b/integration_tests/mongodb-cdc/docker-compose.yaml index 75904fb558f73..ea5258daa29c3 100644 --- a/integration_tests/mongodb-cdc/docker-compose.yaml +++ b/integration_tests/mongodb-cdc/docker-compose.yaml @@ -5,7 +5,7 @@ services: image: mongodb/mongodb-community-server:4.4.23-ubi8 container_name: mongodb ports: - - "7017:27017" + - "27017:27017" command: --replSet rs0 --oplogSize 128 restart: always healthcheck: 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 3ef406f97754f..ec1cdd5feadb1 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 @@ -133,6 +133,10 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re validator.validateAll(isMultiTableShared); } break; + case MONGODB: + ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_URL); + ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_COLLECTION_NAME); + break; default: LOG.warn("Unknown source type"); throw ValidatorUtils.invalidArgument("Unknown source type"); 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 de6587c8acc97..7faaa471fec28 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 @@ -68,6 +68,11 @@ public class DbzConnectorConfig { private static final String SNAPSHOT_MODE_KEY = "debezium.snapshot.mode"; private static final String SNAPSHOT_MODE_BACKFILL = "rw_cdc_backfill"; + public static class MongoDb { + public static final String MONGO_URL = "mongodb.url"; + public static final String MONGO_COLLECTION_NAME = "collection.name"; + } + private static Map extractDebeziumProperties( Map userProperties) { // retain only debezium properties if any diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java new file mode 100644 index 0000000000000..2b9ba316c6c4f --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java @@ -0,0 +1,32 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.risingwave.connector.source.common; + +public class MongoDbValidator extends DatabaseValidator { + + @Override + void validateDbConfig() {} + + @Override + void validateUserPrivilege() { + // TBD + + } + + @Override + void validateTable() { + // do nothing + } +} 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 c4cb6ab72a7ba..1b6c5f99e109c 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 @@ -46,7 +46,8 @@ public class DbzCdcEventConsumer private final BlockingQueue outputChannel; private final long sourceId; - private final JsonConverter converter; + private final JsonConverter payloadConverter; + private final JsonConverter keyConverter; private final String heartbeatTopicPrefix; private final String transactionTopic; @@ -64,14 +65,19 @@ public class DbzCdcEventConsumer // The default JSON converter will output the schema field in the JSON which is unnecessary // to source parser, we use a customized JSON converter to avoid outputting the `schema` // field. - var jsonConverter = new DbzJsonConverter(); + var payloadConverter = new DbzJsonConverter(); final HashMap configs = new HashMap<>(2); // only serialize the value part configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName()); // include record schema to output JSON in { "schema": { ... }, "payload": { ... } } format configs.put(JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, true); - jsonConverter.configure(configs); - this.converter = jsonConverter; + payloadConverter.configure(configs); + this.payloadConverter = payloadConverter; + + var keyConverter = new DbzJsonConverter(); + configs.put(ConverterConfig.TYPE_CONFIG, ConverterType.KEY.getName()); + keyConverter.configure(configs); + this.keyConverter = keyConverter; } private EventType getEventType(SourceRecord record) { @@ -137,7 +143,7 @@ var record = event.value(); { long trxTs = ((Struct) record.value()).getInt64("ts_ms"); byte[] payload = - converter.fromConnectData( + payloadConverter.fromConnectData( record.topic(), record.valueSchema(), record.value()); var message = msgBuilder @@ -170,15 +176,20 @@ var record = event.value(); ? System.currentTimeMillis() : sourceStruct.getInt64("ts_ms"); byte[] payload = - converter.fromConnectData( + payloadConverter.fromConnectData( record.topic(), record.valueSchema(), record.value()); + byte[] key = + keyConverter.fromConnectData( + record.topic(), record.keySchema(), record.key()); var message = msgBuilder .setFullTableName(fullTableName) .setPayload(new String(payload, StandardCharsets.UTF_8)) + .setKey(new String(key, StandardCharsets.UTF_8)) .setSourceTsMs(sourceTsMs) .build(); - LOG.debug("record => {}", message.getPayload()); + LOG.info( + "key => {}, payload => {}", message.getKey(), message.getPayload()); respBuilder.addEvents(message); break; } diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties index cd044f45544a3..57845ebf11ec5 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties @@ -11,3 +11,5 @@ heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} # TODO: set this field in the code name=${collection.name:-RW_CDC_Sharing} provide.transaction.metadata=${transactional:-false} +capture.mode=${debezium.capture.mode:-change_streams_update_full_with_pre_image} +tombstones.on.delete=${debezium.tombstones.on.delete:-false} diff --git a/java/connector-node/risingwave-source-test/pom.xml b/java/connector-node/risingwave-source-test/pom.xml index 68a3807d95f84..16bc9e7c75427 100644 --- a/java/connector-node/risingwave-source-test/pom.xml +++ b/java/connector-node/risingwave-source-test/pom.xml @@ -31,11 +31,19 @@ org.apache.logging.log4j log4j-core + + org.apache.logging.log4j + log4j-slf4j2-impl + junit junit test + + io.debezium + debezium-connector-mongodb + org.assertj assertj-core @@ -60,6 +68,11 @@ org.testcontainers postgresql + + org.testcontainers + mongodb + + org.mongodb mongodb-driver-sync diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java index d454dbdfcc84d..ae996ee5078cb 100644 --- a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java @@ -21,22 +21,20 @@ import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.concurrent.*; -import javax.sql.DataSource; +import java.util.concurrent.Callable; +import java.util.concurrent.Executors; import org.junit.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.testcontainers.containers.MongoDBContainer; public class MongoDbSourceTest { private static final Logger LOG = LoggerFactory.getLogger(MongoDbSourceTest.class.getName()); - // private static final PostgreSQLContainer pg = - // new PostgreSQLContainer<>("postgres:15-alpine") - // .withDatabaseName("test") - // .withUsername("postgres") - // .withCommand("postgres -c wal_level=logical -c max_wal_senders=10"); + private static final MongoDBContainer mongoDbContainer = null; + // new MongoDBContainer("mongodb/mongodb-community-server:4.4.23-ubi8"); - public static Server connectorServer = + public static Server connectorService = ServerBuilder.forPort(SourceTestClient.DEFAULT_PORT) .addService(new ConnectorServiceImpl()) .build(); @@ -48,45 +46,66 @@ public class MongoDbSourceTest { InsecureChannelCredentials.create()) .build()); - private static DataSource pgDataSource; - @BeforeClass public static void init() { try { - connectorServer.start(); + connectorService.start(); + } catch (Exception e) { - e.printStackTrace(); + LOG.error("failed to start connector service", e); + Assert.fail(); } + + // TODO: init data into mongodb + } @AfterClass public static void cleanup() { - connectorServer.shutdown(); + connectorService.shutdown(); + } + + static String getConnectionString() { + return mongoDbContainer.getConnectionString() + "?replicaSet=docker-rs"; } - // create a TPC-H orders table in postgres - // insert 10,000 rows into orders - // check if the number of changes debezium captures is 10,000 @Test - public void testMongoDbSource() throws Exception { - ExecutorService executorService = Executors.newFixedThreadPool(1); + public void testSnapshotLoad() throws Exception { Map props = new HashMap<>(); - props.put("mongodb.url", "mongodb://localhost:7017/?replicaSet=rs0"); + props.put("mongodb.url", "mongodb://localhost:27017/?replicaSet=rs0"); + props.put("collection.name", "dev.test"); + Iterator eventStream = + testClient.getEventStream(ConnectorServiceProto.SourceType.MONGODB, 3001, props); + Callable countTask = + () -> { + int count = 0; + while (eventStream.hasNext()) { + List messages = + eventStream.next().getEventsList(); + for (ConnectorServiceProto.CdcMessage msg : messages) { + System.out.println("recv msg: " + msg.getPayload()); + if (!msg.getPayload().isBlank()) { + count++; + } + } + if (count >= 13) { + return count; + } + } + return count; + }; + + var pool = Executors.newFixedThreadPool(1); + var result = pool.submit(countTask); + Assert.assertEquals(11, result.get().intValue()); + } + + @Test + public void testIncrementalLoad() throws Exception { + Map props = new HashMap<>(); + props.put("mongodb.url", getConnectionString()); props.put("collection.name", "dev.test"); Iterator eventStream = testClient.getEventStream(ConnectorServiceProto.SourceType.MONGODB, 3001, props); - int count = 0; - while (eventStream.hasNext()) { - List messages = eventStream.next().getEventsList(); - for (ConnectorServiceProto.CdcMessage msg : messages) { - if (!msg.getPayload().isBlank()) { - count++; - } - } - if (count >= 10000) { - break; - } - } - LOG.info("number of cdc messages received: {}", count); } } diff --git a/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties b/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties index d46bd5f609266..856287e831da8 100644 --- a/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties +++ b/java/connector-node/risingwave-source-test/src/test/resources/log4j2.properties @@ -1,4 +1,4 @@ -rootLogger.level=ERROR +rootLogger.level=INFO # declare the appender to use appenders=console # appender properties diff --git a/java/pom.xml b/java/pom.xml index 8f2979c132f62..9a18e95ff187f 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -95,7 +95,7 @@ org.apache.logging.log4j - log4j-slf4j-impl + log4j-slf4j2-impl ${log4j.version} @@ -353,6 +353,12 @@ ${testcontainers.version} test + + org.testcontainers + mongodb + ${testcontainers.version} + test + diff --git a/proto/connector_service.proto b/proto/connector_service.proto index d450bcb6e9a89..2b10c066078e4 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -162,6 +162,7 @@ message CdcMessage { string full_table_name = 4; int64 source_ts_ms = 5; bool is_transaction_meta = 6; + string key = 7; } enum SourceType { diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index a2bab33a6c92f..3696376fd6737 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -302,7 +302,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { snapshot_done: false, }, }), - pg_split: None, + postgres_split: None, _phantom: PhantomData, })], ); diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index 9a2383dbb4a96..8bb18be26461a 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -20,7 +20,8 @@ macro_rules! for_all_classified_sources { { { Mysql }, { Postgres }, - { Citus } + { Citus }, + { Mongodb } }, // other sources // todo: file source do not nest with mq source. diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 8743972ea8e6e..335667314fa60 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -656,7 +656,7 @@ mod tests { use nexmark::event::EventType; use super::*; - use crate::source::cdc::{DebeziumCdcSplit, MySqlCdcSplit}; + use crate::source::cdc::{CdcSourceType, DebeziumCdcSplit}; use crate::source::kafka::KafkaSplit; #[test] @@ -674,8 +674,7 @@ mod tests { #[test] fn test_cdc_split_state() -> Result<()> { let offset_str = "{\"sourcePartition\":{\"server\":\"RW_CDC_mydb.products\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1670407377,\"file\":\"binlog.000001\",\"pos\":98587,\"row\":2,\"server_id\":1,\"event\":2}}"; - let mysql_split = MySqlCdcSplit::new(1001, offset_str.to_string()); - let split = DebeziumCdcSplit::new(Some(mysql_split), None); + let split = DebeziumCdcSplit::::new(1001, Some(offset_str.to_string()), None); let split_impl = SplitImpl::MysqlCdc(split); let encoded_split = split_impl.encode_to_bytes(); let restored_split_impl = SplitImpl::restore_from_bytes(encoded_split.as_ref())?; diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 1664640eef03f..0033d0b44d42c 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -28,8 +28,7 @@ use risingwave_pb::connector_service::{ }; use crate::source::cdc::{ - CdcProperties, CdcSourceTypeTrait, CdcSplitBase, Citus, DebeziumCdcSplit, MySqlCdcSplit, Mysql, - Postgres, PostgresCdcSplit, + CdcProperties, CdcSourceTypeTrait, Citus, DebeziumCdcSplit, Mongodb, Mysql, Postgres, }; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -136,15 +135,11 @@ impl ListCdcSplits for DebeziumSplitEnumerator { fn list_cdc_splits(&mut self) -> Vec> { // CDC source only supports single split - let split = MySqlCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - }; - let dbz_split = DebeziumCdcSplit { - mysql_split: Some(split), - pg_split: None, - _phantom: PhantomData, - }; - vec![dbz_split] + vec![DebeziumCdcSplit::::new( + self.source_id, + None, + None, + )] } } @@ -152,16 +147,12 @@ impl ListCdcSplits for DebeziumSplitEnumerator { type CdcSourceType = Postgres; fn list_cdc_splits(&mut self) -> Vec> { - let split = PostgresCdcSplit { - inner: CdcSplitBase::new(self.source_id, None), - server_addr: None, - }; - let dbz_split = DebeziumCdcSplit { - mysql_split: None, - pg_split: Some(split), - _phantom: Default::default(), - }; - vec![dbz_split] + // CDC source only supports single split + vec![DebeziumCdcSplit::::new( + self.source_id, + None, + None, + )] } } @@ -173,16 +164,24 @@ impl ListCdcSplits for DebeziumSplitEnumerator { .iter() .enumerate() .map(|(id, addr)| { - let split = PostgresCdcSplit { - inner: CdcSplitBase::new(id as u32, None), - server_addr: Some(addr.to_string()), - }; - DebeziumCdcSplit { - mysql_split: None, - pg_split: Some(split), - _phantom: Default::default(), - } + DebeziumCdcSplit::::new( + id as u32, + None, + Some(addr.to_string()), + ) }) .collect_vec() } } +impl ListCdcSplits for DebeziumSplitEnumerator { + type CdcSourceType = Mongodb; + + fn list_cdc_splits(&mut self) -> Vec> { + // CDC source only supports single split + vec![DebeziumCdcSplit::::new( + self.source_id, + None, + None + )] + } +} diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index b3a2bc6554c60..6f9d97c5c18d5 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -39,6 +39,7 @@ 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; pub const CITUS_CDC_CONNECTOR: &str = Citus::CDC_CONNECTOR_NAME; +pub const MONGODB_CDC_CONNECTOR: &str = Mongodb::CDC_CONNECTOR_NAME; pub trait CdcSourceTypeTrait: Send + Sync + Clone + 'static { const CDC_CONNECTOR_NAME: &'static str; @@ -53,6 +54,7 @@ impl<'a> From<&'a str> for CdcSourceType { MYSQL_CDC_CONNECTOR => CdcSourceType::Mysql, POSTGRES_CDC_CONNECTOR => CdcSourceType::Postgres, CITUS_CDC_CONNECTOR => CdcSourceType::Citus, + MONGODB_CDC_CONNECTOR => CdcSourceType::Mongodb, _ => CdcSourceType::Unspecified, } } @@ -64,6 +66,7 @@ impl CdcSourceType { CdcSourceType::Mysql => "MySQL", CdcSourceType::Postgres => "Postgres", CdcSourceType::Citus => "Citus", + CdcSourceType::Mongodb => "MongoDB", CdcSourceType::Unspecified => "Unspecified", } } diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index cb9c7dae3d114..0928f34867dc5 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -71,10 +71,13 @@ impl SplitReader for CdcSplitReader { let mut properties = conn_props.properties.clone(); + let mut citus_server_addr = None; // For citus, we need to rewrite the `table.name` to capture sharding tables if matches!(T::source_type(), CdcSourceType::Citus) - && let Some(server_addr) = split.server_addr() + && let Some(ref citus_split) = split.citus_split + && let Some(ref server_addr) = citus_split.server_addr { + citus_server_addr = Some(server_addr.clone()); let host_addr = HostAddr::from_str(&server_addr) .map_err(|err| anyhow!("invalid server address for cdc split. {}", err))?; properties.insert("hostname".to_string(), host_addr.host); @@ -158,7 +161,7 @@ impl SplitReader for CdcSplitReader { tracing::info!(?source_id, "cdc connector started"); match T::source_type() { - CdcSourceType::Mysql | CdcSourceType::Postgres => Ok(Self { + CdcSourceType::Mysql | CdcSourceType::Postgres | CdcSourceType::Mongodb => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), server_addr: None, @@ -172,7 +175,7 @@ impl SplitReader for CdcSplitReader { CdcSourceType::Citus => Ok(Self { source_id: split.split_id() as u64, start_offset: split.start_offset().clone(), - server_addr: split.server_addr(), + server_addr: citus_server_addr, conn_props, split_id, snapshot_done: split.snapshot_done(), diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 4c46b27be75e8..7e1d274a6c314 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -19,7 +19,7 @@ use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; use crate::source::cdc::external::DebeziumOffset; -use crate::source::cdc::CdcSourceTypeTrait; +use crate::source::cdc::{CdcSourceType, CdcSourceTypeTrait}; use crate::source::{SplitId, SplitMetaData}; /// The base states of a CDC split, which will be persisted to checkpoint. @@ -41,6 +41,13 @@ impl CdcSplitBase { } } +trait CdcSplitTrait: Send + Sync { + fn split_id(&self) -> u32; + fn start_offset(&self) -> &Option; + fn is_snapshot_done(&self) -> bool; + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; +} + #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] pub struct MySqlCdcSplit { pub inner: CdcSplitBase, @@ -53,17 +60,36 @@ pub struct PostgresCdcSplit { pub server_addr: Option, } +#[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] +pub struct MongoDbCdcSplit { + pub inner: CdcSplitBase, +} + impl MySqlCdcSplit { - pub fn new(split_id: u32, start_offset: String) -> MySqlCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { let split = CdcSplitBase { split_id, - start_offset: Some(start_offset), + start_offset, snapshot_done: false, }; Self { inner: split } } +} - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { +impl CdcSplitTrait for MySqlCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + + fn is_snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = serde_json::from_str(&start_offset).map_err(|e| { @@ -91,10 +117,10 @@ impl MySqlCdcSplit { } impl PostgresCdcSplit { - pub fn new(split_id: u32, start_offset: String) -> PostgresCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { let split = CdcSplitBase { split_id, - start_offset: Some(start_offset), + start_offset, snapshot_done: false, }; Self { @@ -103,7 +129,31 @@ impl PostgresCdcSplit { } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + pub fn new_with_server_addr( + split_id: u32, + start_offset: Option, + server_addr: Option, + ) -> Self { + let mut result = Self::new(split_id, start_offset); + result.server_addr = server_addr; + result + } +} + +impl CdcSplitTrait for PostgresCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + + fn is_snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = serde_json::from_str(&start_offset).map_err(|e| { @@ -130,26 +180,78 @@ impl PostgresCdcSplit { } } +impl MongoDbCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { + let split = CdcSplitBase { + split_id, + start_offset, + snapshot_done: false, + }; + Self { inner: split } + } + + // TODO: update offset for mongodb + pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.inner.start_offset = Some(start_offset); + Ok(()) + } +} + +impl CdcSplitTrait for MongoDbCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + + fn is_snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + + fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + // TODO + Ok(()) + } +} + #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] pub struct DebeziumCdcSplit { pub mysql_split: Option, - pub pg_split: Option, + + #[serde(rename = "pg_split")] // backward compatibility + pub postgres_split: Option, + pub citus_split: Option, + pub mongodb_split: Option, #[serde(skip)] pub _phantom: PhantomData, } +macro_rules! dispatch_cdc_split { + ($dbz_split:expr, $as_type:tt, {$($cdc_source_type:tt),*}, $body:expr) => { + match T::source_type() { + $( + CdcSourceType::$cdc_source_type => { + $crate::paste! { + $dbz_split.[<$cdc_source_type:lower _split>] + .[]() + .expect(concat!(stringify!([<$cdc_source_type:lower>]), " split must exist")) + .$body + } + } + )* + CdcSourceType::Unspecified => { + unreachable!("invalid debezium split"); + } + } + } +} + impl SplitMetaData for DebeziumCdcSplit { fn id(&self) -> SplitId { - // TODO: may check T to get the specific cdc type - assert!(self.mysql_split.is_some() || self.pg_split.is_some()); - if let Some(split) = &self.mysql_split { - return format!("{}", split.inner.split_id).into(); - } - if let Some(split) = &self.pg_split { - return format!("{}", split.inner.split_id).into(); - } - unreachable!("invalid split") + format!("{}", self.split_id()).into() } fn encode_to_json(&self) -> JsonbVal { @@ -161,61 +263,78 @@ impl SplitMetaData for DebeziumCdcSplit { } fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - // TODO: may check T to get the specific cdc type - assert!(self.mysql_split.is_some() || self.pg_split.is_some()); - if let Some(split) = &mut self.mysql_split { - split.update_with_offset(start_offset)? - } else if let Some(split) = &mut self.pg_split { - split.update_with_offset(start_offset)? - } - Ok(()) + self.update_with_offset(start_offset) } } impl DebeziumCdcSplit { - pub fn new(mysql_split: Option, pg_split: Option) -> Self { - Self { - mysql_split, - pg_split, + pub fn new(split_id: u32, start_offset: Option, server_addr: Option) -> Self { + let mut ret = Self { + mysql_split: None, + postgres_split: None, + citus_split: None, + mongodb_split: None, _phantom: PhantomData, + }; + match T::source_type() { + CdcSourceType::Mysql => { + let split = MySqlCdcSplit::new(split_id, start_offset); + ret.mysql_split = Some(split); + } + CdcSourceType::Postgres => { + let split = PostgresCdcSplit::new(split_id, start_offset); + ret.postgres_split = Some(split); + } + CdcSourceType::Citus => { + let split = + PostgresCdcSplit::new_with_server_addr(split_id, start_offset, server_addr); + ret.citus_split = Some(split); + } + CdcSourceType::Mongodb => { + let split = MongoDbCdcSplit::new(split_id, start_offset); + ret.mongodb_split = Some(split); + } + CdcSourceType::Unspecified => { + unreachable!("invalid debezium split") + } } + ret } pub fn split_id(&self) -> u32 { - if let Some(split) = &self.mysql_split { - return split.inner.split_id; - } - if let Some(split) = &self.pg_split { - return split.inner.split_id; - } - unreachable!("invalid debezium split") + dispatch_cdc_split!(self, ref, { + Mysql, + Postgres, + Citus, + Mongodb + }, split_id()) } pub fn start_offset(&self) -> &Option { - if let Some(split) = &self.mysql_split { - return &split.inner.start_offset; - } - if let Some(split) = &self.pg_split { - return &split.inner.start_offset; - } - unreachable!("invalid debezium split") + dispatch_cdc_split!(&self, ref, { + Mysql, + Postgres, + Citus, + Mongodb + }, start_offset()) } pub fn snapshot_done(&self) -> bool { - if let Some(split) = &self.mysql_split { - return split.inner.snapshot_done; - } - if let Some(split) = &self.pg_split { - return split.inner.snapshot_done; - } - unreachable!("invalid debezium split") + dispatch_cdc_split!(self, ref, { + Mysql, + Postgres, + Citus, + Mongodb + }, is_snapshot_done()) } - pub fn server_addr(&self) -> Option { - if let Some(split) = &self.pg_split { - split.server_addr.clone() - } else { - None - } + pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + dispatch_cdc_split!(self, mut, { + Mysql, + Postgres, + Citus, + Mongodb + }, update_with_offset(start_offset)?); + Ok(()) } } diff --git a/src/connector/src/split_expand.rs b/src/connector/src/split_expand.rs new file mode 100644 index 0000000000000..0b9daacad6016 --- /dev/null +++ b/src/connector/src/split_expand.rs @@ -0,0 +1,1913 @@ +pub mod split { + use std::marker::PhantomData; + use anyhow::anyhow; + use risingwave_common::types::JsonbVal; + use serde::{Deserialize, Serialize}; + use crate::source::cdc::external::DebeziumOffset; + use crate::source::cdc::{CdcSourceType, CdcSourceTypeTrait}; + use crate::source::{SplitId, SplitMetaData}; + trait CdcSplitTrait: Send + Sync { + fn split_id(&self) -> u32; + fn start_offset(&self) -> &Option; + fn snapshot_done(&self) -> bool; + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; + } + /// The base states of a CDC split, which will be persisted to checkpoint. + /// CDC source only has single split, so we use the `source_id` to identify the split. + pub struct CdcSplitBase { + pub split_id: u32, + pub start_offset: Option, + pub snapshot_done: bool, + } + #[automatically_derived] + impl ::core::clone::Clone for CdcSplitBase { + #[inline] + fn clone(&self) -> CdcSplitBase { + CdcSplitBase { + split_id: ::core::clone::Clone::clone(&self.split_id), + start_offset: ::core::clone::Clone::clone(&self.start_offset), + snapshot_done: ::core::clone::Clone::clone(&self.snapshot_done), + } + } + } + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl _serde::Serialize for CdcSplitBase { + fn serialize<__S>( + &self, + __serializer: __S, + ) -> _serde::__private::Result<__S::Ok, __S::Error> + where + __S: _serde::Serializer, + { + let mut __serde_state = _serde::Serializer::serialize_struct( + __serializer, + "CdcSplitBase", + false as usize + 1 + 1 + 1, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "split_id", + &self.split_id, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "start_offset", + &self.start_offset, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "snapshot_done", + &self.snapshot_done, + )?; + _serde::ser::SerializeStruct::end(__serde_state) + } + } + }; + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl<'de> _serde::Deserialize<'de> for CdcSplitBase { + fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + #[allow(non_camel_case_types)] + #[doc(hidden)] + enum __Field { + __field0, + __field1, + __field2, + __ignore, + } + #[doc(hidden)] + struct __FieldVisitor; + impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { + type Value = __Field; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "field identifier") + } + fn visit_u64<__E>( + self, + __value: u64, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + 0u64 => _serde::__private::Ok(__Field::__field0), + 1u64 => _serde::__private::Ok(__Field::__field1), + 2u64 => _serde::__private::Ok(__Field::__field2), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_str<__E>( + self, + __value: &str, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + "split_id" => _serde::__private::Ok(__Field::__field0), + "start_offset" => _serde::__private::Ok(__Field::__field1), + "snapshot_done" => _serde::__private::Ok(__Field::__field2), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_bytes<__E>( + self, + __value: &[u8], + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + b"split_id" => _serde::__private::Ok(__Field::__field0), + b"start_offset" => _serde::__private::Ok(__Field::__field1), + b"snapshot_done" => _serde::__private::Ok(__Field::__field2), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + } + impl<'de> _serde::Deserialize<'de> for __Field { + #[inline] + fn deserialize<__D>( + __deserializer: __D, + ) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) + } + } + #[doc(hidden)] + struct __Visitor<'de> { + marker: _serde::__private::PhantomData, + lifetime: _serde::__private::PhantomData<&'de ()>, + } + impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { + type Value = CdcSplitBase; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "struct CdcSplitBase") + } + #[inline] + fn visit_seq<__A>( + self, + mut __seq: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::SeqAccess<'de>, + { + let __field0 = match _serde::de::SeqAccess::next_element::(&mut __seq)? + { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 0usize, + &"struct CdcSplitBase with 3 elements", + )) + } + }; + let __field1 = match _serde::de::SeqAccess::next_element::>( + &mut __seq, + )? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 1usize, + &"struct CdcSplitBase with 3 elements", + )) + } + }; + let __field2 = + match _serde::de::SeqAccess::next_element::(&mut __seq)? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err( + _serde::de::Error::invalid_length( + 2usize, + &"struct CdcSplitBase with 3 elements", + ), + ) + } + }; + _serde::__private::Ok(CdcSplitBase { + split_id: __field0, + start_offset: __field1, + snapshot_done: __field2, + }) + } + #[inline] + fn visit_map<__A>( + self, + mut __map: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::MapAccess<'de>, + { + let mut __field0: _serde::__private::Option = _serde::__private::None; + let mut __field1: _serde::__private::Option> = + _serde::__private::None; + let mut __field2: _serde::__private::Option = _serde::__private::None; + while let _serde::__private::Some(__key) = + _serde::de::MapAccess::next_key::<__Field>(&mut __map)? + { + match __key { + __Field::__field0 => { + if _serde::__private::Option::is_some(&__field0) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "split_id", + ), + ); + } + __field0 = _serde::__private::Some( + _serde::de::MapAccess::next_value::(&mut __map)?, + ); + } + __Field::__field1 => { + if _serde::__private::Option::is_some(&__field1) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "start_offset", + ), + ); + } + __field1 = _serde::__private::Some( + _serde::de::MapAccess::next_value::>( + &mut __map, + )?, + ); + } + __Field::__field2 => { + if _serde::__private::Option::is_some(&__field2) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "snapshot_done", + ), + ); + } + __field2 = _serde::__private::Some( + _serde::de::MapAccess::next_value::(&mut __map)?, + ); + } + _ => { + let _ = _serde::de::MapAccess::next_value::< + _serde::de::IgnoredAny, + >(&mut __map)?; + } + } + } + let __field0 = match __field0 { + _serde::__private::Some(__field0) => __field0, + _serde::__private::None => { + _serde::__private::de::missing_field("split_id")? + } + }; + let __field1 = match __field1 { + _serde::__private::Some(__field1) => __field1, + _serde::__private::None => { + _serde::__private::de::missing_field("start_offset")? + } + }; + let __field2 = match __field2 { + _serde::__private::Some(__field2) => __field2, + _serde::__private::None => { + _serde::__private::de::missing_field("snapshot_done")? + } + }; + _serde::__private::Ok(CdcSplitBase { + split_id: __field0, + start_offset: __field1, + snapshot_done: __field2, + }) + } + } + #[doc(hidden)] + const FIELDS: &'static [&'static str] = + &["split_id", "start_offset", "snapshot_done"]; + _serde::Deserializer::deserialize_struct( + __deserializer, + "CdcSplitBase", + FIELDS, + __Visitor { + marker: _serde::__private::PhantomData::, + lifetime: _serde::__private::PhantomData, + }, + ) + } + } + }; + #[automatically_derived] + impl ::core::fmt::Debug for CdcSplitBase { + #[inline] + fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { + ::core::fmt::Formatter::debug_struct_field3_finish( + f, + "CdcSplitBase", + "split_id", + &self.split_id, + "start_offset", + &self.start_offset, + "snapshot_done", + &&self.snapshot_done, + ) + } + } + #[automatically_derived] + impl ::core::marker::StructuralPartialEq for CdcSplitBase {} + #[automatically_derived] + impl ::core::cmp::PartialEq for CdcSplitBase { + #[inline] + fn eq(&self, other: &CdcSplitBase) -> bool { + self.split_id == other.split_id + && self.start_offset == other.start_offset + && self.snapshot_done == other.snapshot_done + } + } + #[automatically_derived] + impl ::core::hash::Hash for CdcSplitBase { + #[inline] + fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { + ::core::hash::Hash::hash(&self.split_id, state); + ::core::hash::Hash::hash(&self.start_offset, state); + ::core::hash::Hash::hash(&self.snapshot_done, state) + } + } + impl CdcSplitBase { + pub fn new(split_id: u32, start_offset: Option) -> Self { + Self { + split_id, + start_offset, + snapshot_done: false, + } + } + } + pub struct MySqlCdcSplit { + pub inner: CdcSplitBase, + } + #[automatically_derived] + impl ::core::clone::Clone for MySqlCdcSplit { + #[inline] + fn clone(&self) -> MySqlCdcSplit { + MySqlCdcSplit { + inner: ::core::clone::Clone::clone(&self.inner), + } + } + } + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl _serde::Serialize for MySqlCdcSplit { + fn serialize<__S>( + &self, + __serializer: __S, + ) -> _serde::__private::Result<__S::Ok, __S::Error> + where + __S: _serde::Serializer, + { + let mut __serde_state = _serde::Serializer::serialize_struct( + __serializer, + "MySqlCdcSplit", + false as usize + 1, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "inner", + &self.inner, + )?; + _serde::ser::SerializeStruct::end(__serde_state) + } + } + }; + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl<'de> _serde::Deserialize<'de> for MySqlCdcSplit { + fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + #[allow(non_camel_case_types)] + #[doc(hidden)] + enum __Field { + __field0, + __ignore, + } + #[doc(hidden)] + struct __FieldVisitor; + impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { + type Value = __Field; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "field identifier") + } + fn visit_u64<__E>( + self, + __value: u64, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + 0u64 => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_str<__E>( + self, + __value: &str, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + "inner" => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_bytes<__E>( + self, + __value: &[u8], + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + b"inner" => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + } + impl<'de> _serde::Deserialize<'de> for __Field { + #[inline] + fn deserialize<__D>( + __deserializer: __D, + ) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) + } + } + #[doc(hidden)] + struct __Visitor<'de> { + marker: _serde::__private::PhantomData, + lifetime: _serde::__private::PhantomData<&'de ()>, + } + impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { + type Value = MySqlCdcSplit; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "struct MySqlCdcSplit") + } + #[inline] + fn visit_seq<__A>( + self, + mut __seq: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::SeqAccess<'de>, + { + let __field0 = match _serde::de::SeqAccess::next_element::( + &mut __seq, + )? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 0usize, + &"struct MySqlCdcSplit with 1 element", + )) + } + }; + _serde::__private::Ok(MySqlCdcSplit { inner: __field0 }) + } + #[inline] + fn visit_map<__A>( + self, + mut __map: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::MapAccess<'de>, + { + let mut __field0: _serde::__private::Option = + _serde::__private::None; + while let _serde::__private::Some(__key) = + _serde::de::MapAccess::next_key::<__Field>(&mut __map)? + { + match __key { + __Field::__field0 => { + if _serde::__private::Option::is_some(&__field0) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "inner", + ), + ); + } + __field0 = _serde::__private::Some( + _serde::de::MapAccess::next_value::( + &mut __map, + )?, + ); + } + _ => { + let _ = _serde::de::MapAccess::next_value::< + _serde::de::IgnoredAny, + >(&mut __map)?; + } + } + } + let __field0 = match __field0 { + _serde::__private::Some(__field0) => __field0, + _serde::__private::None => { + _serde::__private::de::missing_field("inner")? + } + }; + _serde::__private::Ok(MySqlCdcSplit { inner: __field0 }) + } + } + #[doc(hidden)] + const FIELDS: &'static [&'static str] = &["inner"]; + _serde::Deserializer::deserialize_struct( + __deserializer, + "MySqlCdcSplit", + FIELDS, + __Visitor { + marker: _serde::__private::PhantomData::, + lifetime: _serde::__private::PhantomData, + }, + ) + } + } + }; + #[automatically_derived] + impl ::core::fmt::Debug for MySqlCdcSplit { + #[inline] + fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { + ::core::fmt::Formatter::debug_struct_field1_finish( + f, + "MySqlCdcSplit", + "inner", + &&self.inner, + ) + } + } + #[automatically_derived] + impl ::core::marker::StructuralPartialEq for MySqlCdcSplit {} + #[automatically_derived] + impl ::core::cmp::PartialEq for MySqlCdcSplit { + #[inline] + fn eq(&self, other: &MySqlCdcSplit) -> bool { + self.inner == other.inner + } + } + #[automatically_derived] + impl ::core::hash::Hash for MySqlCdcSplit { + #[inline] + fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { + ::core::hash::Hash::hash(&self.inner, state) + } + } + pub struct PostgresCdcSplit { + pub inner: CdcSplitBase, + pub server_addr: Option, + } + #[automatically_derived] + impl ::core::clone::Clone for PostgresCdcSplit { + #[inline] + fn clone(&self) -> PostgresCdcSplit { + PostgresCdcSplit { + inner: ::core::clone::Clone::clone(&self.inner), + server_addr: ::core::clone::Clone::clone(&self.server_addr), + } + } + } + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl _serde::Serialize for PostgresCdcSplit { + fn serialize<__S>( + &self, + __serializer: __S, + ) -> _serde::__private::Result<__S::Ok, __S::Error> + where + __S: _serde::Serializer, + { + let mut __serde_state = _serde::Serializer::serialize_struct( + __serializer, + "PostgresCdcSplit", + false as usize + 1 + 1, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "inner", + &self.inner, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "server_addr", + &self.server_addr, + )?; + _serde::ser::SerializeStruct::end(__serde_state) + } + } + }; + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl<'de> _serde::Deserialize<'de> for PostgresCdcSplit { + fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + #[allow(non_camel_case_types)] + #[doc(hidden)] + enum __Field { + __field0, + __field1, + __ignore, + } + #[doc(hidden)] + struct __FieldVisitor; + impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { + type Value = __Field; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "field identifier") + } + fn visit_u64<__E>( + self, + __value: u64, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + 0u64 => _serde::__private::Ok(__Field::__field0), + 1u64 => _serde::__private::Ok(__Field::__field1), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_str<__E>( + self, + __value: &str, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + "inner" => _serde::__private::Ok(__Field::__field0), + "server_addr" => _serde::__private::Ok(__Field::__field1), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_bytes<__E>( + self, + __value: &[u8], + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + b"inner" => _serde::__private::Ok(__Field::__field0), + b"server_addr" => _serde::__private::Ok(__Field::__field1), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + } + impl<'de> _serde::Deserialize<'de> for __Field { + #[inline] + fn deserialize<__D>( + __deserializer: __D, + ) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) + } + } + #[doc(hidden)] + struct __Visitor<'de> { + marker: _serde::__private::PhantomData, + lifetime: _serde::__private::PhantomData<&'de ()>, + } + impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { + type Value = PostgresCdcSplit; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str( + __formatter, + "struct PostgresCdcSplit", + ) + } + #[inline] + fn visit_seq<__A>( + self, + mut __seq: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::SeqAccess<'de>, + { + let __field0 = match _serde::de::SeqAccess::next_element::( + &mut __seq, + )? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 0usize, + &"struct PostgresCdcSplit with 2 elements", + )) + } + }; + let __field1 = match _serde::de::SeqAccess::next_element::>( + &mut __seq, + )? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 1usize, + &"struct PostgresCdcSplit with 2 elements", + )) + } + }; + _serde::__private::Ok(PostgresCdcSplit { + inner: __field0, + server_addr: __field1, + }) + } + #[inline] + fn visit_map<__A>( + self, + mut __map: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::MapAccess<'de>, + { + let mut __field0: _serde::__private::Option = + _serde::__private::None; + let mut __field1: _serde::__private::Option> = + _serde::__private::None; + while let _serde::__private::Some(__key) = + _serde::de::MapAccess::next_key::<__Field>(&mut __map)? + { + match __key { + __Field::__field0 => { + if _serde::__private::Option::is_some(&__field0) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "inner", + ), + ); + } + __field0 = _serde::__private::Some( + _serde::de::MapAccess::next_value::( + &mut __map, + )?, + ); + } + __Field::__field1 => { + if _serde::__private::Option::is_some(&__field1) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "server_addr", + ), + ); + } + __field1 = _serde::__private::Some( + _serde::de::MapAccess::next_value::>( + &mut __map, + )?, + ); + } + _ => { + let _ = _serde::de::MapAccess::next_value::< + _serde::de::IgnoredAny, + >(&mut __map)?; + } + } + } + let __field0 = match __field0 { + _serde::__private::Some(__field0) => __field0, + _serde::__private::None => { + _serde::__private::de::missing_field("inner")? + } + }; + let __field1 = match __field1 { + _serde::__private::Some(__field1) => __field1, + _serde::__private::None => { + _serde::__private::de::missing_field("server_addr")? + } + }; + _serde::__private::Ok(PostgresCdcSplit { + inner: __field0, + server_addr: __field1, + }) + } + } + #[doc(hidden)] + const FIELDS: &'static [&'static str] = &["inner", "server_addr"]; + _serde::Deserializer::deserialize_struct( + __deserializer, + "PostgresCdcSplit", + FIELDS, + __Visitor { + marker: _serde::__private::PhantomData::, + lifetime: _serde::__private::PhantomData, + }, + ) + } + } + }; + #[automatically_derived] + impl ::core::fmt::Debug for PostgresCdcSplit { + #[inline] + fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { + ::core::fmt::Formatter::debug_struct_field2_finish( + f, + "PostgresCdcSplit", + "inner", + &self.inner, + "server_addr", + &&self.server_addr, + ) + } + } + #[automatically_derived] + impl ::core::marker::StructuralPartialEq for PostgresCdcSplit {} + #[automatically_derived] + impl ::core::cmp::PartialEq for PostgresCdcSplit { + #[inline] + fn eq(&self, other: &PostgresCdcSplit) -> bool { + self.inner == other.inner && self.server_addr == other.server_addr + } + } + #[automatically_derived] + impl ::core::hash::Hash for PostgresCdcSplit { + #[inline] + fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { + ::core::hash::Hash::hash(&self.inner, state); + ::core::hash::Hash::hash(&self.server_addr, state) + } + } + pub struct MongoDbCdcSplit { + pub inner: CdcSplitBase, + } + #[automatically_derived] + impl ::core::clone::Clone for MongoDbCdcSplit { + #[inline] + fn clone(&self) -> MongoDbCdcSplit { + MongoDbCdcSplit { + inner: ::core::clone::Clone::clone(&self.inner), + } + } + } + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl _serde::Serialize for MongoDbCdcSplit { + fn serialize<__S>( + &self, + __serializer: __S, + ) -> _serde::__private::Result<__S::Ok, __S::Error> + where + __S: _serde::Serializer, + { + let mut __serde_state = _serde::Serializer::serialize_struct( + __serializer, + "MongoDbCdcSplit", + false as usize + 1, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "inner", + &self.inner, + )?; + _serde::ser::SerializeStruct::end(__serde_state) + } + } + }; + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl<'de> _serde::Deserialize<'de> for MongoDbCdcSplit { + fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + #[allow(non_camel_case_types)] + #[doc(hidden)] + enum __Field { + __field0, + __ignore, + } + #[doc(hidden)] + struct __FieldVisitor; + impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { + type Value = __Field; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "field identifier") + } + fn visit_u64<__E>( + self, + __value: u64, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + 0u64 => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_str<__E>( + self, + __value: &str, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + "inner" => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_bytes<__E>( + self, + __value: &[u8], + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + b"inner" => _serde::__private::Ok(__Field::__field0), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + } + impl<'de> _serde::Deserialize<'de> for __Field { + #[inline] + fn deserialize<__D>( + __deserializer: __D, + ) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) + } + } + #[doc(hidden)] + struct __Visitor<'de> { + marker: _serde::__private::PhantomData, + lifetime: _serde::__private::PhantomData<&'de ()>, + } + impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { + type Value = MongoDbCdcSplit; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str( + __formatter, + "struct MongoDbCdcSplit", + ) + } + #[inline] + fn visit_seq<__A>( + self, + mut __seq: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::SeqAccess<'de>, + { + let __field0 = match _serde::de::SeqAccess::next_element::( + &mut __seq, + )? { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 0usize, + &"struct MongoDbCdcSplit with 1 element", + )) + } + }; + _serde::__private::Ok(MongoDbCdcSplit { inner: __field0 }) + } + #[inline] + fn visit_map<__A>( + self, + mut __map: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::MapAccess<'de>, + { + let mut __field0: _serde::__private::Option = + _serde::__private::None; + while let _serde::__private::Some(__key) = + _serde::de::MapAccess::next_key::<__Field>(&mut __map)? + { + match __key { + __Field::__field0 => { + if _serde::__private::Option::is_some(&__field0) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "inner", + ), + ); + } + __field0 = _serde::__private::Some( + _serde::de::MapAccess::next_value::( + &mut __map, + )?, + ); + } + _ => { + let _ = _serde::de::MapAccess::next_value::< + _serde::de::IgnoredAny, + >(&mut __map)?; + } + } + } + let __field0 = match __field0 { + _serde::__private::Some(__field0) => __field0, + _serde::__private::None => { + _serde::__private::de::missing_field("inner")? + } + }; + _serde::__private::Ok(MongoDbCdcSplit { inner: __field0 }) + } + } + #[doc(hidden)] + const FIELDS: &'static [&'static str] = &["inner"]; + _serde::Deserializer::deserialize_struct( + __deserializer, + "MongoDbCdcSplit", + FIELDS, + __Visitor { + marker: _serde::__private::PhantomData::, + lifetime: _serde::__private::PhantomData, + }, + ) + } + } + }; + #[automatically_derived] + impl ::core::fmt::Debug for MongoDbCdcSplit { + #[inline] + fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { + ::core::fmt::Formatter::debug_struct_field1_finish( + f, + "MongoDbCdcSplit", + "inner", + &&self.inner, + ) + } + } + #[automatically_derived] + impl ::core::marker::StructuralPartialEq for MongoDbCdcSplit {} + #[automatically_derived] + impl ::core::cmp::PartialEq for MongoDbCdcSplit { + #[inline] + fn eq(&self, other: &MongoDbCdcSplit) -> bool { + self.inner == other.inner + } + } + #[automatically_derived] + impl ::core::hash::Hash for MongoDbCdcSplit { + #[inline] + fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { + ::core::hash::Hash::hash(&self.inner, state) + } + } + impl MySqlCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { + let split = CdcSplitBase { + split_id, + start_offset, + snapshot_done: false, + }; + Self { inner: split } + } + } + impl CdcSplitTrait for MySqlCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + fn snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let mut snapshot_done = self.inner.snapshot_done; + if !snapshot_done { + let dbz_offset: DebeziumOffset = + serde_json::from_str(&start_offset).map_err(|e| { + ::anyhow::Error::msg({ + let res = ::alloc::fmt::format(format_args!( + "invalid mysql offset: {0}, error: {1}, split: {2}", + start_offset, e, self.inner.split_id + )); + res + }) + })?; + if !dbz_offset.is_heartbeat { + snapshot_done = match dbz_offset.source_offset.snapshot { + Some(val) => !val, + None => true, + }; + } + } + self.inner.start_offset = Some(start_offset); + self.inner.snapshot_done = snapshot_done; + Ok(()) + } + } + impl PostgresCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { + let split = CdcSplitBase { + split_id, + start_offset, + snapshot_done: false, + }; + Self { + inner: split, + server_addr: None, + } + } + pub fn new_with_server_addr( + split_id: u32, + start_offset: Option, + server_addr: Option, + ) -> Self { + let mut result = Self::new(split_id, start_offset); + result.server_addr = server_addr; + result + } + } + impl CdcSplitTrait for PostgresCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + fn snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + let mut snapshot_done = self.inner.snapshot_done; + if !snapshot_done { + let dbz_offset: DebeziumOffset = + serde_json::from_str(&start_offset).map_err(|e| { + ::anyhow::Error::msg({ + let res = ::alloc::fmt::format(format_args!( + "invalid postgres offset: {0}, error: {1}, split: {2}", + start_offset, e, self.inner.split_id + )); + res + }) + })?; + if !dbz_offset.is_heartbeat { + snapshot_done = dbz_offset + .source_offset + .last_snapshot_record + .unwrap_or(false); + } + } + self.inner.start_offset = Some(start_offset); + self.inner.snapshot_done = snapshot_done; + Ok(()) + } + } + impl MongoDbCdcSplit { + pub fn new(split_id: u32, start_offset: Option) -> Self { + let split = CdcSplitBase { + split_id, + start_offset, + snapshot_done: false, + }; + Self { inner: split } + } + pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + self.inner.start_offset = Some(start_offset); + Ok(()) + } + } + impl CdcSplitTrait for MongoDbCdcSplit { + fn split_id(&self) -> u32 { + self.inner.split_id + } + fn start_offset(&self) -> &Option { + &self.inner.start_offset + } + fn snapshot_done(&self) -> bool { + self.inner.snapshot_done + } + fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + Ok(()) + } + } + pub struct DebeziumCdcSplit { + pub mysql_split: Option, + #[serde(rename = "pg_split")] + pub postgres_split: Option, + pub citus_split: Option, + pub mongodb_split: Option, + #[serde(skip)] + pub _phantom: PhantomData, + } + #[automatically_derived] + impl ::core::clone::Clone for DebeziumCdcSplit { + #[inline] + fn clone(&self) -> DebeziumCdcSplit { + DebeziumCdcSplit { + mysql_split: ::core::clone::Clone::clone(&self.mysql_split), + postgres_split: ::core::clone::Clone::clone(&self.postgres_split), + citus_split: ::core::clone::Clone::clone(&self.citus_split), + mongodb_split: ::core::clone::Clone::clone(&self.mongodb_split), + _phantom: ::core::clone::Clone::clone(&self._phantom), + } + } + } + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl _serde::Serialize for DebeziumCdcSplit { + fn serialize<__S>( + &self, + __serializer: __S, + ) -> _serde::__private::Result<__S::Ok, __S::Error> + where + __S: _serde::Serializer, + { + let mut __serde_state = _serde::Serializer::serialize_struct( + __serializer, + "DebeziumCdcSplit", + false as usize + 1 + 1 + 1 + 1, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "mysql_split", + &self.mysql_split, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "pg_split", + &self.postgres_split, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "citus_split", + &self.citus_split, + )?; + _serde::ser::SerializeStruct::serialize_field( + &mut __serde_state, + "mongodb_split", + &self.mongodb_split, + )?; + _serde::ser::SerializeStruct::end(__serde_state) + } + } + }; + #[doc(hidden)] + #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] + const _: () = { + #[allow(unused_extern_crates, clippy::useless_attribute)] + extern crate serde as _serde; + #[automatically_derived] + impl<'de, T: CdcSourceTypeTrait> _serde::Deserialize<'de> for DebeziumCdcSplit { + fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + #[allow(non_camel_case_types)] + #[doc(hidden)] + enum __Field { + __field0, + __field1, + __field2, + __field3, + __ignore, + } + #[doc(hidden)] + struct __FieldVisitor; + impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { + type Value = __Field; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str(__formatter, "field identifier") + } + fn visit_u64<__E>( + self, + __value: u64, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + 0u64 => _serde::__private::Ok(__Field::__field0), + 1u64 => _serde::__private::Ok(__Field::__field1), + 2u64 => _serde::__private::Ok(__Field::__field2), + 3u64 => _serde::__private::Ok(__Field::__field3), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_str<__E>( + self, + __value: &str, + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + "mysql_split" => _serde::__private::Ok(__Field::__field0), + "pg_split" => _serde::__private::Ok(__Field::__field1), + "citus_split" => _serde::__private::Ok(__Field::__field2), + "mongodb_split" => _serde::__private::Ok(__Field::__field3), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + fn visit_bytes<__E>( + self, + __value: &[u8], + ) -> _serde::__private::Result + where + __E: _serde::de::Error, + { + match __value { + b"mysql_split" => _serde::__private::Ok(__Field::__field0), + b"pg_split" => _serde::__private::Ok(__Field::__field1), + b"citus_split" => _serde::__private::Ok(__Field::__field2), + b"mongodb_split" => _serde::__private::Ok(__Field::__field3), + _ => _serde::__private::Ok(__Field::__ignore), + } + } + } + impl<'de> _serde::Deserialize<'de> for __Field { + #[inline] + fn deserialize<__D>( + __deserializer: __D, + ) -> _serde::__private::Result + where + __D: _serde::Deserializer<'de>, + { + _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) + } + } + #[doc(hidden)] + struct __Visitor<'de, T: CdcSourceTypeTrait> { + marker: _serde::__private::PhantomData>, + lifetime: _serde::__private::PhantomData<&'de ()>, + } + impl<'de, T: CdcSourceTypeTrait> _serde::de::Visitor<'de> for __Visitor<'de, T> { + type Value = DebeziumCdcSplit; + fn expecting( + &self, + __formatter: &mut _serde::__private::Formatter, + ) -> _serde::__private::fmt::Result { + _serde::__private::Formatter::write_str( + __formatter, + "struct DebeziumCdcSplit", + ) + } + #[inline] + fn visit_seq<__A>( + self, + mut __seq: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::SeqAccess<'de>, + { + let __field0 = match _serde::de::SeqAccess::next_element::< + Option, + >(&mut __seq)? + { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 0usize, + &"struct DebeziumCdcSplit with 4 elements", + )) + } + }; + let __field1 = match _serde::de::SeqAccess::next_element::< + Option, + >(&mut __seq)? + { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 1usize, + &"struct DebeziumCdcSplit with 4 elements", + )) + } + }; + let __field2 = match _serde::de::SeqAccess::next_element::< + Option, + >(&mut __seq)? + { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 2usize, + &"struct DebeziumCdcSplit with 4 elements", + )) + } + }; + let __field3 = match _serde::de::SeqAccess::next_element::< + Option, + >(&mut __seq)? + { + _serde::__private::Some(__value) => __value, + _serde::__private::None => { + return _serde::__private::Err(_serde::de::Error::invalid_length( + 3usize, + &"struct DebeziumCdcSplit with 4 elements", + )) + } + }; + let __field4 = _serde::__private::Default::default(); + _serde::__private::Ok(DebeziumCdcSplit { + mysql_split: __field0, + postgres_split: __field1, + citus_split: __field2, + mongodb_split: __field3, + _phantom: __field4, + }) + } + #[inline] + fn visit_map<__A>( + self, + mut __map: __A, + ) -> _serde::__private::Result + where + __A: _serde::de::MapAccess<'de>, + { + let mut __field0: _serde::__private::Option> = + _serde::__private::None; + let mut __field1: _serde::__private::Option> = + _serde::__private::None; + let mut __field2: _serde::__private::Option> = + _serde::__private::None; + let mut __field3: _serde::__private::Option> = + _serde::__private::None; + while let _serde::__private::Some(__key) = + _serde::de::MapAccess::next_key::<__Field>(&mut __map)? + { + match __key { + __Field::__field0 => { + if _serde::__private::Option::is_some(&__field0) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "mysql_split", + ), + ); + } + __field0 = _serde::__private::Some( + _serde::de::MapAccess::next_value::>( + &mut __map, + )?, + ); + } + __Field::__field1 => { + if _serde::__private::Option::is_some(&__field1) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "pg_split", + ), + ); + } + __field1 = _serde::__private::Some( + _serde::de::MapAccess::next_value::< + Option, + >(&mut __map)?, + ); + } + __Field::__field2 => { + if _serde::__private::Option::is_some(&__field2) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "citus_split", + ), + ); + } + __field2 = _serde::__private::Some( + _serde::de::MapAccess::next_value::< + Option, + >(&mut __map)?, + ); + } + __Field::__field3 => { + if _serde::__private::Option::is_some(&__field3) { + return _serde::__private::Err( + <__A::Error as _serde::de::Error>::duplicate_field( + "mongodb_split", + ), + ); + } + __field3 = _serde::__private::Some( + _serde::de::MapAccess::next_value::>( + &mut __map, + )?, + ); + } + _ => { + let _ = _serde::de::MapAccess::next_value::< + _serde::de::IgnoredAny, + >(&mut __map)?; + } + } + } + let __field0 = match __field0 { + _serde::__private::Some(__field0) => __field0, + _serde::__private::None => { + _serde::__private::de::missing_field("mysql_split")? + } + }; + let __field1 = match __field1 { + _serde::__private::Some(__field1) => __field1, + _serde::__private::None => { + _serde::__private::de::missing_field("pg_split")? + } + }; + let __field2 = match __field2 { + _serde::__private::Some(__field2) => __field2, + _serde::__private::None => { + _serde::__private::de::missing_field("citus_split")? + } + }; + let __field3 = match __field3 { + _serde::__private::Some(__field3) => __field3, + _serde::__private::None => { + _serde::__private::de::missing_field("mongodb_split")? + } + }; + _serde::__private::Ok(DebeziumCdcSplit { + mysql_split: __field0, + postgres_split: __field1, + citus_split: __field2, + mongodb_split: __field3, + _phantom: _serde::__private::Default::default(), + }) + } + } + #[doc(hidden)] + const FIELDS: &'static [&'static str] = + &["mysql_split", "pg_split", "citus_split", "mongodb_split"]; + _serde::Deserializer::deserialize_struct( + __deserializer, + "DebeziumCdcSplit", + FIELDS, + __Visitor { + marker: _serde::__private::PhantomData::>, + lifetime: _serde::__private::PhantomData, + }, + ) + } + } + }; + #[automatically_derived] + impl ::core::fmt::Debug for DebeziumCdcSplit { + #[inline] + fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { + ::core::fmt::Formatter::debug_struct_field5_finish( + f, + "DebeziumCdcSplit", + "mysql_split", + &self.mysql_split, + "postgres_split", + &self.postgres_split, + "citus_split", + &self.citus_split, + "mongodb_split", + &self.mongodb_split, + "_phantom", + &&self._phantom, + ) + } + } + #[automatically_derived] + impl ::core::marker::StructuralPartialEq for DebeziumCdcSplit {} + #[automatically_derived] + impl ::core::cmp::PartialEq + for DebeziumCdcSplit + { + #[inline] + fn eq(&self, other: &DebeziumCdcSplit) -> bool { + self.mysql_split == other.mysql_split + && self.postgres_split == other.postgres_split + && self.citus_split == other.citus_split + && self.mongodb_split == other.mongodb_split + && self._phantom == other._phantom + } + } + #[automatically_derived] + impl ::core::hash::Hash for DebeziumCdcSplit { + #[inline] + fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { + ::core::hash::Hash::hash(&self.mysql_split, state); + ::core::hash::Hash::hash(&self.postgres_split, state); + ::core::hash::Hash::hash(&self.citus_split, state); + ::core::hash::Hash::hash(&self.mongodb_split, state); + ::core::hash::Hash::hash(&self._phantom, state) + } + } + impl SplitMetaData for DebeziumCdcSplit { + fn id(&self) -> SplitId { + { + let res = ::alloc::fmt::format(format_args!("{0}", self.split_id())); + res + } + .into() + } + fn encode_to_json(&self) -> JsonbVal { + serde_json::to_value(self.clone()).unwrap().into() + } + fn restore_from_json(value: JsonbVal) -> anyhow::Result { + serde_json::from_value(value.take()).map_err(|e| { + ::anyhow::__private::must_use({ + use ::anyhow::__private::kind::*; + let error = match e { + error => (&error).anyhow_kind().new(error), + }; + error + }) + }) + } + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + match T::source_type() { + CdcSourceType::Mysql => self + .mysql_split + .as_mut() + .expect("mysql split must exist") + .update_with_offset(start_offset)?, + CdcSourceType::Postgres => self + .postgres_split + .as_mut() + .expect("postgres split must exist") + .update_with_offset(start_offset)?, + CdcSourceType::Citus => self + .citus_split + .as_mut() + .expect("citus split must exist") + .update_with_offset(start_offset)?, + CdcSourceType::Mongodb => self + .mongodb_split + .as_mut() + .expect("mongodb split must exist") + .update_with_offset(start_offset)?, + CdcSourceType::Unspecified => { + { + ::core::panicking::panic_fmt(format_args!( + "internal error: entered unreachable code: {0}", + format_args!("invalid debezium split") + )); + }; + } + }; + Ok(()) + } + } + impl DebeziumCdcSplit { + pub fn new_mysql(split_id: u32, start_offset: Option) -> Self { + if !match T::source_type() { + CdcSourceType::Mysql => true, + _ => false, + } { + ::core::panicking::panic( + "assertion failed: matches!(T::source_type(), CdcSourceType::Mysql)", + ) + }; + let split = MySqlCdcSplit::new(split_id, start_offset); + Self { + mysql_split: Some(split), + postgres_split: None, + citus_split: None, + mongodb_split: None, + _phantom: PhantomData, + } + } + pub fn new_postgres(split_id: u32, start_offset: Option) -> Self { + if !match T::source_type() { + CdcSourceType::Postgres => true, + _ => false, + } { + ::core::panicking::panic( + "assertion failed: matches!(T::source_type(), CdcSourceType::Postgres)", + ) + }; + let split = PostgresCdcSplit::new(split_id, start_offset); + Self { + mysql_split: None, + postgres_split: Some(split), + citus_split: None, + mongodb_split: None, + _phantom: PhantomData, + } + } + pub fn new_citus( + split_id: u32, + start_offset: Option, + server_addr: Option, + ) -> Self { + if !match T::source_type() { + CdcSourceType::Citus => true, + _ => false, + } { + ::core::panicking::panic( + "assertion failed: matches!(T::source_type(), CdcSourceType::Citus)", + ) + }; + let split = PostgresCdcSplit::new_with_server_addr(split_id, start_offset, server_addr); + Self { + mysql_split: None, + postgres_split: None, + citus_split: Some(split), + mongodb_split: None, + _phantom: PhantomData, + } + } + pub fn new_mongodb(split_id: u32, start_offset: Option) -> Self { + if !match T::source_type() { + CdcSourceType::Mongodb => true, + _ => false, + } { + ::core::panicking::panic( + "assertion failed: matches!(T::source_type(), CdcSourceType::Mongodb)", + ) + }; + let split = MongoDbCdcSplit::new(split_id, start_offset); + Self { + mysql_split: None, + postgres_split: None, + citus_split: None, + mongodb_split: Some(split), + _phantom: PhantomData, + } + } + pub fn split_id(&self) -> u32 { + match T::source_type() { + CdcSourceType::Mysql => self + .mysql_split + .as_ref() + .expect("mysql split must exist") + .split_id(), + CdcSourceType::Postgres => self + .postgres_split + .as_ref() + .expect("postgres split must exist") + .split_id(), + CdcSourceType::Citus => self + .citus_split + .as_ref() + .expect("citus split must exist") + .split_id(), + CdcSourceType::Mongodb => self + .mongodb_split + .as_ref() + .expect("mongodb split must exist") + .split_id(), + CdcSourceType::Unspecified => { + { + ::core::panicking::panic_fmt(format_args!( + "internal error: entered unreachable code: {0}", + format_args!("invalid debezium split") + )); + }; + } + } + } + pub fn start_offset(&self) -> &Option { + match T::source_type() { + CdcSourceType::Mysql => &self + .mysql_split + .as_ref() + .expect("mysql split must exist") + .start_offset(), + CdcSourceType::Postgres => &self + .postgres_split + .as_ref() + .expect("postgres split must exist") + .start_offset(), + CdcSourceType::Citus => &self + .citus_split + .as_ref() + .expect("citus split must exist") + .start_offset(), + CdcSourceType::Mongodb => &self + .mongodb_split + .as_ref() + .expect("mongodb split must exist") + .start_offset(), + CdcSourceType::Unspecified => { + { + ::core::panicking::panic_fmt(format_args!( + "internal error: entered unreachable code: {0}", + format_args!("invalid debezium split") + )); + }; + } + } + } + pub fn snapshot_done(&self) -> bool { + match T::source_type() { + CdcSourceType::Mysql => self + .mysql_split + .as_ref() + .expect("mysql split must exist") + .snapshot_done(), + CdcSourceType::Postgres => self + .postgres_split + .as_ref() + .expect("postgres split must exist") + .snapshot_done(), + CdcSourceType::Citus => self + .citus_split + .as_ref() + .expect("citus split must exist") + .snapshot_done(), + CdcSourceType::Mongodb => self + .mongodb_split + .as_ref() + .expect("mongodb split must exist") + .snapshot_done(), + CdcSourceType::Unspecified => { + { + ::core::panicking::panic_fmt(format_args!( + "internal error: entered unreachable code: {0}", + format_args!("invalid debezium split") + )); + }; + } + } + } + pub fn server_addr(&self) -> Option { + match T::source_type() { + CdcSourceType::Mysql | CdcSourceType::Postgres | CdcSourceType::Mongodb => None, + CdcSourceType::Citus => self + .citus_split + .as_ref() + .expect("split must exist") + .server_addr + .clone(), + CdcSourceType::Unspecified => { + { + ::core::panicking::panic_fmt(format_args!( + "internal error: entered unreachable code: {0}", + format_args!("invalid debezium split") + )); + }; + } + } + } + } +} From 3798597a89041893aca5c748d303d114798bcbf5 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Fri, 2 Feb 2024 17:05:14 +0800 Subject: [PATCH 04/23] fix sql parser --- .../source/SourceValidateHandler.java | 1 + .../source/common/MongoDbValidator.java | 10 +++------- .../connector/source/MongoDbSourceTest.java | 1 - src/connector/src/source/cdc/split.rs | 20 +++++-------------- src/frontend/src/handler/create_source.rs | 5 ++++- src/sqlparser/src/ast/statement.rs | 11 ++++++++++ 6 files changed, 24 insertions(+), 24 deletions(-) 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 ec1cdd5feadb1..a880088f52e5e 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 @@ -136,6 +136,7 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re case MONGODB: ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_URL); ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_COLLECTION_NAME); + // TODO: validate mongodb connectivity and replica set config break; default: LOG.warn("Unknown source type"); diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java index 2b9ba316c6c4f..6e2374fe337bd 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java @@ -14,19 +14,15 @@ package com.risingwave.connector.source.common; +// TODO public class MongoDbValidator extends DatabaseValidator { @Override void validateDbConfig() {} @Override - void validateUserPrivilege() { - // TBD - - } + void validateUserPrivilege() {} @Override - void validateTable() { - // do nothing - } + void validateTable() {} } diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java index ae996ee5078cb..d5e81de97d89c 100644 --- a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java +++ b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java @@ -83,7 +83,6 @@ public void testSnapshotLoad() throws Exception { List messages = eventStream.next().getEventsList(); for (ConnectorServiceProto.CdcMessage msg : messages) { - System.out.println("recv msg: " + msg.getPayload()); if (!msg.getPayload().isBlank()) { count++; } diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 7e1d274a6c314..4fff7a5cc18f3 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -117,7 +117,7 @@ impl CdcSplitTrait for MySqlCdcSplit { } impl PostgresCdcSplit { - pub fn new(split_id: u32, start_offset: Option) -> Self { + pub fn new(split_id: u32, start_offset: Option, server_addr: Option) -> Self { let split = CdcSplitBase { split_id, start_offset, @@ -125,19 +125,9 @@ impl PostgresCdcSplit { }; Self { inner: split, - server_addr: None, + server_addr, } } - - pub fn new_with_server_addr( - split_id: u32, - start_offset: Option, - server_addr: Option, - ) -> Self { - let mut result = Self::new(split_id, start_offset); - result.server_addr = server_addr; - result - } } impl CdcSplitTrait for PostgresCdcSplit { @@ -216,6 +206,7 @@ impl CdcSplitTrait for MongoDbCdcSplit { } } +/// We use this struct to wrap the specific split, which act as an interface to other modules #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] pub struct DebeziumCdcSplit { pub mysql_split: Option, @@ -282,12 +273,11 @@ impl DebeziumCdcSplit { ret.mysql_split = Some(split); } CdcSourceType::Postgres => { - let split = PostgresCdcSplit::new(split_id, start_offset); + let split = PostgresCdcSplit::new(split_id, start_offset, None); ret.postgres_split = Some(split); } CdcSourceType::Citus => { - let split = - PostgresCdcSplit::new_with_server_addr(split_id, start_offset, server_addr); + let split = PostgresCdcSplit::new(split_id, start_offset, server_addr); ret.citus_split = Some(split); } CdcSourceType::Mongodb => { diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index c0d3248b5af8a..158349260457d 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -37,7 +37,7 @@ use risingwave_connector::schema::schema_registry::{ use risingwave_connector::source::cdc::external::CdcTableType; use risingwave_connector::source::cdc::{ CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CITUS_CDC_CONNECTOR, - MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, + MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, MONGODB_CDC_CONNECTOR, }; use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; @@ -933,6 +933,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Debezium => vec![Encode::Json], ), + MONGODB_CDC_CONNECTOR => hashmap!( + Format::DebeziumMongo => vec![Encode::Json], + ), NATS_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Json], ), diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 5624a74c621ed..dd00705843ef2 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -216,9 +216,12 @@ impl Parser { if connector.contains("-cdc") { let expected = if cdc_source_job { ConnectorSchema::plain_json() + } else if connector.contains("mongodb") { + ConnectorSchema::debezium_mongo_json() } else { ConnectorSchema::debezium_json() }; + if self.peek_source_schema_format() { let schema = parse_source_schema(self)?.into_v2(); if schema != expected { @@ -293,6 +296,14 @@ impl ConnectorSchema { } } + pub const fn debezium_mongo_json() -> Self { + ConnectorSchema { + format: Format::DebeziumMongo, + row_encode: Encode::Json, + row_options: Vec::new(), + } + } + /// Create a new source schema with `Native` format and encoding. pub const fn native() -> Self { ConnectorSchema { From d62252cc58f5b99c4a3c44d0e0a9c1862d2fde55 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 3 Feb 2024 13:58:29 +0800 Subject: [PATCH 05/23] fix validate --- .../source/SourceValidateHandler.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) 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 a880088f52e5e..bef55dd700da5 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 @@ -62,33 +62,35 @@ public static ConnectorServiceProto.ValidateSourceResponse validateResponse(Stri .build(); } - public static void ensurePropNotBlank(Map props, String name) { + private static void ensurePropNotBlank(Map props, String name) { if (StringUtils.isBlank(props.get(name))) { throw ValidatorUtils.invalidArgument( String.format("'%s' not found, please check the WITH properties", name)); } } - public static void validateSource(ConnectorServiceProto.ValidateSourceRequest request) - throws Exception { - var props = request.getPropertiesMap(); - + static void ensureRequiredProps(Map props, boolean isMultiTableShared) { ensurePropNotBlank(props, DbzConnectorConfig.HOST); ensurePropNotBlank(props, DbzConnectorConfig.PORT); ensurePropNotBlank(props, DbzConnectorConfig.DB_NAME); ensurePropNotBlank(props, DbzConnectorConfig.USER); ensurePropNotBlank(props, DbzConnectorConfig.PASSWORD); - - var commonParam = request.getCommonParam(); - boolean isMultiTableShared = commonParam.getIsMultiTableShared(); // ensure table name is passed by user in non-sharing mode if (!isMultiTableShared) { ensurePropNotBlank(props, DbzConnectorConfig.TABLE_NAME); } + } + + public static void validateSource(ConnectorServiceProto.ValidateSourceRequest request) + throws Exception { + var props = request.getPropertiesMap(); + var commonParam = request.getCommonParam(); + boolean isMultiTableShared = commonParam.getIsMultiTableShared(); TableSchema tableSchema = TableSchema.fromProto(request.getTableSchema()); switch (request.getSourceType()) { case POSTGRES: + ensureRequiredProps(props, isMultiTableShared); ensurePropNotBlank(props, DbzConnectorConfig.PG_SCHEMA_NAME); ensurePropNotBlank(props, DbzConnectorConfig.PG_SLOT_NAME); ensurePropNotBlank(props, DbzConnectorConfig.PG_PUB_NAME); @@ -100,6 +102,7 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re break; case CITUS: + ensureRequiredProps(props, isMultiTableShared); ensurePropNotBlank(props, DbzConnectorConfig.TABLE_NAME); ensurePropNotBlank(props, DbzConnectorConfig.PG_SCHEMA_NAME); try (var coordinatorValidator = new CitusValidator(props, tableSchema)) { @@ -128,6 +131,7 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re break; case MYSQL: + ensureRequiredProps(props, isMultiTableShared); ensurePropNotBlank(props, DbzConnectorConfig.MYSQL_SERVER_ID); try (var validator = new MySqlValidator(props, tableSchema)) { validator.validateAll(isMultiTableShared); From b4709b6ed9eb32ec10182bd0d9e1bc0edc7938d8 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 3 Feb 2024 15:03:27 +0800 Subject: [PATCH 06/23] minor --- java/connector-node/risingwave-source-cdc/pom.xml | 4 ++++ src/connector/src/parser/unified/debezium.rs | 5 ++++- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-source-cdc/pom.xml b/java/connector-node/risingwave-source-cdc/pom.xml index 683f472a65b14..1b55e2f277964 100644 --- a/java/connector-node/risingwave-source-cdc/pom.xml +++ b/java/connector-node/risingwave-source-cdc/pom.xml @@ -43,6 +43,10 @@ io.debezium debezium-connector-mysql + + io.debezium + debezium-connector-mongodb + diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 7291b1b359735..bbb916c829a80 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -221,7 +221,10 @@ where &bson_doc.take(), )?) } else { - unreachable!("the result of access must match the type_expected") + unreachable!( + "access result must match the type_expected. path: {:?}, payload: {:?}, type_expected: {:?}", + path, payload, type_expected + ) } } ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), From cc777ae559af29efc5d130f74d41417941dbf18a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 3 Feb 2024 15:14:54 +0800 Subject: [PATCH 07/23] clean code --- .../source/common/MongoDbValidator.java | 28 - src/connector/src/source/base.rs | 3 +- .../src/source/cdc/enumerator/mod.rs | 2 +- src/connector/src/split_expand.rs | 1913 ----------------- src/frontend/src/handler/create_source.rs | 2 +- 5 files changed, 4 insertions(+), 1944 deletions(-) delete mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java delete mode 100644 src/connector/src/split_expand.rs diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java deleted file mode 100644 index 6e2374fe337bd..0000000000000 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java +++ /dev/null @@ -1,28 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package com.risingwave.connector.source.common; - -// TODO -public class MongoDbValidator extends DatabaseValidator { - - @Override - void validateDbConfig() {} - - @Override - void validateUserPrivilege() {} - - @Override - void validateTable() {} -} diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 335667314fa60..9cf496fa01413 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -674,7 +674,8 @@ mod tests { #[test] fn test_cdc_split_state() -> Result<()> { let offset_str = "{\"sourcePartition\":{\"server\":\"RW_CDC_mydb.products\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1670407377,\"file\":\"binlog.000001\",\"pos\":98587,\"row\":2,\"server_id\":1,\"event\":2}}"; - let split = DebeziumCdcSplit::::new(1001, Some(offset_str.to_string()), None); + let split = + DebeziumCdcSplit::::new(1001, Some(offset_str.to_string()), None); let split_impl = SplitImpl::MysqlCdc(split); let encoded_split = split_impl.encode_to_bytes(); let restored_split_impl = SplitImpl::restore_from_bytes(encoded_split.as_ref())?; diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 0033d0b44d42c..cec88c6948aaa 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -181,7 +181,7 @@ impl ListCdcSplits for DebeziumSplitEnumerator { vec![DebeziumCdcSplit::::new( self.source_id, None, - None + None, )] } } diff --git a/src/connector/src/split_expand.rs b/src/connector/src/split_expand.rs deleted file mode 100644 index 0b9daacad6016..0000000000000 --- a/src/connector/src/split_expand.rs +++ /dev/null @@ -1,1913 +0,0 @@ -pub mod split { - use std::marker::PhantomData; - use anyhow::anyhow; - use risingwave_common::types::JsonbVal; - use serde::{Deserialize, Serialize}; - use crate::source::cdc::external::DebeziumOffset; - use crate::source::cdc::{CdcSourceType, CdcSourceTypeTrait}; - use crate::source::{SplitId, SplitMetaData}; - trait CdcSplitTrait: Send + Sync { - fn split_id(&self) -> u32; - fn start_offset(&self) -> &Option; - fn snapshot_done(&self) -> bool; - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; - } - /// The base states of a CDC split, which will be persisted to checkpoint. - /// CDC source only has single split, so we use the `source_id` to identify the split. - pub struct CdcSplitBase { - pub split_id: u32, - pub start_offset: Option, - pub snapshot_done: bool, - } - #[automatically_derived] - impl ::core::clone::Clone for CdcSplitBase { - #[inline] - fn clone(&self) -> CdcSplitBase { - CdcSplitBase { - split_id: ::core::clone::Clone::clone(&self.split_id), - start_offset: ::core::clone::Clone::clone(&self.start_offset), - snapshot_done: ::core::clone::Clone::clone(&self.snapshot_done), - } - } - } - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl _serde::Serialize for CdcSplitBase { - fn serialize<__S>( - &self, - __serializer: __S, - ) -> _serde::__private::Result<__S::Ok, __S::Error> - where - __S: _serde::Serializer, - { - let mut __serde_state = _serde::Serializer::serialize_struct( - __serializer, - "CdcSplitBase", - false as usize + 1 + 1 + 1, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "split_id", - &self.split_id, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "start_offset", - &self.start_offset, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "snapshot_done", - &self.snapshot_done, - )?; - _serde::ser::SerializeStruct::end(__serde_state) - } - } - }; - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl<'de> _serde::Deserialize<'de> for CdcSplitBase { - fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - #[allow(non_camel_case_types)] - #[doc(hidden)] - enum __Field { - __field0, - __field1, - __field2, - __ignore, - } - #[doc(hidden)] - struct __FieldVisitor; - impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { - type Value = __Field; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "field identifier") - } - fn visit_u64<__E>( - self, - __value: u64, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - 0u64 => _serde::__private::Ok(__Field::__field0), - 1u64 => _serde::__private::Ok(__Field::__field1), - 2u64 => _serde::__private::Ok(__Field::__field2), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_str<__E>( - self, - __value: &str, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - "split_id" => _serde::__private::Ok(__Field::__field0), - "start_offset" => _serde::__private::Ok(__Field::__field1), - "snapshot_done" => _serde::__private::Ok(__Field::__field2), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_bytes<__E>( - self, - __value: &[u8], - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - b"split_id" => _serde::__private::Ok(__Field::__field0), - b"start_offset" => _serde::__private::Ok(__Field::__field1), - b"snapshot_done" => _serde::__private::Ok(__Field::__field2), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - } - impl<'de> _serde::Deserialize<'de> for __Field { - #[inline] - fn deserialize<__D>( - __deserializer: __D, - ) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) - } - } - #[doc(hidden)] - struct __Visitor<'de> { - marker: _serde::__private::PhantomData, - lifetime: _serde::__private::PhantomData<&'de ()>, - } - impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { - type Value = CdcSplitBase; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "struct CdcSplitBase") - } - #[inline] - fn visit_seq<__A>( - self, - mut __seq: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::SeqAccess<'de>, - { - let __field0 = match _serde::de::SeqAccess::next_element::(&mut __seq)? - { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 0usize, - &"struct CdcSplitBase with 3 elements", - )) - } - }; - let __field1 = match _serde::de::SeqAccess::next_element::>( - &mut __seq, - )? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 1usize, - &"struct CdcSplitBase with 3 elements", - )) - } - }; - let __field2 = - match _serde::de::SeqAccess::next_element::(&mut __seq)? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err( - _serde::de::Error::invalid_length( - 2usize, - &"struct CdcSplitBase with 3 elements", - ), - ) - } - }; - _serde::__private::Ok(CdcSplitBase { - split_id: __field0, - start_offset: __field1, - snapshot_done: __field2, - }) - } - #[inline] - fn visit_map<__A>( - self, - mut __map: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::MapAccess<'de>, - { - let mut __field0: _serde::__private::Option = _serde::__private::None; - let mut __field1: _serde::__private::Option> = - _serde::__private::None; - let mut __field2: _serde::__private::Option = _serde::__private::None; - while let _serde::__private::Some(__key) = - _serde::de::MapAccess::next_key::<__Field>(&mut __map)? - { - match __key { - __Field::__field0 => { - if _serde::__private::Option::is_some(&__field0) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "split_id", - ), - ); - } - __field0 = _serde::__private::Some( - _serde::de::MapAccess::next_value::(&mut __map)?, - ); - } - __Field::__field1 => { - if _serde::__private::Option::is_some(&__field1) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "start_offset", - ), - ); - } - __field1 = _serde::__private::Some( - _serde::de::MapAccess::next_value::>( - &mut __map, - )?, - ); - } - __Field::__field2 => { - if _serde::__private::Option::is_some(&__field2) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "snapshot_done", - ), - ); - } - __field2 = _serde::__private::Some( - _serde::de::MapAccess::next_value::(&mut __map)?, - ); - } - _ => { - let _ = _serde::de::MapAccess::next_value::< - _serde::de::IgnoredAny, - >(&mut __map)?; - } - } - } - let __field0 = match __field0 { - _serde::__private::Some(__field0) => __field0, - _serde::__private::None => { - _serde::__private::de::missing_field("split_id")? - } - }; - let __field1 = match __field1 { - _serde::__private::Some(__field1) => __field1, - _serde::__private::None => { - _serde::__private::de::missing_field("start_offset")? - } - }; - let __field2 = match __field2 { - _serde::__private::Some(__field2) => __field2, - _serde::__private::None => { - _serde::__private::de::missing_field("snapshot_done")? - } - }; - _serde::__private::Ok(CdcSplitBase { - split_id: __field0, - start_offset: __field1, - snapshot_done: __field2, - }) - } - } - #[doc(hidden)] - const FIELDS: &'static [&'static str] = - &["split_id", "start_offset", "snapshot_done"]; - _serde::Deserializer::deserialize_struct( - __deserializer, - "CdcSplitBase", - FIELDS, - __Visitor { - marker: _serde::__private::PhantomData::, - lifetime: _serde::__private::PhantomData, - }, - ) - } - } - }; - #[automatically_derived] - impl ::core::fmt::Debug for CdcSplitBase { - #[inline] - fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { - ::core::fmt::Formatter::debug_struct_field3_finish( - f, - "CdcSplitBase", - "split_id", - &self.split_id, - "start_offset", - &self.start_offset, - "snapshot_done", - &&self.snapshot_done, - ) - } - } - #[automatically_derived] - impl ::core::marker::StructuralPartialEq for CdcSplitBase {} - #[automatically_derived] - impl ::core::cmp::PartialEq for CdcSplitBase { - #[inline] - fn eq(&self, other: &CdcSplitBase) -> bool { - self.split_id == other.split_id - && self.start_offset == other.start_offset - && self.snapshot_done == other.snapshot_done - } - } - #[automatically_derived] - impl ::core::hash::Hash for CdcSplitBase { - #[inline] - fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { - ::core::hash::Hash::hash(&self.split_id, state); - ::core::hash::Hash::hash(&self.start_offset, state); - ::core::hash::Hash::hash(&self.snapshot_done, state) - } - } - impl CdcSplitBase { - pub fn new(split_id: u32, start_offset: Option) -> Self { - Self { - split_id, - start_offset, - snapshot_done: false, - } - } - } - pub struct MySqlCdcSplit { - pub inner: CdcSplitBase, - } - #[automatically_derived] - impl ::core::clone::Clone for MySqlCdcSplit { - #[inline] - fn clone(&self) -> MySqlCdcSplit { - MySqlCdcSplit { - inner: ::core::clone::Clone::clone(&self.inner), - } - } - } - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl _serde::Serialize for MySqlCdcSplit { - fn serialize<__S>( - &self, - __serializer: __S, - ) -> _serde::__private::Result<__S::Ok, __S::Error> - where - __S: _serde::Serializer, - { - let mut __serde_state = _serde::Serializer::serialize_struct( - __serializer, - "MySqlCdcSplit", - false as usize + 1, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "inner", - &self.inner, - )?; - _serde::ser::SerializeStruct::end(__serde_state) - } - } - }; - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl<'de> _serde::Deserialize<'de> for MySqlCdcSplit { - fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - #[allow(non_camel_case_types)] - #[doc(hidden)] - enum __Field { - __field0, - __ignore, - } - #[doc(hidden)] - struct __FieldVisitor; - impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { - type Value = __Field; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "field identifier") - } - fn visit_u64<__E>( - self, - __value: u64, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - 0u64 => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_str<__E>( - self, - __value: &str, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - "inner" => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_bytes<__E>( - self, - __value: &[u8], - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - b"inner" => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - } - impl<'de> _serde::Deserialize<'de> for __Field { - #[inline] - fn deserialize<__D>( - __deserializer: __D, - ) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) - } - } - #[doc(hidden)] - struct __Visitor<'de> { - marker: _serde::__private::PhantomData, - lifetime: _serde::__private::PhantomData<&'de ()>, - } - impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { - type Value = MySqlCdcSplit; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "struct MySqlCdcSplit") - } - #[inline] - fn visit_seq<__A>( - self, - mut __seq: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::SeqAccess<'de>, - { - let __field0 = match _serde::de::SeqAccess::next_element::( - &mut __seq, - )? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 0usize, - &"struct MySqlCdcSplit with 1 element", - )) - } - }; - _serde::__private::Ok(MySqlCdcSplit { inner: __field0 }) - } - #[inline] - fn visit_map<__A>( - self, - mut __map: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::MapAccess<'de>, - { - let mut __field0: _serde::__private::Option = - _serde::__private::None; - while let _serde::__private::Some(__key) = - _serde::de::MapAccess::next_key::<__Field>(&mut __map)? - { - match __key { - __Field::__field0 => { - if _serde::__private::Option::is_some(&__field0) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "inner", - ), - ); - } - __field0 = _serde::__private::Some( - _serde::de::MapAccess::next_value::( - &mut __map, - )?, - ); - } - _ => { - let _ = _serde::de::MapAccess::next_value::< - _serde::de::IgnoredAny, - >(&mut __map)?; - } - } - } - let __field0 = match __field0 { - _serde::__private::Some(__field0) => __field0, - _serde::__private::None => { - _serde::__private::de::missing_field("inner")? - } - }; - _serde::__private::Ok(MySqlCdcSplit { inner: __field0 }) - } - } - #[doc(hidden)] - const FIELDS: &'static [&'static str] = &["inner"]; - _serde::Deserializer::deserialize_struct( - __deserializer, - "MySqlCdcSplit", - FIELDS, - __Visitor { - marker: _serde::__private::PhantomData::, - lifetime: _serde::__private::PhantomData, - }, - ) - } - } - }; - #[automatically_derived] - impl ::core::fmt::Debug for MySqlCdcSplit { - #[inline] - fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { - ::core::fmt::Formatter::debug_struct_field1_finish( - f, - "MySqlCdcSplit", - "inner", - &&self.inner, - ) - } - } - #[automatically_derived] - impl ::core::marker::StructuralPartialEq for MySqlCdcSplit {} - #[automatically_derived] - impl ::core::cmp::PartialEq for MySqlCdcSplit { - #[inline] - fn eq(&self, other: &MySqlCdcSplit) -> bool { - self.inner == other.inner - } - } - #[automatically_derived] - impl ::core::hash::Hash for MySqlCdcSplit { - #[inline] - fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { - ::core::hash::Hash::hash(&self.inner, state) - } - } - pub struct PostgresCdcSplit { - pub inner: CdcSplitBase, - pub server_addr: Option, - } - #[automatically_derived] - impl ::core::clone::Clone for PostgresCdcSplit { - #[inline] - fn clone(&self) -> PostgresCdcSplit { - PostgresCdcSplit { - inner: ::core::clone::Clone::clone(&self.inner), - server_addr: ::core::clone::Clone::clone(&self.server_addr), - } - } - } - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl _serde::Serialize for PostgresCdcSplit { - fn serialize<__S>( - &self, - __serializer: __S, - ) -> _serde::__private::Result<__S::Ok, __S::Error> - where - __S: _serde::Serializer, - { - let mut __serde_state = _serde::Serializer::serialize_struct( - __serializer, - "PostgresCdcSplit", - false as usize + 1 + 1, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "inner", - &self.inner, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "server_addr", - &self.server_addr, - )?; - _serde::ser::SerializeStruct::end(__serde_state) - } - } - }; - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl<'de> _serde::Deserialize<'de> for PostgresCdcSplit { - fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - #[allow(non_camel_case_types)] - #[doc(hidden)] - enum __Field { - __field0, - __field1, - __ignore, - } - #[doc(hidden)] - struct __FieldVisitor; - impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { - type Value = __Field; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "field identifier") - } - fn visit_u64<__E>( - self, - __value: u64, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - 0u64 => _serde::__private::Ok(__Field::__field0), - 1u64 => _serde::__private::Ok(__Field::__field1), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_str<__E>( - self, - __value: &str, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - "inner" => _serde::__private::Ok(__Field::__field0), - "server_addr" => _serde::__private::Ok(__Field::__field1), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_bytes<__E>( - self, - __value: &[u8], - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - b"inner" => _serde::__private::Ok(__Field::__field0), - b"server_addr" => _serde::__private::Ok(__Field::__field1), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - } - impl<'de> _serde::Deserialize<'de> for __Field { - #[inline] - fn deserialize<__D>( - __deserializer: __D, - ) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) - } - } - #[doc(hidden)] - struct __Visitor<'de> { - marker: _serde::__private::PhantomData, - lifetime: _serde::__private::PhantomData<&'de ()>, - } - impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { - type Value = PostgresCdcSplit; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str( - __formatter, - "struct PostgresCdcSplit", - ) - } - #[inline] - fn visit_seq<__A>( - self, - mut __seq: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::SeqAccess<'de>, - { - let __field0 = match _serde::de::SeqAccess::next_element::( - &mut __seq, - )? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 0usize, - &"struct PostgresCdcSplit with 2 elements", - )) - } - }; - let __field1 = match _serde::de::SeqAccess::next_element::>( - &mut __seq, - )? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 1usize, - &"struct PostgresCdcSplit with 2 elements", - )) - } - }; - _serde::__private::Ok(PostgresCdcSplit { - inner: __field0, - server_addr: __field1, - }) - } - #[inline] - fn visit_map<__A>( - self, - mut __map: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::MapAccess<'de>, - { - let mut __field0: _serde::__private::Option = - _serde::__private::None; - let mut __field1: _serde::__private::Option> = - _serde::__private::None; - while let _serde::__private::Some(__key) = - _serde::de::MapAccess::next_key::<__Field>(&mut __map)? - { - match __key { - __Field::__field0 => { - if _serde::__private::Option::is_some(&__field0) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "inner", - ), - ); - } - __field0 = _serde::__private::Some( - _serde::de::MapAccess::next_value::( - &mut __map, - )?, - ); - } - __Field::__field1 => { - if _serde::__private::Option::is_some(&__field1) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "server_addr", - ), - ); - } - __field1 = _serde::__private::Some( - _serde::de::MapAccess::next_value::>( - &mut __map, - )?, - ); - } - _ => { - let _ = _serde::de::MapAccess::next_value::< - _serde::de::IgnoredAny, - >(&mut __map)?; - } - } - } - let __field0 = match __field0 { - _serde::__private::Some(__field0) => __field0, - _serde::__private::None => { - _serde::__private::de::missing_field("inner")? - } - }; - let __field1 = match __field1 { - _serde::__private::Some(__field1) => __field1, - _serde::__private::None => { - _serde::__private::de::missing_field("server_addr")? - } - }; - _serde::__private::Ok(PostgresCdcSplit { - inner: __field0, - server_addr: __field1, - }) - } - } - #[doc(hidden)] - const FIELDS: &'static [&'static str] = &["inner", "server_addr"]; - _serde::Deserializer::deserialize_struct( - __deserializer, - "PostgresCdcSplit", - FIELDS, - __Visitor { - marker: _serde::__private::PhantomData::, - lifetime: _serde::__private::PhantomData, - }, - ) - } - } - }; - #[automatically_derived] - impl ::core::fmt::Debug for PostgresCdcSplit { - #[inline] - fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { - ::core::fmt::Formatter::debug_struct_field2_finish( - f, - "PostgresCdcSplit", - "inner", - &self.inner, - "server_addr", - &&self.server_addr, - ) - } - } - #[automatically_derived] - impl ::core::marker::StructuralPartialEq for PostgresCdcSplit {} - #[automatically_derived] - impl ::core::cmp::PartialEq for PostgresCdcSplit { - #[inline] - fn eq(&self, other: &PostgresCdcSplit) -> bool { - self.inner == other.inner && self.server_addr == other.server_addr - } - } - #[automatically_derived] - impl ::core::hash::Hash for PostgresCdcSplit { - #[inline] - fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { - ::core::hash::Hash::hash(&self.inner, state); - ::core::hash::Hash::hash(&self.server_addr, state) - } - } - pub struct MongoDbCdcSplit { - pub inner: CdcSplitBase, - } - #[automatically_derived] - impl ::core::clone::Clone for MongoDbCdcSplit { - #[inline] - fn clone(&self) -> MongoDbCdcSplit { - MongoDbCdcSplit { - inner: ::core::clone::Clone::clone(&self.inner), - } - } - } - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl _serde::Serialize for MongoDbCdcSplit { - fn serialize<__S>( - &self, - __serializer: __S, - ) -> _serde::__private::Result<__S::Ok, __S::Error> - where - __S: _serde::Serializer, - { - let mut __serde_state = _serde::Serializer::serialize_struct( - __serializer, - "MongoDbCdcSplit", - false as usize + 1, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "inner", - &self.inner, - )?; - _serde::ser::SerializeStruct::end(__serde_state) - } - } - }; - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl<'de> _serde::Deserialize<'de> for MongoDbCdcSplit { - fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - #[allow(non_camel_case_types)] - #[doc(hidden)] - enum __Field { - __field0, - __ignore, - } - #[doc(hidden)] - struct __FieldVisitor; - impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { - type Value = __Field; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "field identifier") - } - fn visit_u64<__E>( - self, - __value: u64, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - 0u64 => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_str<__E>( - self, - __value: &str, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - "inner" => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_bytes<__E>( - self, - __value: &[u8], - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - b"inner" => _serde::__private::Ok(__Field::__field0), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - } - impl<'de> _serde::Deserialize<'de> for __Field { - #[inline] - fn deserialize<__D>( - __deserializer: __D, - ) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) - } - } - #[doc(hidden)] - struct __Visitor<'de> { - marker: _serde::__private::PhantomData, - lifetime: _serde::__private::PhantomData<&'de ()>, - } - impl<'de> _serde::de::Visitor<'de> for __Visitor<'de> { - type Value = MongoDbCdcSplit; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str( - __formatter, - "struct MongoDbCdcSplit", - ) - } - #[inline] - fn visit_seq<__A>( - self, - mut __seq: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::SeqAccess<'de>, - { - let __field0 = match _serde::de::SeqAccess::next_element::( - &mut __seq, - )? { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 0usize, - &"struct MongoDbCdcSplit with 1 element", - )) - } - }; - _serde::__private::Ok(MongoDbCdcSplit { inner: __field0 }) - } - #[inline] - fn visit_map<__A>( - self, - mut __map: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::MapAccess<'de>, - { - let mut __field0: _serde::__private::Option = - _serde::__private::None; - while let _serde::__private::Some(__key) = - _serde::de::MapAccess::next_key::<__Field>(&mut __map)? - { - match __key { - __Field::__field0 => { - if _serde::__private::Option::is_some(&__field0) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "inner", - ), - ); - } - __field0 = _serde::__private::Some( - _serde::de::MapAccess::next_value::( - &mut __map, - )?, - ); - } - _ => { - let _ = _serde::de::MapAccess::next_value::< - _serde::de::IgnoredAny, - >(&mut __map)?; - } - } - } - let __field0 = match __field0 { - _serde::__private::Some(__field0) => __field0, - _serde::__private::None => { - _serde::__private::de::missing_field("inner")? - } - }; - _serde::__private::Ok(MongoDbCdcSplit { inner: __field0 }) - } - } - #[doc(hidden)] - const FIELDS: &'static [&'static str] = &["inner"]; - _serde::Deserializer::deserialize_struct( - __deserializer, - "MongoDbCdcSplit", - FIELDS, - __Visitor { - marker: _serde::__private::PhantomData::, - lifetime: _serde::__private::PhantomData, - }, - ) - } - } - }; - #[automatically_derived] - impl ::core::fmt::Debug for MongoDbCdcSplit { - #[inline] - fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { - ::core::fmt::Formatter::debug_struct_field1_finish( - f, - "MongoDbCdcSplit", - "inner", - &&self.inner, - ) - } - } - #[automatically_derived] - impl ::core::marker::StructuralPartialEq for MongoDbCdcSplit {} - #[automatically_derived] - impl ::core::cmp::PartialEq for MongoDbCdcSplit { - #[inline] - fn eq(&self, other: &MongoDbCdcSplit) -> bool { - self.inner == other.inner - } - } - #[automatically_derived] - impl ::core::hash::Hash for MongoDbCdcSplit { - #[inline] - fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { - ::core::hash::Hash::hash(&self.inner, state) - } - } - impl MySqlCdcSplit { - pub fn new(split_id: u32, start_offset: Option) -> Self { - let split = CdcSplitBase { - split_id, - start_offset, - snapshot_done: false, - }; - Self { inner: split } - } - } - impl CdcSplitTrait for MySqlCdcSplit { - fn split_id(&self) -> u32 { - self.inner.split_id - } - fn start_offset(&self) -> &Option { - &self.inner.start_offset - } - fn snapshot_done(&self) -> bool { - self.inner.snapshot_done - } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - let mut snapshot_done = self.inner.snapshot_done; - if !snapshot_done { - let dbz_offset: DebeziumOffset = - serde_json::from_str(&start_offset).map_err(|e| { - ::anyhow::Error::msg({ - let res = ::alloc::fmt::format(format_args!( - "invalid mysql offset: {0}, error: {1}, split: {2}", - start_offset, e, self.inner.split_id - )); - res - }) - })?; - if !dbz_offset.is_heartbeat { - snapshot_done = match dbz_offset.source_offset.snapshot { - Some(val) => !val, - None => true, - }; - } - } - self.inner.start_offset = Some(start_offset); - self.inner.snapshot_done = snapshot_done; - Ok(()) - } - } - impl PostgresCdcSplit { - pub fn new(split_id: u32, start_offset: Option) -> Self { - let split = CdcSplitBase { - split_id, - start_offset, - snapshot_done: false, - }; - Self { - inner: split, - server_addr: None, - } - } - pub fn new_with_server_addr( - split_id: u32, - start_offset: Option, - server_addr: Option, - ) -> Self { - let mut result = Self::new(split_id, start_offset); - result.server_addr = server_addr; - result - } - } - impl CdcSplitTrait for PostgresCdcSplit { - fn split_id(&self) -> u32 { - self.inner.split_id - } - fn start_offset(&self) -> &Option { - &self.inner.start_offset - } - fn snapshot_done(&self) -> bool { - self.inner.snapshot_done - } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - let mut snapshot_done = self.inner.snapshot_done; - if !snapshot_done { - let dbz_offset: DebeziumOffset = - serde_json::from_str(&start_offset).map_err(|e| { - ::anyhow::Error::msg({ - let res = ::alloc::fmt::format(format_args!( - "invalid postgres offset: {0}, error: {1}, split: {2}", - start_offset, e, self.inner.split_id - )); - res - }) - })?; - if !dbz_offset.is_heartbeat { - snapshot_done = dbz_offset - .source_offset - .last_snapshot_record - .unwrap_or(false); - } - } - self.inner.start_offset = Some(start_offset); - self.inner.snapshot_done = snapshot_done; - Ok(()) - } - } - impl MongoDbCdcSplit { - pub fn new(split_id: u32, start_offset: Option) -> Self { - let split = CdcSplitBase { - split_id, - start_offset, - snapshot_done: false, - }; - Self { inner: split } - } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.inner.start_offset = Some(start_offset); - Ok(()) - } - } - impl CdcSplitTrait for MongoDbCdcSplit { - fn split_id(&self) -> u32 { - self.inner.split_id - } - fn start_offset(&self) -> &Option { - &self.inner.start_offset - } - fn snapshot_done(&self) -> bool { - self.inner.snapshot_done - } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { - Ok(()) - } - } - pub struct DebeziumCdcSplit { - pub mysql_split: Option, - #[serde(rename = "pg_split")] - pub postgres_split: Option, - pub citus_split: Option, - pub mongodb_split: Option, - #[serde(skip)] - pub _phantom: PhantomData, - } - #[automatically_derived] - impl ::core::clone::Clone for DebeziumCdcSplit { - #[inline] - fn clone(&self) -> DebeziumCdcSplit { - DebeziumCdcSplit { - mysql_split: ::core::clone::Clone::clone(&self.mysql_split), - postgres_split: ::core::clone::Clone::clone(&self.postgres_split), - citus_split: ::core::clone::Clone::clone(&self.citus_split), - mongodb_split: ::core::clone::Clone::clone(&self.mongodb_split), - _phantom: ::core::clone::Clone::clone(&self._phantom), - } - } - } - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl _serde::Serialize for DebeziumCdcSplit { - fn serialize<__S>( - &self, - __serializer: __S, - ) -> _serde::__private::Result<__S::Ok, __S::Error> - where - __S: _serde::Serializer, - { - let mut __serde_state = _serde::Serializer::serialize_struct( - __serializer, - "DebeziumCdcSplit", - false as usize + 1 + 1 + 1 + 1, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "mysql_split", - &self.mysql_split, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "pg_split", - &self.postgres_split, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "citus_split", - &self.citus_split, - )?; - _serde::ser::SerializeStruct::serialize_field( - &mut __serde_state, - "mongodb_split", - &self.mongodb_split, - )?; - _serde::ser::SerializeStruct::end(__serde_state) - } - } - }; - #[doc(hidden)] - #[allow(non_upper_case_globals, unused_attributes, unused_qualifications)] - const _: () = { - #[allow(unused_extern_crates, clippy::useless_attribute)] - extern crate serde as _serde; - #[automatically_derived] - impl<'de, T: CdcSourceTypeTrait> _serde::Deserialize<'de> for DebeziumCdcSplit { - fn deserialize<__D>(__deserializer: __D) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - #[allow(non_camel_case_types)] - #[doc(hidden)] - enum __Field { - __field0, - __field1, - __field2, - __field3, - __ignore, - } - #[doc(hidden)] - struct __FieldVisitor; - impl<'de> _serde::de::Visitor<'de> for __FieldVisitor { - type Value = __Field; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str(__formatter, "field identifier") - } - fn visit_u64<__E>( - self, - __value: u64, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - 0u64 => _serde::__private::Ok(__Field::__field0), - 1u64 => _serde::__private::Ok(__Field::__field1), - 2u64 => _serde::__private::Ok(__Field::__field2), - 3u64 => _serde::__private::Ok(__Field::__field3), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_str<__E>( - self, - __value: &str, - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - "mysql_split" => _serde::__private::Ok(__Field::__field0), - "pg_split" => _serde::__private::Ok(__Field::__field1), - "citus_split" => _serde::__private::Ok(__Field::__field2), - "mongodb_split" => _serde::__private::Ok(__Field::__field3), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - fn visit_bytes<__E>( - self, - __value: &[u8], - ) -> _serde::__private::Result - where - __E: _serde::de::Error, - { - match __value { - b"mysql_split" => _serde::__private::Ok(__Field::__field0), - b"pg_split" => _serde::__private::Ok(__Field::__field1), - b"citus_split" => _serde::__private::Ok(__Field::__field2), - b"mongodb_split" => _serde::__private::Ok(__Field::__field3), - _ => _serde::__private::Ok(__Field::__ignore), - } - } - } - impl<'de> _serde::Deserialize<'de> for __Field { - #[inline] - fn deserialize<__D>( - __deserializer: __D, - ) -> _serde::__private::Result - where - __D: _serde::Deserializer<'de>, - { - _serde::Deserializer::deserialize_identifier(__deserializer, __FieldVisitor) - } - } - #[doc(hidden)] - struct __Visitor<'de, T: CdcSourceTypeTrait> { - marker: _serde::__private::PhantomData>, - lifetime: _serde::__private::PhantomData<&'de ()>, - } - impl<'de, T: CdcSourceTypeTrait> _serde::de::Visitor<'de> for __Visitor<'de, T> { - type Value = DebeziumCdcSplit; - fn expecting( - &self, - __formatter: &mut _serde::__private::Formatter, - ) -> _serde::__private::fmt::Result { - _serde::__private::Formatter::write_str( - __formatter, - "struct DebeziumCdcSplit", - ) - } - #[inline] - fn visit_seq<__A>( - self, - mut __seq: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::SeqAccess<'de>, - { - let __field0 = match _serde::de::SeqAccess::next_element::< - Option, - >(&mut __seq)? - { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 0usize, - &"struct DebeziumCdcSplit with 4 elements", - )) - } - }; - let __field1 = match _serde::de::SeqAccess::next_element::< - Option, - >(&mut __seq)? - { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 1usize, - &"struct DebeziumCdcSplit with 4 elements", - )) - } - }; - let __field2 = match _serde::de::SeqAccess::next_element::< - Option, - >(&mut __seq)? - { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 2usize, - &"struct DebeziumCdcSplit with 4 elements", - )) - } - }; - let __field3 = match _serde::de::SeqAccess::next_element::< - Option, - >(&mut __seq)? - { - _serde::__private::Some(__value) => __value, - _serde::__private::None => { - return _serde::__private::Err(_serde::de::Error::invalid_length( - 3usize, - &"struct DebeziumCdcSplit with 4 elements", - )) - } - }; - let __field4 = _serde::__private::Default::default(); - _serde::__private::Ok(DebeziumCdcSplit { - mysql_split: __field0, - postgres_split: __field1, - citus_split: __field2, - mongodb_split: __field3, - _phantom: __field4, - }) - } - #[inline] - fn visit_map<__A>( - self, - mut __map: __A, - ) -> _serde::__private::Result - where - __A: _serde::de::MapAccess<'de>, - { - let mut __field0: _serde::__private::Option> = - _serde::__private::None; - let mut __field1: _serde::__private::Option> = - _serde::__private::None; - let mut __field2: _serde::__private::Option> = - _serde::__private::None; - let mut __field3: _serde::__private::Option> = - _serde::__private::None; - while let _serde::__private::Some(__key) = - _serde::de::MapAccess::next_key::<__Field>(&mut __map)? - { - match __key { - __Field::__field0 => { - if _serde::__private::Option::is_some(&__field0) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "mysql_split", - ), - ); - } - __field0 = _serde::__private::Some( - _serde::de::MapAccess::next_value::>( - &mut __map, - )?, - ); - } - __Field::__field1 => { - if _serde::__private::Option::is_some(&__field1) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "pg_split", - ), - ); - } - __field1 = _serde::__private::Some( - _serde::de::MapAccess::next_value::< - Option, - >(&mut __map)?, - ); - } - __Field::__field2 => { - if _serde::__private::Option::is_some(&__field2) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "citus_split", - ), - ); - } - __field2 = _serde::__private::Some( - _serde::de::MapAccess::next_value::< - Option, - >(&mut __map)?, - ); - } - __Field::__field3 => { - if _serde::__private::Option::is_some(&__field3) { - return _serde::__private::Err( - <__A::Error as _serde::de::Error>::duplicate_field( - "mongodb_split", - ), - ); - } - __field3 = _serde::__private::Some( - _serde::de::MapAccess::next_value::>( - &mut __map, - )?, - ); - } - _ => { - let _ = _serde::de::MapAccess::next_value::< - _serde::de::IgnoredAny, - >(&mut __map)?; - } - } - } - let __field0 = match __field0 { - _serde::__private::Some(__field0) => __field0, - _serde::__private::None => { - _serde::__private::de::missing_field("mysql_split")? - } - }; - let __field1 = match __field1 { - _serde::__private::Some(__field1) => __field1, - _serde::__private::None => { - _serde::__private::de::missing_field("pg_split")? - } - }; - let __field2 = match __field2 { - _serde::__private::Some(__field2) => __field2, - _serde::__private::None => { - _serde::__private::de::missing_field("citus_split")? - } - }; - let __field3 = match __field3 { - _serde::__private::Some(__field3) => __field3, - _serde::__private::None => { - _serde::__private::de::missing_field("mongodb_split")? - } - }; - _serde::__private::Ok(DebeziumCdcSplit { - mysql_split: __field0, - postgres_split: __field1, - citus_split: __field2, - mongodb_split: __field3, - _phantom: _serde::__private::Default::default(), - }) - } - } - #[doc(hidden)] - const FIELDS: &'static [&'static str] = - &["mysql_split", "pg_split", "citus_split", "mongodb_split"]; - _serde::Deserializer::deserialize_struct( - __deserializer, - "DebeziumCdcSplit", - FIELDS, - __Visitor { - marker: _serde::__private::PhantomData::>, - lifetime: _serde::__private::PhantomData, - }, - ) - } - } - }; - #[automatically_derived] - impl ::core::fmt::Debug for DebeziumCdcSplit { - #[inline] - fn fmt(&self, f: &mut ::core::fmt::Formatter) -> ::core::fmt::Result { - ::core::fmt::Formatter::debug_struct_field5_finish( - f, - "DebeziumCdcSplit", - "mysql_split", - &self.mysql_split, - "postgres_split", - &self.postgres_split, - "citus_split", - &self.citus_split, - "mongodb_split", - &self.mongodb_split, - "_phantom", - &&self._phantom, - ) - } - } - #[automatically_derived] - impl ::core::marker::StructuralPartialEq for DebeziumCdcSplit {} - #[automatically_derived] - impl ::core::cmp::PartialEq - for DebeziumCdcSplit - { - #[inline] - fn eq(&self, other: &DebeziumCdcSplit) -> bool { - self.mysql_split == other.mysql_split - && self.postgres_split == other.postgres_split - && self.citus_split == other.citus_split - && self.mongodb_split == other.mongodb_split - && self._phantom == other._phantom - } - } - #[automatically_derived] - impl ::core::hash::Hash for DebeziumCdcSplit { - #[inline] - fn hash<__H: ::core::hash::Hasher>(&self, state: &mut __H) -> () { - ::core::hash::Hash::hash(&self.mysql_split, state); - ::core::hash::Hash::hash(&self.postgres_split, state); - ::core::hash::Hash::hash(&self.citus_split, state); - ::core::hash::Hash::hash(&self.mongodb_split, state); - ::core::hash::Hash::hash(&self._phantom, state) - } - } - impl SplitMetaData for DebeziumCdcSplit { - fn id(&self) -> SplitId { - { - let res = ::alloc::fmt::format(format_args!("{0}", self.split_id())); - res - } - .into() - } - fn encode_to_json(&self) -> JsonbVal { - serde_json::to_value(self.clone()).unwrap().into() - } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| { - ::anyhow::__private::must_use({ - use ::anyhow::__private::kind::*; - let error = match e { - error => (&error).anyhow_kind().new(error), - }; - error - }) - }) - } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - match T::source_type() { - CdcSourceType::Mysql => self - .mysql_split - .as_mut() - .expect("mysql split must exist") - .update_with_offset(start_offset)?, - CdcSourceType::Postgres => self - .postgres_split - .as_mut() - .expect("postgres split must exist") - .update_with_offset(start_offset)?, - CdcSourceType::Citus => self - .citus_split - .as_mut() - .expect("citus split must exist") - .update_with_offset(start_offset)?, - CdcSourceType::Mongodb => self - .mongodb_split - .as_mut() - .expect("mongodb split must exist") - .update_with_offset(start_offset)?, - CdcSourceType::Unspecified => { - { - ::core::panicking::panic_fmt(format_args!( - "internal error: entered unreachable code: {0}", - format_args!("invalid debezium split") - )); - }; - } - }; - Ok(()) - } - } - impl DebeziumCdcSplit { - pub fn new_mysql(split_id: u32, start_offset: Option) -> Self { - if !match T::source_type() { - CdcSourceType::Mysql => true, - _ => false, - } { - ::core::panicking::panic( - "assertion failed: matches!(T::source_type(), CdcSourceType::Mysql)", - ) - }; - let split = MySqlCdcSplit::new(split_id, start_offset); - Self { - mysql_split: Some(split), - postgres_split: None, - citus_split: None, - mongodb_split: None, - _phantom: PhantomData, - } - } - pub fn new_postgres(split_id: u32, start_offset: Option) -> Self { - if !match T::source_type() { - CdcSourceType::Postgres => true, - _ => false, - } { - ::core::panicking::panic( - "assertion failed: matches!(T::source_type(), CdcSourceType::Postgres)", - ) - }; - let split = PostgresCdcSplit::new(split_id, start_offset); - Self { - mysql_split: None, - postgres_split: Some(split), - citus_split: None, - mongodb_split: None, - _phantom: PhantomData, - } - } - pub fn new_citus( - split_id: u32, - start_offset: Option, - server_addr: Option, - ) -> Self { - if !match T::source_type() { - CdcSourceType::Citus => true, - _ => false, - } { - ::core::panicking::panic( - "assertion failed: matches!(T::source_type(), CdcSourceType::Citus)", - ) - }; - let split = PostgresCdcSplit::new_with_server_addr(split_id, start_offset, server_addr); - Self { - mysql_split: None, - postgres_split: None, - citus_split: Some(split), - mongodb_split: None, - _phantom: PhantomData, - } - } - pub fn new_mongodb(split_id: u32, start_offset: Option) -> Self { - if !match T::source_type() { - CdcSourceType::Mongodb => true, - _ => false, - } { - ::core::panicking::panic( - "assertion failed: matches!(T::source_type(), CdcSourceType::Mongodb)", - ) - }; - let split = MongoDbCdcSplit::new(split_id, start_offset); - Self { - mysql_split: None, - postgres_split: None, - citus_split: None, - mongodb_split: Some(split), - _phantom: PhantomData, - } - } - pub fn split_id(&self) -> u32 { - match T::source_type() { - CdcSourceType::Mysql => self - .mysql_split - .as_ref() - .expect("mysql split must exist") - .split_id(), - CdcSourceType::Postgres => self - .postgres_split - .as_ref() - .expect("postgres split must exist") - .split_id(), - CdcSourceType::Citus => self - .citus_split - .as_ref() - .expect("citus split must exist") - .split_id(), - CdcSourceType::Mongodb => self - .mongodb_split - .as_ref() - .expect("mongodb split must exist") - .split_id(), - CdcSourceType::Unspecified => { - { - ::core::panicking::panic_fmt(format_args!( - "internal error: entered unreachable code: {0}", - format_args!("invalid debezium split") - )); - }; - } - } - } - pub fn start_offset(&self) -> &Option { - match T::source_type() { - CdcSourceType::Mysql => &self - .mysql_split - .as_ref() - .expect("mysql split must exist") - .start_offset(), - CdcSourceType::Postgres => &self - .postgres_split - .as_ref() - .expect("postgres split must exist") - .start_offset(), - CdcSourceType::Citus => &self - .citus_split - .as_ref() - .expect("citus split must exist") - .start_offset(), - CdcSourceType::Mongodb => &self - .mongodb_split - .as_ref() - .expect("mongodb split must exist") - .start_offset(), - CdcSourceType::Unspecified => { - { - ::core::panicking::panic_fmt(format_args!( - "internal error: entered unreachable code: {0}", - format_args!("invalid debezium split") - )); - }; - } - } - } - pub fn snapshot_done(&self) -> bool { - match T::source_type() { - CdcSourceType::Mysql => self - .mysql_split - .as_ref() - .expect("mysql split must exist") - .snapshot_done(), - CdcSourceType::Postgres => self - .postgres_split - .as_ref() - .expect("postgres split must exist") - .snapshot_done(), - CdcSourceType::Citus => self - .citus_split - .as_ref() - .expect("citus split must exist") - .snapshot_done(), - CdcSourceType::Mongodb => self - .mongodb_split - .as_ref() - .expect("mongodb split must exist") - .snapshot_done(), - CdcSourceType::Unspecified => { - { - ::core::panicking::panic_fmt(format_args!( - "internal error: entered unreachable code: {0}", - format_args!("invalid debezium split") - )); - }; - } - } - } - pub fn server_addr(&self) -> Option { - match T::source_type() { - CdcSourceType::Mysql | CdcSourceType::Postgres | CdcSourceType::Mongodb => None, - CdcSourceType::Citus => self - .citus_split - .as_ref() - .expect("split must exist") - .server_addr - .clone(), - CdcSourceType::Unspecified => { - { - ::core::panicking::panic_fmt(format_args!( - "internal error: entered unreachable code: {0}", - format_args!("invalid debezium split") - )); - }; - } - } - } - } -} diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 158349260457d..4560bbe1ceea5 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -37,7 +37,7 @@ use risingwave_connector::schema::schema_registry::{ use risingwave_connector::source::cdc::external::CdcTableType; use risingwave_connector::source::cdc::{ CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CITUS_CDC_CONNECTOR, - MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, MONGODB_CDC_CONNECTOR, + MONGODB_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}; From e4782a51edf4000ca8860da18f2b264fc37b4309 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 3 Feb 2024 16:44:22 +0800 Subject: [PATCH 08/23] clean node 2 --- .../debezium-mongo/docker-compose.yaml | 2 +- .../mongodb-cdc/config-replica.js | 6 ---- integration_tests/mongodb-cdc/create_mv.sql | 7 ----- .../mongodb-cdc/create_source.sql | 6 ---- integration_tests/mongodb-cdc/data_check | 1 - .../mongodb-cdc/docker-compose.yaml | 30 ------------------- 6 files changed, 1 insertion(+), 51 deletions(-) delete mode 100644 integration_tests/mongodb-cdc/config-replica.js delete mode 100644 integration_tests/mongodb-cdc/create_mv.sql delete mode 100644 integration_tests/mongodb-cdc/create_source.sql delete mode 100644 integration_tests/mongodb-cdc/data_check delete mode 100644 integration_tests/mongodb-cdc/docker-compose.yaml diff --git a/integration_tests/debezium-mongo/docker-compose.yaml b/integration_tests/debezium-mongo/docker-compose.yaml index 4056aae12c7d3..afa753440f77d 100644 --- a/integration_tests/debezium-mongo/docker-compose.yaml +++ b/integration_tests/debezium-mongo/docker-compose.yaml @@ -31,7 +31,7 @@ services: image: mongo:4.4 container_name: mongodb ports: - - "7017:27017" + - "27017:27017" command: --replSet rs0 --oplogSize 128 restart: always healthcheck: diff --git a/integration_tests/mongodb-cdc/config-replica.js b/integration_tests/mongodb-cdc/config-replica.js deleted file mode 100644 index a4c29a401791c..0000000000000 --- a/integration_tests/mongodb-cdc/config-replica.js +++ /dev/null @@ -1,6 +0,0 @@ -rsconf = { - _id: "rs0", - members: [{ _id: 0, host: "localhost:27017", priority: 1.0 }], -}; -rs.initiate(rsconf); -rs.status(); diff --git a/integration_tests/mongodb-cdc/create_mv.sql b/integration_tests/mongodb-cdc/create_mv.sql deleted file mode 100644 index 17ce354009c7b..0000000000000 --- a/integration_tests/mongodb-cdc/create_mv.sql +++ /dev/null @@ -1,7 +0,0 @@ -CREATE MATERIALIZED VIEW normalized_users AS -SELECT - payload ->> 'name' as name, - payload ->> 'email' as email, - payload ->> 'address' as address -FROM - users; \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/create_source.sql b/integration_tests/mongodb-cdc/create_source.sql deleted file mode 100644 index 9c95e87644d0c..0000000000000 --- a/integration_tests/mongodb-cdc/create_source.sql +++ /dev/null @@ -1,6 +0,0 @@ -CREATE TABLE users (_id JSONB PRIMARY KEY, payload JSONB) WITH ( - connector = 'kafka', - kafka.topic = 'dbserver1.random_data.users', - kafka.brokers = 'message_queue:29092', - kafka.scan.startup.mode = 'earliest' -) FORMAT DEBEZIUM_MONGO ENCODE JSON; \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/data_check b/integration_tests/mongodb-cdc/data_check deleted file mode 100644 index c57752e1fd9b2..0000000000000 --- a/integration_tests/mongodb-cdc/data_check +++ /dev/null @@ -1 +0,0 @@ -users,normalized_users \ No newline at end of file diff --git a/integration_tests/mongodb-cdc/docker-compose.yaml b/integration_tests/mongodb-cdc/docker-compose.yaml deleted file mode 100644 index ea5258daa29c3..0000000000000 --- a/integration_tests/mongodb-cdc/docker-compose.yaml +++ /dev/null @@ -1,30 +0,0 @@ -version: '3.1' - -services: - mongodb: - image: mongodb/mongodb-community-server:4.4.23-ubi8 - container_name: mongodb - ports: - - "27017:27017" - command: --replSet rs0 --oplogSize 128 - restart: always - healthcheck: - test: "echo 'db.runCommand({ping: 1})' | mongo" - interval: 5s - timeout: 10s - retries: 3 - - mongodb-setup: - image: mongodb/mongodb-community-server:4.4.23-ubi8 - container_name: mongodb-setup - depends_on: - - mongodb - entrypoint: - [ - "bash", - "-c", - "sleep 10 && mongo --host mongodb:27017 /config-replica.js && sleep 10" - ] - restart: "no" - volumes: - - ./config-replica.js:/config-replica.js From 8e73db34aecab6156f135e382239ee55654aed7d Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sat, 3 Feb 2024 21:29:21 +0800 Subject: [PATCH 09/23] support delete --- .../src/parser/debezium/mongo_json_parser.rs | 138 ++++++++++++++---- .../src/parser/debezium/simd_json_parser.rs | 32 ++++ src/connector/src/parser/mod.rs | 8 +- src/connector/src/parser/unified/debezium.rs | 56 ++++++- src/connector/src/parser/unified/mod.rs | 3 + src/connector/src/source/base.rs | 5 +- .../src/source/cdc/source/message.rs | 6 +- src/connector/src/source/cdc/split.rs | 11 +- 8 files changed, 208 insertions(+), 51 deletions(-) diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 9233bc987eb17..1a9648c8326be 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -14,17 +14,17 @@ use std::fmt::Debug; -use risingwave_common::error::ErrorCode::{self, ProtocolError}; +use risingwave_common::error::ErrorCode::ProtocolError; use risingwave_common::error::{Result, RwError}; use risingwave_common::types::DataType; -use simd_json::prelude::MutableObject; -use simd_json::BorrowedValue; -use crate::only_parse_payload; -use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; -use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; +use crate::parser::simd_json_parser::{DebeziumJsonAccessBuilder, DebeziumMongoJsonAccessBuilder}; +use crate::parser::unified::debezium::DebeziumChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; -use crate::parser::{ByteStreamSourceParser, ParserFormat, SourceStreamChunkRowWriter}; +use crate::parser::{ + AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, JsonProperties, ParserFormat, + SourceStreamChunkRowWriter, +}; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; #[derive(Debug)] @@ -33,10 +33,30 @@ pub struct DebeziumMongoJsonParser { id_column: SourceColumnDesc, payload_column: SourceColumnDesc, source_ctx: SourceContextRef, + key_builder: AccessBuilderImpl, + payload_builder: AccessBuilderImpl, +} + +// key and payload in DEBEZIUM_MONGO format are accessed in different ways +async fn build_accessor_builder(config: EncodingProperties) -> Result { + match config { + EncodingProperties::Json(_) => Ok(AccessBuilderImpl::DebeziumJson( + DebeziumJsonAccessBuilder::new()?, + )), + EncodingProperties::MongoJson(_) => Ok(AccessBuilderImpl::DebeziumMongoJson( + DebeziumMongoJsonAccessBuilder::new()?, + )), + _ => Err(RwError::from(ProtocolError( + "unsupported encoding for DEBEZIUM_MONGO format".to_string(), + ))), + } } impl DebeziumMongoJsonParser { - pub fn new(rw_columns: Vec, source_ctx: SourceContextRef) -> Result { + pub async fn new( + rw_columns: Vec, + source_ctx: SourceContextRef, + ) -> Result { let id_column = rw_columns .iter() .find(|desc| { @@ -69,37 +89,54 @@ impl DebeziumMongoJsonParser { ))); } + let key_builder = + build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default())) + .await?; + let payload_builder = + build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default())) + .await?; + Ok(Self { rw_columns, id_column, payload_column, - source_ctx, + key_builder, + payload_builder, }) } - #[allow(clippy::unused_async)] pub async fn parse_inner( - &self, - mut payload: Vec, + &mut self, + key: Option>, + payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, ) -> Result<()> { - let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) - .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + let key_accessor = match key { + None => None, + Some(data) => Some(self.key_builder.generate_accessor(data).await?), + }; + let payload_accessor = match payload { + None => None, + Some(data) => Some(self.payload_builder.generate_accessor(data).await?), + }; + // let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) + // .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; + // // Event can be configured with and without the "payload" field present. // See https://github.com/risingwavelabs/risingwave/issues/10178 - let payload = if let Some(payload) = event.get_mut("payload") { - std::mem::take(payload) - } else { - event - }; - - let accessor = JsonAccess::new_with_options(payload, &JsonParseOptions::DEBEZIUM); - - let row_op = DebeziumChangeEvent::with_value(MongoProjection::new(accessor)); + // let payload = if let Some(payload) = event.get_mut("payload") { + // std::mem::take(payload) + // } else { + // event + // }; + // + // let payload_accessor = JsonAccess::new_with_options(payload, &JsonParseOptions::DEBEZIUM); + // let row_op = DebeziumChangeEvent::with_value(MongoProjection::new(payload_accessor)); + let row_op = DebeziumChangeEvent::new_mongodb_event(key_accessor, payload_accessor); apply_row_operation_on_stream_chunk_writer(row_op, &mut writer).map_err(Into::into) } } @@ -119,11 +156,12 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { async fn parse_one<'a>( &'a mut self, - _key: Option>, + key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, ) -> Result<()> { - only_parse_payload!(self, payload, writer) + // only_parse_payload!(self, payload, writer) + self.parse_inner(key, payload, writer).await } } @@ -160,13 +198,44 @@ mod tests { let a = extract_bson_id(&DataType::Varchar, &pld).unwrap(); assert_eq!(a, Some(ScalarImpl::Utf8("5d505646cf6d4fe581014ab2".into()))); } - fn get_columns() -> Vec { - let descs = vec![ - SourceColumnDesc::simple("_id", DataType::Int64, ColumnId::from(0)), + + #[tokio::test] + async fn test_parse_delete_message() { + let (key, payload) = ( + // key + br#"{"schema":null,"payload":{"id":"{\"$oid\": \"65bc9fb6c485f419a7a877fe\"}"}}"#.to_vec(), + // payload + br#"{"schema":null,"payload":{"before":null,"after":null,"updateDescription":null,"source":{"version":"2.4.2.Final","connector":"mongodb","name":"RW_CDC_3001","ts_ms":1706968217000,"snapshot":"false","db":"dev","sequence":null,"rs":"rs0","collection":"test","ord":2,"lsid":null,"txnNumber":null,"wallTime":null},"op":"d","ts_ms":1706968217377,"transaction":null}}"#.to_vec() + ); + + let columns = vec![ + SourceColumnDesc::simple("_id", DataType::Varchar, ColumnId::from(0)), SourceColumnDesc::simple("payload", DataType::Jsonb, ColumnId::from(1)), ]; + let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()) + .await + .unwrap(); + + let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); + + let writer = builder.row_writer(); + parser + .parse_inner(Some(key), Some(payload), writer) + .await + .unwrap(); + let chunk = builder.finish(); + let mut rows = chunk.rows(); - descs + let (op, row) = rows.next().unwrap(); + assert_eq!(op, Op::Delete); + // oid + assert_eq!( + row.datum_at(0).to_owned_datum(), + (Some(ScalarImpl::Utf8("65bc9fb6c485f419a7a877fe".into()))) + ); + + // payload should be null + assert_eq!(row.datum_at(1).to_owned_datum(), None); } #[tokio::test] @@ -177,14 +246,19 @@ mod tests { // data without payload and schema field br#"{"before":null,"after":"{\"_id\": {\"$numberLong\": \"1004\"},\"first_name\": \"Anne\",\"last_name\": \"Kretchmar\",\"email\": \"annek@noanswer.org\"}","patch":null,"filter":null,"updateDescription":null,"source":{"version":"2.1.4.Final","connector":"mongodb","name":"dbserver1","ts_ms":1681879044000,"snapshot":"last","db":"inventory","sequence":null,"rs":"rs0","collection":"customers","ord":1,"lsid":null,"txnNumber":null},"op":"r","ts_ms":1681879054736,"transaction":null}"#.to_vec()]; - let columns = get_columns(); + let columns = vec![ + SourceColumnDesc::simple("_id", DataType::Int64, ColumnId::from(0)), + SourceColumnDesc::simple("payload", DataType::Jsonb, ColumnId::from(1)), + ]; for data in input { - let parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()).unwrap(); + let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()) + .await + .unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); let writer = builder.row_writer(); - parser.parse_inner(data, writer).await.unwrap(); + parser.parse_inner(None, Some(data), writer).await.unwrap(); let chunk = builder.finish(); let mut rows = chunk.rows(); let (op, row) = rows.next().unwrap(); diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 5efdd237e9e32..d1a2d75985e7d 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -18,6 +18,7 @@ use risingwave_common::error::{ErrorCode, Result, RwError}; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; +use crate::parser::unified::debezium::MongoJsonAccess; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::AccessBuilder; @@ -54,6 +55,37 @@ impl AccessBuilder for DebeziumJsonAccessBuilder { } } +#[derive(Debug)] +pub struct DebeziumMongoJsonAccessBuilder { + value: Option>, +} + +impl DebeziumMongoJsonAccessBuilder { + pub fn new() -> Result { + Ok(Self { value: None }) + } +} + +impl AccessBuilder for DebeziumMongoJsonAccessBuilder { + #[allow(clippy::unused_async)] + async fn generate_accessor(&mut self, payload: Vec) -> Result> { + self.value = Some(payload); + let mut event: BorrowedValue<'_> = + simd_json::to_borrowed_value(self.value.as_mut().unwrap()) + .map_err(|e| RwError::from(ErrorCode::ProtocolError(e.to_string())))?; + + let payload = if let Some(payload) = event.get_mut("payload") { + std::mem::take(payload) + } else { + event + }; + + Ok(AccessImpl::MongoJson(MongoJsonAccess::new( + JsonAccess::new_with_options(payload, &JsonParseOptions::DEBEZIUM), + ))) + } +} + #[cfg(test)] mod tests { use std::convert::TryInto; diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 2688de5cb0bee..7ae0131500b50 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -50,6 +50,7 @@ use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; use crate::common::AwsAuthProps; use crate::parser::maxwell::MaxwellParser; +use crate::parser::simd_json_parser::DebeziumMongoJsonAccessBuilder; use crate::parser::unified::AccessError; use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; @@ -724,6 +725,7 @@ pub enum AccessBuilderImpl { Bytes(BytesAccessBuilder), DebeziumAvro(DebeziumAvroAccessBuilder), DebeziumJson(DebeziumJsonAccessBuilder), + DebeziumMongoJson(DebeziumMongoJsonAccessBuilder), } impl AccessBuilderImpl { @@ -756,6 +758,7 @@ impl AccessBuilderImpl { Self::Bytes(builder) => builder.generate_accessor(payload).await?, Self::DebeziumAvro(builder) => builder.generate_accessor(payload).await?, Self::DebeziumJson(builder) => builder.generate_accessor(payload).await?, + Self::DebeziumMongoJson(builder) => builder.generate_accessor(payload).await?, }; Ok(accessor) } @@ -803,7 +806,9 @@ impl ByteStreamSourceParserImpl { CsvParser::new(rw_columns, *config, source_ctx).map(Self::Csv) } (ProtocolProperties::DebeziumMongo, EncodingProperties::Json(_)) => { - DebeziumMongoJsonParser::new(rw_columns, source_ctx).map(Self::DebeziumMongoJson) + DebeziumMongoJsonParser::new(rw_columns, source_ctx) + .await + .map(Self::DebeziumMongoJson) } (ProtocolProperties::Canal, EncodingProperties::Json(config)) => { CanalJsonParser::new(rw_columns, source_ctx, config).map(Self::CanalJson) @@ -917,6 +922,7 @@ pub enum EncodingProperties { Protobuf(ProtobufProperties), Csv(CsvProperties), Json(JsonProperties), + MongoJson(JsonProperties), Bytes(BytesProperties), Native, #[default] diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index bbb916c829a80..69a4113057356 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -22,6 +22,7 @@ use crate::source::{ConnectorProperties, SourceColumnDesc}; pub struct DebeziumChangeEvent { value_accessor: Option, key_accessor: Option, + is_mongo: bool, } const BEFORE: &str = "before"; @@ -94,6 +95,16 @@ where Self { value_accessor, key_accessor, + is_mongo: false, + } + } + + pub fn new_mongodb_event(key_accessor: Option, value_accessor: Option) -> Self { + assert!(key_accessor.is_some() || value_accessor.is_some()); + Self { + value_accessor, + key_accessor, + is_mongo: true, } } @@ -120,6 +131,16 @@ where fn access_field(&self, desc: &SourceColumnDesc) -> super::AccessResult { match self.op()? { ChangeEventOperation::Delete => { + // For delete events of MongoDB, the "before" and "after" field both are null in the value, + // we need to extract the _id field from the key. + if self.is_mongo && desc.name == "_id" { + return self + .key_accessor + .as_ref() + .expect("key_accessor must be provided for delete operation") + .access(&[&desc.name], Some(&desc.data_type)); + } + if let Some(va) = self.value_accessor.as_ref() { va.access(&[BEFORE, &desc.name], Some(&desc.data_type)) } else { @@ -160,14 +181,16 @@ where } } -pub struct MongoProjection { +pub struct MongoJsonAccess { accessor: A, } pub fn extract_bson_id(id_type: &DataType, bson_doc: &serde_json::Value) -> anyhow::Result { - let id_field = bson_doc - .get("_id") - .ok_or_else(|| anyhow::format_err!("Debezuim Mongo requires document has a `_id` field"))?; + let id_field = if let Some(value) = bson_doc.get("_id") { + value + } else { + bson_doc + }; let id: Datum = match id_type { DataType::Jsonb => ScalarImpl::Jsonb(id_field.clone().into()).into(), DataType::Varchar => match id_field { @@ -201,13 +224,13 @@ pub fn extract_bson_id(id_type: &DataType, bson_doc: &serde_json::Value) -> anyh }; Ok(id) } -impl MongoProjection { +impl MongoJsonAccess { pub fn new(accessor: A) -> Self { Self { accessor } } } -impl Access for MongoProjection +impl Access for MongoJsonAccess where A: Access, { @@ -228,6 +251,27 @@ where } } ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), + ["_id"] => { + let ret = self.accessor.access(path, type_expected); + // For a DELETE message, the "_id" field is not in the payload, and named as "id" in the key. + // e.g. {"schema":null,"payload":{"id":"{\"$oid\": \"65bc9fb6c485f419a7a877fe\"}"}} + if matches!(ret, Err(AccessError::Undefined { .. })) { + let id_bson = self.accessor.access(&["id"], Some(&DataType::Jsonb))?; + if let Some(ScalarImpl::Jsonb(bson_doc)) = id_bson { + Ok(extract_bson_id( + type_expected.unwrap_or(&DataType::Jsonb), + &bson_doc.take(), + )?) + } else { + unreachable!( + "access result must match the type_expected. path: [\"id\"], id: {:?}, type_expected: {:?}", + id_bson, type_expected + ) + } + } else { + ret + } + } _ => self.accessor.access(path, type_expected), } } diff --git a/src/connector/src/parser/unified/mod.rs b/src/connector/src/parser/unified/mod.rs index bd52f4628ef2f..a282f2eec7d33 100644 --- a/src/connector/src/parser/unified/mod.rs +++ b/src/connector/src/parser/unified/mod.rs @@ -22,6 +22,7 @@ use self::avro::AvroAccess; use self::bytes::BytesAccess; use self::json::JsonAccess; use self::protobuf::ProtobufAccess; +use crate::parser::unified::debezium::MongoJsonAccess; use crate::source::SourceColumnDesc; pub mod avro; @@ -45,6 +46,7 @@ pub enum AccessImpl<'a, 'b> { Bytes(BytesAccess<'a>), Protobuf(ProtobufAccess), Json(JsonAccess<'a, 'b>), + MongoJson(MongoJsonAccess>), } impl Access for AccessImpl<'_, '_> { @@ -54,6 +56,7 @@ impl Access for AccessImpl<'_, '_> { Self::Bytes(accessor) => accessor.access(path, type_expected), Self::Protobuf(accessor) => accessor.access(path, type_expected), Self::Json(accessor) => accessor.access(path, type_expected), + Self::MongoJson(accessor) => accessor.access(path, type_expected), } } } diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 9cf496fa01413..6f8d9ebbb0138 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -656,7 +656,7 @@ mod tests { use nexmark::event::EventType; use super::*; - use crate::source::cdc::{CdcSourceType, DebeziumCdcSplit}; + use crate::source::cdc::{DebeziumCdcSplit, Mysql}; use crate::source::kafka::KafkaSplit; #[test] @@ -674,8 +674,7 @@ mod tests { #[test] fn test_cdc_split_state() -> Result<()> { let offset_str = "{\"sourcePartition\":{\"server\":\"RW_CDC_mydb.products\"},\"sourceOffset\":{\"transaction_id\":null,\"ts_sec\":1670407377,\"file\":\"binlog.000001\",\"pos\":98587,\"row\":2,\"server_id\":1,\"event\":2}}"; - let split = - DebeziumCdcSplit::::new(1001, Some(offset_str.to_string()), None); + let split = DebeziumCdcSplit::::new(1001, Some(offset_str.to_string()), None); let split_impl = SplitImpl::MysqlCdc(split); let encoded_split = split_impl.encode_to_bytes(); let restored_split_impl = SplitImpl::restore_from_bytes(encoded_split.as_ref())?; diff --git a/src/connector/src/source/cdc/source/message.rs b/src/connector/src/source/cdc/source/message.rs index 28fe52c52cd1e..5df937a83fbe8 100644 --- a/src/connector/src/source/cdc/source/message.rs +++ b/src/connector/src/source/cdc/source/message.rs @@ -29,7 +29,11 @@ pub struct DebeziumCdcMeta { impl From for SourceMessage { fn from(message: CdcMessage) -> Self { SourceMessage { - key: None, + key: if message.key.is_empty() { + None // only data message has key + } else { + Some(message.key.as_bytes().to_vec()) + }, payload: if message.payload.is_empty() { None // heartbeat message } else { diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 4fff7a5cc18f3..068cb3a7d8636 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -179,12 +179,6 @@ impl MongoDbCdcSplit { }; Self { inner: split } } - - // TODO: update offset for mongodb - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - self.inner.start_offset = Some(start_offset); - Ok(()) - } } impl CdcSplitTrait for MongoDbCdcSplit { @@ -200,8 +194,9 @@ impl CdcSplitTrait for MongoDbCdcSplit { self.inner.snapshot_done } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { - // TODO + fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + // TODO: update snapshot_done for mongodb + self.inner.start_offset = Some(start_offset); Ok(()) } } From d17022462014a7feb718ba491603e8ff35598592 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 4 Feb 2024 00:39:11 +0800 Subject: [PATCH 10/23] make clippy happy --- src/compute/tests/cdc_tests.rs | 15 ++---------- .../src/parser/debezium/mongo_json_parser.rs | 24 ++++++------------- src/connector/src/parser/mod.rs | 4 +--- src/connector/src/parser/unified/debezium.rs | 5 ++-- src/connector/src/source/cdc/source/reader.rs | 2 +- src/connector/src/source/cdc/split.rs | 2 +- 6 files changed, 15 insertions(+), 37 deletions(-) diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 3696376fd6737..cfcc9028fa763 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -16,7 +16,6 @@ #![feature(coroutines)] use std::collections::{HashMap, HashSet}; -use std::marker::PhantomData; use std::str::FromStr; use std::sync::atomic::AtomicU64; use std::sync::Arc; @@ -34,7 +33,7 @@ use risingwave_connector::source::cdc::external::mock_external_table::MockExtern use risingwave_connector::source::cdc::external::{ DebeziumOffset, DebeziumSourceOffset, ExternalTableReaderImpl, MySqlOffset, SchemaTableName, }; -use risingwave_connector::source::cdc::{CdcSplitBase, DebeziumCdcSplit, MySqlCdcSplit}; +use risingwave_connector::source::cdc::DebeziumCdcSplit; use risingwave_connector::source::SplitImpl; use risingwave_hummock_sdk::to_committed_batch_query_epoch; use risingwave_storage::memory::MemoryStateStore; @@ -294,17 +293,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { 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, - }, - }), - postgres_split: None, - _phantom: PhantomData, - })], + vec![SplitImpl::MysqlCdc(DebeziumCdcSplit::new(0, None, None))], ); let init_barrier = Barrier::new_test_barrier(curr_epoch).with_mutation(Mutation::Add(AddMutation { diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 1a9648c8326be..553d4ad433776 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -38,7 +38,7 @@ pub struct DebeziumMongoJsonParser { } // key and payload in DEBEZIUM_MONGO format are accessed in different ways -async fn build_accessor_builder(config: EncodingProperties) -> Result { +fn build_accessor_builder(config: EncodingProperties) -> Result { match config { EncodingProperties::Json(_) => Ok(AccessBuilderImpl::DebeziumJson( DebeziumJsonAccessBuilder::new()?, @@ -53,10 +53,7 @@ async fn build_accessor_builder(config: EncodingProperties) -> Result, - source_ctx: SourceContextRef, - ) -> Result { + pub fn new(rw_columns: Vec, source_ctx: SourceContextRef) -> Result { let id_column = rw_columns .iter() .find(|desc| { @@ -90,11 +87,9 @@ impl DebeziumMongoJsonParser { } let key_builder = - build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default())) - .await?; + build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default()))?; let payload_builder = - build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default())) - .await?; + build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default()))?; Ok(Self { rw_columns, @@ -212,12 +207,8 @@ mod tests { SourceColumnDesc::simple("_id", DataType::Varchar, ColumnId::from(0)), SourceColumnDesc::simple("payload", DataType::Jsonb, ColumnId::from(1)), ]; - let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()) - .await - .unwrap(); - + let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); - let writer = builder.row_writer(); parser .parse_inner(Some(key), Some(payload), writer) @@ -251,9 +242,8 @@ mod tests { SourceColumnDesc::simple("payload", DataType::Jsonb, ColumnId::from(1)), ]; for data in input { - let mut parser = DebeziumMongoJsonParser::new(columns.clone(), Default::default()) - .await - .unwrap(); + let mut parser = + DebeziumMongoJsonParser::new(columns.clone(), Default::default()).unwrap(); let mut builder = SourceStreamChunkBuilder::with_capacity(columns.clone(), 3); diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 7ae0131500b50..885e501466bb7 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -806,9 +806,7 @@ impl ByteStreamSourceParserImpl { CsvParser::new(rw_columns, *config, source_ctx).map(Self::Csv) } (ProtocolProperties::DebeziumMongo, EncodingProperties::Json(_)) => { - DebeziumMongoJsonParser::new(rw_columns, source_ctx) - .await - .map(Self::DebeziumMongoJson) + DebeziumMongoJsonParser::new(rw_columns, source_ctx).map(Self::DebeziumMongoJson) } (ProtocolProperties::Canal, EncodingProperties::Json(config)) => { CanalJsonParser::new(rw_columns, source_ctx, config).map(Self::CanalJson) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 69a4113057356..58395e3f0589c 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -251,10 +251,11 @@ where } } ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), + // To handle a DELETE message, we need to extract the "_id" field from the message key, because it is not in the payload. + // In addition, the "_id" field is named as "id" in the key. An example of message key: + // {"schema":null,"payload":{"id":"{\"$oid\": \"65bc9fb6c485f419a7a877fe\"}"}} ["_id"] => { let ret = self.accessor.access(path, type_expected); - // For a DELETE message, the "_id" field is not in the payload, and named as "id" in the key. - // e.g. {"schema":null,"payload":{"id":"{\"$oid\": \"65bc9fb6c485f419a7a877fe\"}"}} if matches!(ret, Err(AccessError::Undefined { .. })) { let id_bson = self.accessor.access(&["id"], Some(&DataType::Jsonb))?; if let Some(ScalarImpl::Jsonb(bson_doc)) = id_bson { diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 0928f34867dc5..c8dde3a8012bb 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -78,7 +78,7 @@ impl SplitReader for CdcSplitReader { && let Some(ref server_addr) = citus_split.server_addr { citus_server_addr = Some(server_addr.clone()); - let host_addr = HostAddr::from_str(&server_addr) + let host_addr = HostAddr::from_str(server_addr) .map_err(|err| anyhow!("invalid server address for cdc split. {}", err))?; properties.insert("hostname".to_string(), host_addr.host); properties.insert("port".to_string(), host_addr.port.to_string()); diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 068cb3a7d8636..f1f2299f56fd3 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -296,7 +296,7 @@ impl DebeziumCdcSplit { } pub fn start_offset(&self) -> &Option { - dispatch_cdc_split!(&self, ref, { + dispatch_cdc_split!(self, ref, { Mysql, Postgres, Citus, From 6fab3331c419f872e8e910b294352c1903c9410b Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 4 Feb 2024 00:55:32 +0800 Subject: [PATCH 11/23] clean code --- .../connector/source/MongoDbSourceTest.java | 110 ------------------ .../src/parser/debezium/mongo_json_parser.rs | 15 --- 2 files changed, 125 deletions(-) delete mode 100644 java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java diff --git a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java b/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java deleted file mode 100644 index d5e81de97d89c..0000000000000 --- a/java/connector-node/risingwave-source-test/src/test/java/com/risingwave/connector/source/MongoDbSourceTest.java +++ /dev/null @@ -1,110 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package com.risingwave.connector.source; - -import com.risingwave.connector.ConnectorServiceImpl; -import com.risingwave.proto.ConnectorServiceProto; -import io.grpc.*; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; -import org.junit.*; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import org.testcontainers.containers.MongoDBContainer; - -public class MongoDbSourceTest { - private static final Logger LOG = LoggerFactory.getLogger(MongoDbSourceTest.class.getName()); - - private static final MongoDBContainer mongoDbContainer = null; - // new MongoDBContainer("mongodb/mongodb-community-server:4.4.23-ubi8"); - - public static Server connectorService = - ServerBuilder.forPort(SourceTestClient.DEFAULT_PORT) - .addService(new ConnectorServiceImpl()) - .build(); - - public static SourceTestClient testClient = - new SourceTestClient( - Grpc.newChannelBuilder( - "localhost:" + SourceTestClient.DEFAULT_PORT, - InsecureChannelCredentials.create()) - .build()); - - @BeforeClass - public static void init() { - try { - connectorService.start(); - - } catch (Exception e) { - LOG.error("failed to start connector service", e); - Assert.fail(); - } - - // TODO: init data into mongodb - - } - - @AfterClass - public static void cleanup() { - connectorService.shutdown(); - } - - static String getConnectionString() { - return mongoDbContainer.getConnectionString() + "?replicaSet=docker-rs"; - } - - @Test - public void testSnapshotLoad() throws Exception { - Map props = new HashMap<>(); - props.put("mongodb.url", "mongodb://localhost:27017/?replicaSet=rs0"); - props.put("collection.name", "dev.test"); - Iterator eventStream = - testClient.getEventStream(ConnectorServiceProto.SourceType.MONGODB, 3001, props); - Callable countTask = - () -> { - int count = 0; - while (eventStream.hasNext()) { - List messages = - eventStream.next().getEventsList(); - for (ConnectorServiceProto.CdcMessage msg : messages) { - if (!msg.getPayload().isBlank()) { - count++; - } - } - if (count >= 13) { - return count; - } - } - return count; - }; - - var pool = Executors.newFixedThreadPool(1); - var result = pool.submit(countTask); - Assert.assertEquals(11, result.get().intValue()); - } - - @Test - public void testIncrementalLoad() throws Exception { - Map props = new HashMap<>(); - props.put("mongodb.url", getConnectionString()); - props.put("collection.name", "dev.test"); - Iterator eventStream = - testClient.getEventStream(ConnectorServiceProto.SourceType.MONGODB, 3001, props); - } -} diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index daaa8412b7377..8de29588a4ae8 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -117,21 +117,6 @@ impl DebeziumMongoJsonParser { Some(data) => Some(self.payload_builder.generate_accessor(data).await?), }; - // let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) - // .map_err(|e| RwError::from(ProtocolError(e.to_string())))?; - // - // Event can be configured with and without the "payload" field present. - // See https://github.com/risingwavelabs/risingwave/issues/10178 - - // let payload = if let Some(payload) = event.get_mut("payload") { - // std::mem::take(payload) - // } else { - // event - // }; - // - // let payload_accessor = JsonAccess::new_with_options(payload, &JsonParseOptions::DEBEZIUM); - // let row_op = DebeziumChangeEvent::with_value(MongoProjection::new(payload_accessor)); - let row_op = DebeziumChangeEvent::new_mongodb_event(key_accessor, payload_accessor); apply_row_operation_on_stream_chunk_writer(row_op, &mut writer).map_err(Into::into) } From 17af0155a22af7eb4896e1ae5869ac8cadf5bbac Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 4 Feb 2024 11:45:13 +0800 Subject: [PATCH 12/23] update snapshot_done flag when snapshot is finished --- src/connector/src/source/cdc/split.rs | 24 +++++++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index f1f2299f56fd3..25991317f290d 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -195,8 +195,30 @@ impl CdcSplitTrait for MongoDbCdcSplit { } fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - // TODO: update snapshot_done for mongodb + let mut snapshot_done = self.inner.snapshot_done; + // extract snapshot state from debezium offset + if !snapshot_done { + let dbz_offset: DebeziumOffset = serde_json::from_str(&start_offset).map_err(|e| { + anyhow!( + start_offset, + source_id = self.inner.split_id, + "mongodb split: invalid debezium offset", + e + ) + })?; + + // heartbeat event should not update the `snapshot_done` flag + if !dbz_offset.is_heartbeat { + snapshot_done = match dbz_offset.source_offset.snapshot { + Some(val) => !val, + None => true, + }; + } + } + self.inner.start_offset = Some(start_offset); + // if snapshot_done is already true, it will remain true + self.inner.snapshot_done = snapshot_done; Ok(()) } } From 1fc3a52f6a91a66b2fe2a52eeb6503c133c6d852 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 4 Feb 2024 12:30:12 +0800 Subject: [PATCH 13/23] minor --- .../connector/source/core/DbzCdcEventConsumer.java | 8 ++++++-- .../src/main/resources/mongodb.properties | 4 +++- src/connector/src/parser/mod.rs | 1 - src/connector/src/source/cdc/split.rs | 6 +++--- 4 files changed, 12 insertions(+), 7 deletions(-) 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 1b6c5f99e109c..bcc532038c9c6 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 @@ -188,8 +188,11 @@ var record = event.value(); .setKey(new String(key, StandardCharsets.UTF_8)) .setSourceTsMs(sourceTsMs) .build(); - LOG.info( - "key => {}, payload => {}", message.getKey(), message.getPayload()); + LOG.debug( + "offset => {}, key => {}, payload => {}", + message.getOffset(), + message.getKey(), + message.getPayload()); respBuilder.addEvents(message); break; } @@ -201,6 +204,7 @@ var record = event.value(); committer.markProcessed(event); } + LOG.debug("recv {} events", respBuilder.getEventsCount()); // skip empty batch if (respBuilder.getEventsCount() > 0) { respBuilder.setSourceId(sourceId); diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties index 57845ebf11ec5..615fd553f6600 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties @@ -11,5 +11,7 @@ heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} # TODO: set this field in the code name=${collection.name:-RW_CDC_Sharing} provide.transaction.metadata=${transactional:-false} -capture.mode=${debezium.capture.mode:-change_streams_update_full_with_pre_image} +# update event messages include the full document +capture.mode=${debezium.capture.mode:-change_streams_update_full} +# disable tombstones event tombstones.on.delete=${debezium.tombstones.on.delete:-false} diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index 2116c807e0813..d148b0561a22e 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -50,7 +50,6 @@ use self::util::get_kafka_topic; use crate::common::AwsAuthProps; use crate::parser::maxwell::MaxwellParser; use crate::parser::simd_json_parser::DebeziumMongoJsonAccessBuilder; -use crate::parser::unified::AccessError; use crate::parser::util::{ extract_header_inner_from_meta, extract_headers_from_meta, extreact_timestamp_from_meta, }; diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 25991317f290d..1d3546860cf13 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -200,10 +200,10 @@ impl CdcSplitTrait for MongoDbCdcSplit { if !snapshot_done { let dbz_offset: DebeziumOffset = serde_json::from_str(&start_offset).map_err(|e| { anyhow!( + "invalid mongodb offset: {}, error: {}, split: {}", start_offset, - source_id = self.inner.split_id, - "mongodb split: invalid debezium offset", - e + e, + self.inner.split_id ) })?; From ef7ca658faf0f690b3068ef91e5a17336f026141 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 4 Feb 2024 13:50:30 +0800 Subject: [PATCH 14/23] clippy --- src/connector/src/parser/unified/debezium.rs | 1 - 1 file changed, 1 deletion(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index b48bfa24d782a..067324ec01980 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -15,7 +15,6 @@ use risingwave_common::types::{DataType, Datum, ScalarImpl}; use super::{Access, AccessError, AccessResult, ChangeEvent, ChangeEventOperation}; -use crate::parser::unified::uncategorized; use crate::parser::TransactionControl; use crate::source::{ConnectorProperties, SourceColumnDesc}; From bcabf968d42a50633e898110d1b790e870e9f5a6 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 5 Feb 2024 15:36:00 +0800 Subject: [PATCH 15/23] minor --- .../risingwave/connector/source/core/DbzCdcEngineRunner.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java index c8e02aa9a50f9..2b55adc44a306 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/core/DbzCdcEngineRunner.java @@ -47,12 +47,14 @@ public static CdcEngineRunner newCdcEngineRunner( config.getResolvedDebeziumProps(), (success, message, error) -> { if (!success) { - responseObserver.onError(error); LOG.error( "engine#{} terminated with error. message: {}", sourceId, message, error); + if (error != null) { + responseObserver.onError(error); + } } else { LOG.info("engine#{} stopped normally. {}", sourceId, message); responseObserver.onCompleted(); From 5c7420ffe8b206d72ec56d6135d2a98d75394c8a Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 18 Feb 2024 15:20:40 +0800 Subject: [PATCH 16/23] fix comments --- src/connector/src/parser/unified/debezium.rs | 18 ++++--- src/connector/src/source/cdc/split.rs | 52 +++++++------------- 2 files changed, 29 insertions(+), 41 deletions(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 067324ec01980..5a54a550e816d 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -257,10 +257,11 @@ where &bson_doc.take(), )?) } else { - unreachable!( - "access result must match the type_expected. path: {:?}, payload: {:?}, type_expected: {:?}", - path, payload, type_expected - ) + // fail to extract the "_id" field from the message payload + Err(AccessError::Undefined { + name: "_id".to_string(), + path: path[0].to_string(), + }) } } ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), @@ -277,10 +278,11 @@ where &bson_doc.take(), )?) } else { - unreachable!( - "access result must match the type_expected. path: [\"id\"], id: {:?}, type_expected: {:?}", - id_bson, type_expected - ) + // fail to extract the "_id" field from the message key + Err(AccessError::Undefined { + name: "_id".to_string(), + path: "id".to_string(), + }) } } else { ret diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index 27a39946c015b..0fa639ebbaebe 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -196,14 +196,13 @@ impl CdcSplitTrait for MongoDbCdcSplit { let mut snapshot_done = self.inner.snapshot_done; // extract snapshot state from debezium offset if !snapshot_done { - let dbz_offset: DebeziumOffset = serde_json::from_str(&start_offset).map_err(|e| { - anyhow!( - "invalid mongodb offset: {}, error: {}, split: {}", - start_offset, - e, - self.inner.split_id - ) - })?; + let dbz_offset: DebeziumOffset = + serde_json::from_str(&start_offset).with_context(|| { + format!( + "invalid mongodb offset: {}, split: {}", + start_offset, self.inner.split_id + ) + })?; // heartbeat event should not update the `snapshot_done` flag if !dbz_offset.is_heartbeat { @@ -235,7 +234,7 @@ pub struct DebeziumCdcSplit { pub _phantom: PhantomData, } -macro_rules! dispatch_cdc_split { +macro_rules! dispatch_cdc_split_inner { ($dbz_split:expr, $as_type:tt, {$($cdc_source_type:tt),*}, $body:expr) => { match T::source_type() { $( @@ -255,6 +254,13 @@ macro_rules! dispatch_cdc_split { } } +// call corresponding split method of the specific cdc source type +macro_rules! dispatch_cdc_split { + ($dbz_split:expr, $as_type:tt, $body:expr) => { + dispatch_cdc_split_inner!($dbz_split, $as_type, {Mysql, Postgres, Citus, Mongodb}, $body) + } +} + impl SplitMetaData for DebeziumCdcSplit { fn id(&self) -> SplitId { format!("{}", self.split_id()).into() @@ -307,39 +313,19 @@ impl DebeziumCdcSplit { } pub fn split_id(&self) -> u32 { - dispatch_cdc_split!(self, ref, { - Mysql, - Postgres, - Citus, - Mongodb - }, split_id()) + dispatch_cdc_split!(self, ref, split_id()) } pub fn start_offset(&self) -> &Option { - dispatch_cdc_split!(self, ref, { - Mysql, - Postgres, - Citus, - Mongodb - }, start_offset()) + dispatch_cdc_split!(self, ref, start_offset()) } pub fn snapshot_done(&self) -> bool { - dispatch_cdc_split!(self, ref, { - Mysql, - Postgres, - Citus, - Mongodb - }, is_snapshot_done()) + dispatch_cdc_split!(self, ref, is_snapshot_done()) } pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { - dispatch_cdc_split!(self, mut, { - Mysql, - Postgres, - Citus, - Mongodb - }, update_with_offset(start_offset)?); + dispatch_cdc_split!(self, mut, update_with_offset(start_offset)?); Ok(()) } } From 5987352de5748719707feb472ff406a263d91aac Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 18 Feb 2024 15:55:43 +0800 Subject: [PATCH 17/23] throw error --- src/connector/src/parser/unified/debezium.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/connector/src/parser/unified/debezium.rs b/src/connector/src/parser/unified/debezium.rs index 5a54a550e816d..589fdfc8cfc45 100644 --- a/src/connector/src/parser/unified/debezium.rs +++ b/src/connector/src/parser/unified/debezium.rs @@ -261,7 +261,7 @@ where Err(AccessError::Undefined { name: "_id".to_string(), path: path[0].to_string(), - }) + })? } } ["after" | "before", "payload"] => self.access(&[path[0]], Some(&DataType::Jsonb)), @@ -282,7 +282,7 @@ where Err(AccessError::Undefined { name: "_id".to_string(), path: "id".to_string(), - }) + })? } } else { ret From a3f3f543841766a2aa9b47a3659f967a5b124302 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Sun, 18 Feb 2024 17:10:19 +0800 Subject: [PATCH 18/23] fix --- src/connector/src/parser/debezium/mongo_json_parser.rs | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 94c36285b91e9..9f19b1690dcd1 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -18,7 +18,7 @@ use anyhow::Context; use risingwave_common::bail; use risingwave_common::types::DataType; -use crate::parser::simd_json_parser::{DebeziumJsonAccessBuilder, DebeziumMongoJsonAccessBuilder}; +use crate::parser::simd_json_parser::DebeziumMongoJsonAccessBuilder; use crate::parser::unified::debezium::DebeziumChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; use crate::parser::{ @@ -37,12 +37,8 @@ pub struct DebeziumMongoJsonParser { payload_builder: AccessBuilderImpl, } -// key and payload in DEBEZIUM_MONGO format are accessed in different ways fn build_accessor_builder(config: EncodingProperties) -> anyhow::Result { match config { - EncodingProperties::Json(_) => Ok(AccessBuilderImpl::DebeziumJson( - DebeziumJsonAccessBuilder::new()?, - )), EncodingProperties::MongoJson(_) => Ok(AccessBuilderImpl::DebeziumMongoJson( DebeziumMongoJsonAccessBuilder::new()?, )), @@ -79,6 +75,7 @@ impl DebeziumMongoJsonParser { bail!("Debezium Mongo needs no more columns except `_id` and `payload` in table"); } + // encodings are fixed to MongoJson let key_builder = build_accessor_builder(EncodingProperties::MongoJson(JsonProperties::default()))?; let payload_builder = From f53cf2a8c6076f99afdde56cb2bbc4c771e38484 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 20 Feb 2024 12:42:07 +0800 Subject: [PATCH 19/23] impl validator --- .../source/SourceValidateHandler.java | 3 +- .../source/common/CdcSourceMode.java | 22 --------- .../source/common/DbzConnectorConfig.java | 10 ++++ .../source/common/MongoDbValidator.java | 48 +++++++++++++++++++ .../src/main/resources/mongodb.properties | 4 +- .../risingwave-source-cdc/pom.xml | 4 ++ java/pom.xml | 12 ++++- 7 files changed, 75 insertions(+), 28 deletions(-) delete mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/CdcSourceMode.java create mode 100644 java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java 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 bef55dd700da5..d6afb17fb0ccf 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 @@ -140,7 +140,8 @@ public static void validateSource(ConnectorServiceProto.ValidateSourceRequest re case MONGODB: ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_URL); ensurePropNotBlank(props, DbzConnectorConfig.MongoDb.MONGO_COLLECTION_NAME); - // TODO: validate mongodb connectivity and replica set config + var validator = new MongoDbValidator(props); + validator.validateDbConfig(); break; default: LOG.warn("Unknown source type"); 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 deleted file mode 100644 index 209e034d0fa72..0000000000000 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/CdcSourceMode.java +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -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 7faaa471fec28..5406f6fd5e952 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 @@ -14,6 +14,7 @@ package com.risingwave.connector.source.common; +import com.mongodb.ConnectionString; import com.risingwave.connector.api.source.SourceTypeE; import com.risingwave.connector.cdc.debezium.internal.ConfigurableOffsetBackingStore; import java.io.IOException; @@ -233,6 +234,15 @@ public DbzConnectorConfig( mongodbProps.setProperty( ConfigurableOffsetBackingStore.OFFSET_STATE_VALUE, startOffset); } + + var mongodbUrl = userProps.get("mongodb.url"); + var collection = userProps.get("collection.name"); + var connectionStr = new ConnectionString(mongodbUrl); + var connectorName = + String.format( + "MongoDB_%d:%s:%s", sourceId, connectionStr.getHosts(), collection); + mongodbProps.setProperty("name", connectorName); + dbzProps.putAll(mongodbProps); } else { diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java new file mode 100644 index 0000000000000..86fa74a91041d --- /dev/null +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java @@ -0,0 +1,48 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package com.risingwave.connector.source.common; + +import com.mongodb.client.MongoClient; +import com.mongodb.client.MongoClients; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class MongoDbValidator extends DatabaseValidator { + private static final Logger LOG = LoggerFactory.getLogger(MongoDbValidator.class); + + String mongodbUrl; + + public MongoDbValidator(Map userProps) { + this.mongodbUrl = userProps.get("mongodb.url"); + } + + @Override + public void validateDbConfig() { + // check connectivity + try (MongoClient mongoClient = MongoClients.create(mongodbUrl)) { + var desc = mongoClient.getClusterDescription(); + LOG.info("MongoDB cluster description: {}", desc); + } + } + + @Override + void validateUserPrivilege() {} + + @Override + void validateTable() { + // do nothing + } +} diff --git a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties index 615fd553f6600..a1a5b92e9f63f 100644 --- a/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties +++ b/java/connector-node/risingwave-connector-service/src/main/resources/mongodb.properties @@ -6,10 +6,8 @@ mongodb.connection.string=${mongodb.url} collection.include.list=${collection.name} # default heartbeat interval 5 mins heartbeat.interval.ms=${debezium.heartbeat.interval.ms:-300000} -# In sharing cdc source 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. # TODO: set this field in the code -name=${collection.name:-RW_CDC_Sharing} +name=${collection.name} provide.transaction.metadata=${transactional:-false} # update event messages include the full document capture.mode=${debezium.capture.mode:-change_streams_update_full} diff --git a/java/connector-node/risingwave-source-cdc/pom.xml b/java/connector-node/risingwave-source-cdc/pom.xml index 1b55e2f277964..fd1dccd3f092f 100644 --- a/java/connector-node/risingwave-source-cdc/pom.xml +++ b/java/connector-node/risingwave-source-cdc/pom.xml @@ -62,5 +62,9 @@ org.postgresql postgresql + + org.mongodb + mongodb-driver-sync + diff --git a/java/pom.xml b/java/pom.xml index 9a18e95ff187f..5c501385c1b62 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -84,6 +84,9 @@ 4.15.0 1.18.0 1.17.6 + 42.5.1 + 8.0.28 + 4.11.1 @@ -169,12 +172,17 @@ org.postgresql postgresql - 42.5.1 + ${postgresql.version} mysql mysql-connector-java - 8.0.28 + ${mysql.connector.java.version} + + + org.mongodb + mongodb-driver-sync + ${mongodb.driver.sync.version} org.elasticsearch From 71f6f9da93c1a96fe67480c05ab10237dc061fd6 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 26 Feb 2024 16:42:39 +0800 Subject: [PATCH 20/23] fix clippy --- src/connector/src/source/cdc/source/reader.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index 86410c5441e61..3e63d506fb9bf 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -83,7 +83,7 @@ impl SplitReader for CdcSplitReader { { citus_server_addr = Some(server_addr.clone()); let host_addr = - HostAddr::from_str(&server_addr).context("invalid server address for cdc split")?; + HostAddr::from_str(server_addr).context("invalid server address for cdc split")?; properties.insert("hostname".to_string(), host_addr.host); properties.insert("port".to_string(), host_addr.port.to_string()); // rewrite table name with suffix to capture all shards in the split From 6ec7be3ca8adde70c938c562c264da5f7cdc4bcb Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 26 Feb 2024 16:48:20 +0800 Subject: [PATCH 21/23] add todo for mongodb user privilege check --- .../connector/source/common/MongoDbValidator.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java index 86fa74a91041d..adb50e2d0344e 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java @@ -34,12 +34,20 @@ public void validateDbConfig() { // check connectivity try (MongoClient mongoClient = MongoClients.create(mongodbUrl)) { var desc = mongoClient.getClusterDescription(); - LOG.info("MongoDB cluster description: {}", desc); + LOG.info("test connectivity: MongoDB cluster description: {}", desc); } } @Override - void validateUserPrivilege() {} + void validateUserPrivilege() { + // TODO: check user privilege + // https://debezium.io/documentation/reference/stable/connectors/mongodb.html#setting-up-mongodb + // You must also have a MongoDB user that has the appropriate roles to read the admin + // database where the oplog can be read. Additionally, the user must also be able to read + // the config database in the configuration server of a sharded cluster and must have + // listDatabases privilege action. When change streams are used (the default) the user also + // must have cluster-wide privilege actions find and changeStream. + } @Override void validateTable() { From 89f19116817afaf6fca01008c5f83a8bb55bc7f5 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 26 Feb 2024 16:59:18 +0800 Subject: [PATCH 22/23] minor --- .../risingwave/connector/source/common/MongoDbValidator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java index adb50e2d0344e..0ce5634e34b46 100644 --- a/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java +++ b/java/connector-node/risingwave-connector-service/src/main/java/com/risingwave/connector/source/common/MongoDbValidator.java @@ -51,6 +51,6 @@ void validateUserPrivilege() { @Override void validateTable() { - // do nothing + // do nothing since MongoDB is schemaless } } From 5388f6c70e5f169085e731f9640734a8078f0f32 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Thu, 29 Feb 2024 14:38:17 +0800 Subject: [PATCH 23/23] add some comments --- proto/connector_service.proto | 2 ++ src/sqlparser/src/ast/statement.rs | 1 + 2 files changed, 3 insertions(+) diff --git a/proto/connector_service.proto b/proto/connector_service.proto index 2b10c066078e4..a03af6305192b 100644 --- a/proto/connector_service.proto +++ b/proto/connector_service.proto @@ -156,12 +156,14 @@ message SinkCoordinatorStreamResponse { /* Source Service */ message CdcMessage { + // The value of the Debezium message string payload = 1; string partition = 2; string offset = 3; string full_table_name = 4; int64 source_ts_ms = 5; bool is_transaction_meta = 6; + // The key of the Debezium message, which only used by `mongodb-cdc` connector. string key = 7; } diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 5c8a20250c16d..1f801dde92c62 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -219,6 +219,7 @@ impl Parser { // 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 + // FIXME: parse input `connector` to enum type instead using string here if connector.contains("-cdc") { let expected = if cdc_source_job { ConnectorSchema::plain_json()