From 6b6e22d483b5d6ff785df5ad2ca87bcf90b44194 Mon Sep 17 00:00:00 2001 From: monster <60029759+MonsterChenzhuo@users.noreply.github.com> Date: Fri, 26 May 2023 16:42:30 +0800 Subject: [PATCH] Rewrite ci logic --- docs/en/connector-v2/sink/MongoDB.md | 12 ++- release-note.md | 2 + .../connector-mongodb-e2e/pom.xml | 15 +-- .../v2/mongodb/AbstractMongodbIT.java | 95 +------------------ .../connector/v2/mongodb/MongodbCDCIT.java | 85 ++++++----------- .../e2e/connector/v2/mongodb/MongodbIT.java | 4 +- ...ongodb.conf => fake_cdc_sink_mongodb.conf} | 60 +++++++++--- .../src/test/resources/ddl/mysql_cdc.sql | 42 -------- .../test/resources/docker/server-gtids/my.cnf | 65 ------------- .../src/test/resources/docker/setup.sql | 32 ------- 10 files changed, 96 insertions(+), 316 deletions(-) rename seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/{mysqlcdc_to_mongodb.conf => fake_cdc_sink_mongodb.conf} (59%) delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/ddl/mysql_cdc.sql delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf delete mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/setup.sql diff --git a/docs/en/connector-v2/sink/MongoDB.md b/docs/en/connector-v2/sink/MongoDB.md index ba9369ab449e..8ce704e7748f 100644 --- a/docs/en/connector-v2/sink/MongoDB.md +++ b/docs/en/connector-v2/sink/MongoDB.md @@ -15,6 +15,10 @@ Key Features - [ ] [exactly-once](../../concept/connector-v2-features.md) - [x] [cdc](../../concept/connector-v2-features.md) +**Tips** + +> 1.If you want to use CDC-written features, please enable the upsert-enable configuration. + Description ----------- @@ -222,11 +226,15 @@ sink { ## Changelog -### 2.2.0-beta 2022-09-26 +### 2.2.0-beta - Add MongoDB Source Connector -### Next Version +### 2.3.1-release - [Feature]Refactor mongodb source connector([4620](https://github.com/apache/incubator-seatunnel/pull/4620)) +### Next Version + +- [Feature]Mongodb support cdc sink([4833](https://github.com/apache/seatunnel/pull/4833)) + diff --git a/release-note.md b/release-note.md index 1ee3b49860d8..dd3888e822a2 100644 --- a/release-note.md +++ b/release-note.md @@ -19,6 +19,8 @@ - [Connector-V2] [Kafka] Fix KafkaProducer resources have never been released. (#4302) - [Connector-V2] [Kafka] Fix the permission problem caused by client.id. (#4246) - [Connector-V2] [Kafka] Fix KafkaConsumerThread exit caused by commit offset error. (#4379) +- [Connector-V2] [Mongodb] Mongodb support cdc sink. (#4833) + ### Zeta(ST-Engine) diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/pom.xml index 7825f7ba5e92..96c097c7fa71 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/pom.xml @@ -40,22 +40,9 @@ org.apache.seatunnel - connector-cdc-mysql + connector-fake ${project.version} test - - org.apache.seatunnel - connector-cdc-mysql - ${project.version} - test-jar - test - - - org.testcontainers - mysql - ${testcontainer.version} - test - diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/AbstractMongodbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/AbstractMongodbIT.java index 3f0f7fb1cc38..e6103518ac33 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/AbstractMongodbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/AbstractMongodbIT.java @@ -17,16 +17,11 @@ package org.apache.seatunnel.e2e.connector.v2.mongodb; -import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlContainer; -import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.MySqlVersion; -import org.apache.seatunnel.connectors.seatunnel.cdc.mysql.testutils.UniqueDatabase; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; import org.bson.Document; import org.testcontainers.containers.GenericContainer; -import org.testcontainers.containers.output.Slf4jLogConsumer; -import org.testcontainers.utility.DockerLoggerFactory; import com.mongodb.client.MongoClient; import com.mongodb.client.MongoClients; @@ -35,29 +30,14 @@ import com.mongodb.client.model.Sorts; import lombok.extern.slf4j.Slf4j; -import java.sql.Connection; -import java.sql.DriverManager; -import java.sql.ResultSet; -import java.sql.SQLException; import java.util.ArrayList; import java.util.Arrays; -import java.util.LinkedHashMap; import java.util.List; import java.util.Random; @Slf4j public abstract class AbstractMongodbIT extends TestSuiteBase implements TestResource { - protected static final String MYSQL_HOST = "mysql_cdc_e2e"; - - protected static final String MYSQL_USER_NAME = "st_user"; - - protected static final String MYSQL_USER_PASSWORD = "seatunnel"; - - protected static final String MYSQL_DATABASE = "mysql_cdc"; - - protected static final MySqlContainer MYSQL_CONTAINER = createMySqlContainer(MySqlVersion.V8_0); - protected static final Random RANDOM = new Random(); protected static final List TEST_MATCH_DATASET = generateTestDataSet(5); @@ -88,16 +68,11 @@ public abstract class AbstractMongodbIT extends TestSuiteBase implements TestRes protected static final String MONGODB_CDC_RESULT_TABLE = "test_cdc_table"; - protected static final String SOURCE_SQL = "select * from products"; - - protected final UniqueDatabase inventoryDatabase = - new UniqueDatabase(MYSQL_CONTAINER, MYSQL_DATABASE, "mysqluser", "mysqlpw"); - protected GenericContainer mongodbContainer; protected MongoClient client; - protected void initConnection() { + public void initConnection() { String host = mongodbContainer.getContainerIpAddress(); int port = mongodbContainer.getFirstMappedPort(); String url = String.format("mongodb://%s:%d/%s", host, port, MONGODB_DATABASE); @@ -122,24 +97,7 @@ protected void clearDate(String table) { client.getDatabase(MONGODB_DATABASE).getCollection(table).drop(); } - protected static MySqlContainer createMySqlContainer(MySqlVersion version) { - MySqlContainer mySqlContainer = - new MySqlContainer(version) - .withConfigurationOverride("docker/server-gtids/my.cnf") - .withSetupSQL("docker/setup.sql") - .withNetwork(NETWORK) - .withNetworkAliases(MYSQL_HOST) - .withDatabaseName(MYSQL_DATABASE) - .withUsername(MYSQL_USER_NAME) - .withPassword(MYSQL_USER_PASSWORD) - .withLogConsumer( - new Slf4jLogConsumer( - DockerLoggerFactory.getLogger("mysql-docker-image"))); - - return mySqlContainer; - } - - protected static List generateTestDataSet(int count) { + public static List generateTestDataSet(int count) { List dataSet = new ArrayList<>(); for (int i = 0; i < count; i++) { @@ -212,53 +170,4 @@ protected List readMongodbData(String collection) { } return documents; } - - protected List> querySql(String sql) { - try (Connection connection = getJdbcConnection()) { - ResultSet resultSet = connection.createStatement().executeQuery(sql); - List> result = new ArrayList<>(); - int columnCount = resultSet.getMetaData().getColumnCount(); - while (resultSet.next()) { - LinkedHashMap row = new LinkedHashMap<>(); - for (int i = 1; i <= columnCount; i++) { - String columnName = resultSet.getMetaData().getColumnName(i); - Object columnValue = resultSet.getObject(i); - row.put(columnName, columnValue); - } - result.add(row); - } - return result; - } catch (SQLException e) { - throw new RuntimeException(e); - } - } - - protected Connection getJdbcConnection() throws SQLException { - return DriverManager.getConnection( - MYSQL_CONTAINER.getJdbcUrl(), - MYSQL_CONTAINER.getUsername(), - MYSQL_CONTAINER.getPassword()); - } - - protected void upsertDeleteSourceTable() { - executeSql( - "INSERT INTO mysql_cdc.products (name,description,weight)\n" - + "VALUES ('car battery','12V car battery',31)"); - - executeSql( - "INSERT INTO mysql_cdc.products (name,description,weight)\n" - + "VALUES ('rocks','box of assorted rocks',35)"); - - executeSql("DELETE FROM mysql_cdc.products where weight = 35"); - - executeSql("UPDATE mysql_cdc.products SET name = 'monster' where weight = 35"); - } - - private void executeSql(String sql) { - try (Connection connection = getJdbcConnection()) { - connection.createStatement().execute(sql); - } catch (SQLException e) { - throw new RuntimeException(e); - } - } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbCDCIT.java index 7301cba3e936..f1f605a352c6 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbCDCIT.java @@ -23,9 +23,11 @@ import org.awaitility.Awaitility; import org.bson.Document; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.containers.output.Slf4jLogConsumer; import org.testcontainers.containers.wait.strategy.HttpWaitStrategy; @@ -35,69 +37,47 @@ import lombok.extern.slf4j.Slf4j; +import java.io.IOException; import java.time.Duration; -import java.util.LinkedHashMap; +import java.util.ArrayList; +import java.util.Arrays; import java.util.List; -import java.util.concurrent.CompletableFuture; +import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; import java.util.stream.Stream; import static java.net.HttpURLConnection.HTTP_OK; import static java.net.HttpURLConnection.HTTP_UNAUTHORIZED; -import static org.awaitility.Awaitility.await; @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") + type = {EngineType.SPARK}, + disabledReason = "Spark engine will lose the row kind of record") @Slf4j public class MongodbCDCIT extends AbstractMongodbIT { @TestTemplate - public void testMongodbCDCSink(TestContainer container) { - CompletableFuture executeJobFuture = - CompletableFuture.supplyAsync( - () -> { - try { - container.executeJob("/cdcIT/mysqlcdc_to_mongodb.conf"); - } catch (Exception e) { - log.error("Commit task exception :" + e.getMessage()); - throw new RuntimeException(e); - } - return null; - }); - - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - List> expected = querySql(SOURCE_SQL); - List actual = readMongodbData(MONGODB_CDC_RESULT_TABLE); - - List> actualMapped = - actual.stream() - .map(LinkedHashMap::new) - .peek(map -> map.remove("_id")) - .collect(Collectors.toList()); - - Assertions.assertIterableEquals(expected, actualMapped); - }); - - upsertDeleteSourceTable(); - await().atMost(60000, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - List> expected = querySql(SOURCE_SQL); - List actual = readMongodbData(MONGODB_CDC_RESULT_TABLE); - - List> actualMapped = - actual.stream() - .map(LinkedHashMap::new) - .peek(map -> map.remove("_id")) - .collect(Collectors.toList()); - - Assertions.assertIterableEquals(expected, actualMapped); - }); + public void testMongodbCDCSink(TestContainer container) + throws IOException, InterruptedException { + Container.ExecResult queryResult = + container.executeJob("/cdcIT/fake_cdc_sink_mongodb.conf"); + Assertions.assertEquals(0, queryResult.getExitCode(), queryResult.getStderr()); + Assertions.assertIterableEquals( + Stream.>of(Arrays.asList(1L, "A_1", 100), Arrays.asList(3L, "C", 100)) + .collect(Collectors.toList()), + readMongodbData(MONGODB_CDC_RESULT_TABLE).stream() + .peek(e -> e.remove("_id")) + .map(Document::entrySet) + .map(Set::stream) + .map( + entryStream -> + entryStream + .map(Map.Entry::getValue) + .collect(Collectors.toCollection(ArrayList::new))) + .collect(Collectors.toList())); + clearDate(MONGODB_CDC_RESULT_TABLE); } @BeforeAll @@ -119,7 +99,7 @@ public void startUp() { .withStartupTimeout(Duration.ofMinutes(2))) .withLogConsumer( new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MONGODB_IMAGE))); - // Used for local testing + // For local test use // mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); Startables.deepStart(Stream.of(mongodbContainer)).join(); log.info("Mongodb container started"); @@ -131,14 +111,9 @@ public void startUp() { .atMost(180, TimeUnit.SECONDS) .untilAsserted(this::initConnection); this.initSourceData(); - - log.info("The second stage: Starting Mysql containers..."); - Startables.deepStart(Stream.of(MYSQL_CONTAINER)).join(); - log.info("Mysql Containers are started"); - inventoryDatabase.createAndInitialize(); - log.info("Mysql ddl execution is complete"); } + @AfterAll @Override public void tearDown() { if (client != null) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java index a10a45501787..f4c6e75425ca 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/java/org/apache/seatunnel/e2e/connector/v2/mongodb/MongodbIT.java @@ -21,6 +21,7 @@ import org.awaitility.Awaitility; import org.bson.Document; +import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; @@ -174,7 +175,7 @@ public void startUp() { .withStartupTimeout(Duration.ofMinutes(2))) .withLogConsumer( new Slf4jLogConsumer(DockerLoggerFactory.getLogger(MONGODB_IMAGE))); - // Used for local testing + // For local test use // mongodbContainer.setPortBindings(Collections.singletonList("27017:27017")); Startables.deepStart(Stream.of(mongodbContainer)).join(); log.info("Mongodb container started"); @@ -188,6 +189,7 @@ public void startUp() { this.initSourceData(); } + @AfterAll @Override public void tearDown() { if (client != null) { diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/mysqlcdc_to_mongodb.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/fake_cdc_sink_mongodb.conf similarity index 59% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/mysqlcdc_to_mongodb.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/fake_cdc_sink_mongodb.conf index 0b4aaa0539a6..a75b0b9b1b64 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/mysqlcdc_to_mongodb.conf +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/cdcIT/fake_cdc_sink_mongodb.conf @@ -19,20 +19,57 @@ ###### env { - # You can set engine configuration here execution.parallelism = 1 - job.mode = "STREAMING" - execution.checkpoint.interval = 5000 + job.mode = "BATCH" } source { - MySQL-CDC { - result_table_name = "customers_mysql_cdc" - server-id = 5652 - username = "st_user" - password = "seatunnel" - table-names = ["mysql_cdc.products"] - base-url = "jdbc:mysql://mysql_cdc_e2e:3306/mysql_cdc" + FakeSource { + schema = { + fields { + c_int = bigint + name = string + score = int + } + } + rows = [ + { + kind = INSERT + fields = [1, "A", 100] + }, + { + kind = INSERT + fields = [2, "B", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + }, + { + kind = INSERT + fields = [3, "C", 100] + } + { + kind = UPDATE_BEFORE + fields = [1, "A", 100] + }, + { + kind = UPDATE_AFTER + fields = [1, "A_1", 100] + }, + { + kind = DELETE + fields = [2, "B", 100] + } + ] } } @@ -47,8 +84,7 @@ sink { fields { c_int = int name = string - description = string - weight = float + score = int } } } diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/ddl/mysql_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/ddl/mysql_cdc.sql deleted file mode 100644 index fc14ce9c350e..000000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/ddl/mysql_cdc.sql +++ /dev/null @@ -1,42 +0,0 @@ --- --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You 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. --- - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: inventory --- ---------------------------------------------------------------------------------------------------------------- -CREATE DATABASE IF NOT EXISTS `mysql_cdc`; - -use mysql_cdc; --- Create a mysql data source table -CREATE TABLE products ( - c_int INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY, - name VARCHAR(255) NOT NULL DEFAULT 'SeaTunnel', - description VARCHAR(512), - weight INTEGER -); -ALTER TABLE products AUTO_INCREMENT = 101; - -INSERT INTO products -VALUES (default,"scooter","Small 2-wheel scooter",3), - (default,"car battery","12V car battery",8), - (default,"12-pack drill bits","12-pack of drill bits with sizes ranging from #40 to #3",8), - (default,"hammer","12oz carpenter's hammer",2), - (default,"hammer","14oz carpenter's hammer",5), - (default,"hammer","16oz carpenter's hammer",6), - (default,"rocks","box of assorted rocks",7), - (default,"jacket","water resistent black wind breaker",1), - (default,"spare tire","24 inch spare tire",22); \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf deleted file mode 100644 index a390897885d0..000000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/server-gtids/my.cnf +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You 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. -# - -# For advice on how to change settings please see -# http://dev.mysql.com/doc/refman/5.7/en/server-configuration-defaults.html - -[mysqld] -# -# Remove leading # and set to the amount of RAM for the most important data -# cache in MySQL. Start at 70% of total RAM for dedicated server, else 10%. -# innodb_buffer_pool_size = 128M -# -# Remove leading # to turn on a very important data integrity option: logging -# changes to the binary log between backups. -# log_bin -# -# Remove leading # to set options mainly useful for reporting servers. -# The server defaults are faster for transactions and fast SELECTs. -# Adjust sizes as needed, experiment to find the optimal values. -# join_buffer_size = 128M -# sort_buffer_size = 2M -# read_rnd_buffer_size = 2M -skip-host-cache -skip-name-resolve -#datadir=/var/lib/mysql -#socket=/var/lib/mysql/mysql.sock -secure-file-priv=/var/lib/mysql -user=mysql - -# Disabling symbolic-links is recommended to prevent assorted security risks -symbolic-links=0 - -#log-error=/var/log/mysqld.log -#pid-file=/var/run/mysqld/mysqld.pid - -# ---------------------------------------------- -# Enable the binlog for replication & CDC -# ---------------------------------------------- - -# Enable binary replication log and set the prefix, expiration, and log format. -# The prefix is arbitrary, expiration can be short for integration tests but would -# be longer on a production system. Row-level info is required for ingest to work. -# Server ID is required, but this will vary on production systems -server-id = 223344 -log_bin = mysql-bin -expire_logs_days = 1 -binlog_format = row - -# enable gtid mode -gtid_mode = on -enforce_gtid_consistency = on \ No newline at end of file diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/setup.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/setup.sql deleted file mode 100644 index 2edd6c917e41..000000000000 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-mongodb-e2e/src/test/resources/docker/setup.sql +++ /dev/null @@ -1,32 +0,0 @@ --- --- Licensed to the Apache Software Foundation (ASF) under one or more --- contributor license agreements. See the NOTICE file distributed with --- this work for additional information regarding copyright ownership. --- The ASF licenses this file to You 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. --- - --- In production you would almost certainly limit the replication user must be on the follower (slave) machine, --- to prevent other clients accessing the log from other machines. For example, 'replicator'@'follower.acme.com'. --- However, in this database we'll grant 2 users different privileges: --- --- 1) 'st_user' - all privileges required by the snapshot reader AND binlog reader (used for testing) --- 2) 'mysqluser' - all privileges --- -GRANT SELECT, RELOAD, SHOW DATABASES, REPLICATION SLAVE, REPLICATION CLIENT, LOCK TABLES ON *.* TO 'st_user'@'%'; -CREATE USER 'mysqluser' IDENTIFIED BY 'mysqlpw'; -GRANT ALL PRIVILEGES ON *.* TO 'mysqluser'@'%'; - --- ---------------------------------------------------------------------------------------------------------------- --- DATABASE: emptydb --- ---------------------------------------------------------------------------------------------------------------- -CREATE DATABASE emptydb;