diff --git a/README.md b/README.md index 3624b3930a..9a7dae8057 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,7 @@ This README is meant as a brief walkthrough on the core features of CDC Connecto | [mysql-cdc](docs/content/connectors/mysql-cdc.md) |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | | [oceanbase-cdc](/docs/content/connectors/oceanbase-cdc.md) |
  • [OceanBase CE](https://open.oceanbase.com): 3.1.x, 4.x
  • [OceanBase EE](https://www.oceanbase.com/product/oceanbase): 2.x, 3.x, 4.x | OceanBase Driver: 2.4.x | | [oracle-cdc](docs/content/connectors/oracle-cdc.md) |
  • [Oracle](https://www.oracle.com/index.html): 11, 12, 19 | Oracle Driver: 19.3.0.0 | -| [postgres-cdc](docs/content/connectors/postgres-cdc.md) |
  • [PostgreSQL](https://www.postgresql.org): 9.6, 10, 11, 12 | JDBC Driver: 42.2.27 | +| [postgres-cdc](docs/content/connectors/postgres-cdc.md) |
  • [PostgreSQL](https://www.postgresql.org): 9.6, 10, 11, 12 | JDBC Driver: 42.5.1 | | [sqlserver-cdc](docs/content/connectors/sqlserver-cdc.md) |
  • [Sqlserver](https://www.microsoft.com/sql-server): 2012, 2014, 2016, 2017, 2019 | JDBC Driver: 7.2.2.jre8 | | [tidb-cdc](docs/content/connectors/tidb-cdc.md) |
  • [TiDB](https://www.pingcap.com): 5.1.x, 5.2.x, 5.3.x, 5.4.x, 6.0.0 | JDBC Driver: 8.0.27 | | [Db2-cdc](docs/content/connectors/db2-cdc.md) |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | Db2 Driver: 11.5.0.0 | diff --git a/docs/content/about.md b/docs/content/about.md index 609ce34bb7..458c15eff2 100644 --- a/docs/content/about.md +++ b/docs/content/about.md @@ -13,7 +13,7 @@ The CDC Connectors for Apache Flink® integrate Debezium as the engin | [mysql-cdc](connectors/mysql-cdc.md) |
  • [MySQL](https://dev.mysql.com/doc): 5.6, 5.7, 8.0.x
  • [RDS MySQL](https://www.aliyun.com/product/rds/mysql): 5.6, 5.7, 8.0.x
  • [PolarDB MySQL](https://www.aliyun.com/product/polardb): 5.6, 5.7, 8.0.x
  • [Aurora MySQL](https://aws.amazon.com/cn/rds/aurora): 5.6, 5.7, 8.0.x
  • [MariaDB](https://mariadb.org): 10.x
  • [PolarDB X](https://github.com/ApsaraDB/galaxysql): 2.0.1 | JDBC Driver: 8.0.27 | | [oceanbase-cdc](connectors/oceanbase-cdc.md) |
  • [OceanBase CE](https://open.oceanbase.com): 3.1.x, 4.x
  • [OceanBase EE](https://www.oceanbase.com/product/oceanbase): 2.x, 3.x, 4.x | OceanBase Driver: 2.4.x | | [oracle-cdc](connectors/oracle-cdc.md) |
  • [Oracle](https://www.oracle.com/index.html): 11, 12, 19 | Oracle Driver: 19.3.0.0 | -| [postgres-cdc](connectors/postgres-cdc.md) |
  • [PostgreSQL](https://www.postgresql.org): 9.6, 10, 11, 12 | JDBC Driver: 42.2.12 | +| [postgres-cdc](connectors/postgres-cdc.md) |
  • [PostgreSQL](https://www.postgresql.org): 9.6, 10, 11, 12 | JDBC Driver: 42.5.1 | | [sqlserver-cdc](connectors/sqlserver-cdc.md) |
  • [Sqlserver](https://www.microsoft.com/sql-server): 2012, 2014, 2016, 2017, 2019 | JDBC Driver: 7.2.2.jre8 | | [tidb-cdc](connectors/tidb-cdc.md) |
  • [TiDB](https://www.pingcap.com/): 5.1.x, 5.2.x, 5.3.x, 5.4.x, 6.0.0 | JDBC Driver: 8.0.27 | | [db2-cdc](connectors/db2-cdc.md) |
  • [Db2](https://www.ibm.com/products/db2): 11.5 | DB2 Driver: 11.5.0.0 | diff --git a/docs/content/connectors/postgres-cdc.md b/docs/content/connectors/postgres-cdc.md index dc287d4975..9f7643a753 100644 --- a/docs/content/connectors/postgres-cdc.md +++ b/docs/content/connectors/postgres-cdc.md @@ -24,7 +24,7 @@ In order to setup the Postgres CDC connector, the following table provides depen Download [flink-sql-connector-postgres-cdc-2.4-SNAPSHOT.jar](https://repo1.maven.org/maven2/com/ververica/flink-sql-connector-postgres-cdc/2.4-SNAPSHOT/flink-sql-connector-postgres-cdc-2.4-SNAPSHOT.jar) and put it under `/lib/`. -**Note:** flink-sql-connector-postgres-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-postgres-cdc-2.2.1.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-postgres-cdc), the released version will be available in the Maven central warehouse. +**Note:** flink-sql-connector-postgres-cdc-XXX-SNAPSHOT version is the code corresponding to the development branch. Users need to download the source code and compile the corresponding jar. Users should use the released version, such as [flink-sql-connector-postgres-cdc-2.3.0.jar](https://mvnrepository.com/artifact/com.ververica/flink-sql-connector-postgres-cdc), the released version will be available in the Maven central warehouse. How to create a Postgres CDC table ---------------- @@ -47,7 +47,9 @@ CREATE TABLE shipments ( 'password' = 'postgres', 'database-name' = 'postgres', 'schema-name' = 'public', - 'table-name' = 'shipments' + 'table-name' = 'shipments', + -- experimental feature: incremental snapshot (default off) + 'scan.incremental.snapshot.enabled' = 'true' ); -- read snapshot and binlogs from shipments table @@ -103,7 +105,7 @@ Connector Options (none) String Database name of the PostgreSQL server to monitor. - + schema-name required @@ -139,9 +141,10 @@ Connector Options optional decoderbufs String - The name of the Postgres logical decoding plug-in installed on the server. + The name of the Postgres logical decoding plug-in installed on the server. Supported values are decoderbufs, wal2json, wal2json_rds, wal2json_streaming, wal2json_rds_streaming and pgoutput. - + + changelog-mode optional @@ -149,7 +152,14 @@ Connector Options String The changelog mode used for encoding streaming changes. Supported values are all (which encodes changes as retract stream using all RowKinds) and upsert (which encodes changes as upsert stream that describes idempotent updates on a key).
    upsert mode can be used for tables with primary keys when replica identity FULL is not an option. Primary keys must be set to use upsert mode. - + + + heartbeat.interval.ms + optional + 30s + Duration + The interval of sending heartbeat event for tracing the latest available replication slot offsets + debezium.* optional @@ -157,8 +167,8 @@ Connector Options String Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from Postgres server. For example: 'debezium.snapshot.mode' = 'never'. - See more about the Debezium's Postgres Connector properties - + See more about the Debezium's Postgres Connector properties + debezium.snapshot.select.statement.overrides optional @@ -183,12 +193,123 @@ Connector Options
    For example: 'debezium.snapshot.select.statement.overrides.schema.table' = 'select * from schema.table where to_char(rq, 'yyyy-MM-dd')'. + + scan.incremental.snapshot.enabled + optional + false + Boolean + Incremental snapshot is a new mechanism to read snapshot of a table. Compared to the old snapshot mechanism, + the incremental snapshot has many advantages, including: + (1) source can be parallel during snapshot reading, + (2) source can perform checkpoints in the chunk granularity during snapshot reading, + (3) source doesn't need to acquire global read lock (FLUSH TABLES WITH READ LOCK) before snapshot reading. + Please see Incremental Snapshot Readingsection for more detailed information. + + - + +
    Note: `slot.name` is recommended to set for different tables to avoid the potential `PSQLException: ERROR: replication slot "flink" is active for PID 974` error. See more [here](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#postgresql-property-slot-name). +### Incremental Snapshot Options + +The following options is available only when `scan.incremental.snapshot.enabled=true`: + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    OptionRequiredDefaultTypeDescription
    scan.incremental.snapshot.chunk.sizeoptional8096IntegerThe chunk size (number of rows) of table snapshot, captured tables are split into multiple chunks when read the snapshot of table.
    scan.startup.modeoptionalinitialStringOptional startup mode for Postgres CDC consumer, valid enumerations are "initial" + and "latest-offset". + Please see Startup Reading Position section for more detailed information.
    chunk-meta.group.sizeoptional1000IntegerThe group size of chunk meta, if the meta size exceeds the group size, the meta will be divided into multiple groups.
    connect.timeoutoptional30sDurationThe maximum time that the connector should wait after trying to connect to the PostgreSQL database server before timing out.
    connect.pool.sizeoptional30IntegerThe connection pool size.
    connect.max-retriesoptional3IntegerThe max retry times that the connector should retry to build database server connection.
    scan.snapshot.fetch.sizeoptional1024IntegerThe maximum fetch size for per poll when read table snapshot.
    scan.incremental.snapshot.chunk.key-columnoptional(none)StringThe chunk key of table snapshot, captured tables are split into multiple chunks by a chunk key when read the snapshot of table. + By default, the chunk key is the first column of the primary key. This column must be a column of the primary key.
    chunk-key.even-distribution.factor.lower-boundoptional0.05dDoubleThe lower bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not. + The table chunks would use evenly calculation optimization when the data distribution is even, and the query for splitting would happen when it is uneven. + The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount.
    chunk-key.even-distribution.factor.upper-boundoptional1000.0dDoubleThe upper bound of chunk key distribution factor. The distribution factor is used to determine whether the table is evenly distribution or not. + The table chunks would use evenly calculation optimization when the data distribution is even, and the query for splitting would happen when it is uneven. + The distribution factor could be calculated by (MAX(id) - MIN(id) + 1) / rowCount.
    +
    + Available Metadata ---------------- @@ -212,7 +333,7 @@ The following format metadata can be exposed as read-only (VIRTUAL) columns in a schema_name STRING NOT NULL Name of the schema that contain the row. - + database_name STRING NOT NULL @@ -229,7 +350,10 @@ The following format metadata can be exposed as read-only (VIRTUAL) columns in a Limitation -------- -### Can't perform checkpoint during scanning snapshot of tables +### Can't perform checkpoint during scanning snapshot of tables when incremental snapshot is disabled + +When `scan.incremental.snapshot.enabled=false`, we have the following limitation. + During scanning snapshot of database tables, since there is no recoverable position, we can't perform checkpoints. In order to not perform checkpoints, Postgres CDC source will keep the checkpoint waiting to timeout. The timeout checkpoint will be recognized as failed checkpoint, by default, this will trigger a failover for the Flink job. So if the database table is large, it is recommended to add following Flink configurations to avoid failover because of the timeout checkpoints: ``` @@ -265,17 +389,77 @@ CREATE TABLE products ( Features -------- -### Exactly-Once Processing +### Incremental Snapshot Reading (Experimental) -The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#how-the-postgresql-connector-works). +Incremental snapshot reading is a new mechanism to read snapshot of a table. Compared to the old snapshot mechanism, the incremental snapshot has many advantages, including: +* (1) PostgreSQL CDC Source can be parallel during snapshot reading +* (2) PostgreSQL CDC Source can perform checkpoints in the chunk granularity during snapshot reading +* (3) PostgreSQL CDC Source doesn't need to acquire global read lock before snapshot reading -### Single Thread Reading +During the incremental snapshot reading, the PostgreSQL CDC Source firstly splits snapshot chunks (splits) by primary key of table, +and then PostgreSQL CDC Source assigns the chunks to multiple readers to read the data of snapshot chunk. -The Postgres CDC source can't work in parallel reading, because there is only one task can receive binlog events. +### Exactly-Once Processing + +The Postgres CDC connector is a Flink Source connector which will read database snapshot first and then continues to read binlogs with **exactly-once processing** even failures happen. Please read [How the connector works](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#how-the-postgresql-connector-works). ### DataStream Source -The Postgres CDC connector can also be a DataStream source. You can create a SourceFunction as the following shows: +The Postgres CDC connector can also be a DataStream source. There are two modes for the DataStream source: + +- incremental snapshot based, which allows parallel reading +- SourceFunction based, which only supports single thread reading + +#### Incremental Snapshot based DataStream (Experimental) + +```java +import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import com.ververica.cdc.connectors.postgres.source.PostgresSourceBuilder; +import com.ververica.cdc.debezium.DebeziumDeserializationSchema; +import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; + +public class PostgresParallelSourceExample { + + public static void main(String[] args) throws Exception { + + DebeziumDeserializationSchema deserializer = + new JsonDebeziumDeserializationSchema(); + + JdbcIncrementalSource postgresIncrementalSource = + new PostgresSourceBuilder() + .hostname("localhost") + .port(5432) + .database("postgres") + .schemaList("inventory") + .tableList("inventory.products") + .username("postgres") + .password("postgres") + .slotName("flink") + .decodingPluginName("decoderbufs") // use pgoutput for PostgreSQL 10+ + .deserializer(deserializer) + .includeSchemaChanges(true) // output the schema changes as well + .splitSize(2) // the split size of each snapshot split + .build(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.enableCheckpointing(3000); + + env.fromSource( + postgresIncrementalSource, + WatermarkStrategy.noWatermarks(), + "PostgresParallelSource") + .setParallelism(2) + .print(); + + env.execute("Output Postgres Snapshot"); + } +} +``` + +#### SourceFunction-based DataStream ```java import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java index 27dbf95452..7167d2a438 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/config/JdbcSourceConfig.java @@ -36,6 +36,7 @@ public abstract class JdbcSourceConfig extends BaseSourceConfig { protected final String username; protected final String password; protected final List databaseList; + protected final List schemaList; protected final List tableList; protected final int fetchSize; protected final String serverTimeZone; @@ -47,6 +48,7 @@ public abstract class JdbcSourceConfig extends BaseSourceConfig { public JdbcSourceConfig( StartupOptions startupOptions, List databaseList, + List schemaList, List tableList, int splitSize, int splitMetaGroupSize, @@ -83,6 +85,7 @@ public JdbcSourceConfig( this.username = username; this.password = password; this.databaseList = databaseList; + this.schemaList = schemaList; this.tableList = tableList; this.fetchSize = fetchSize; this.serverTimeZone = serverTimeZone; diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java index ab6ba1d89f..35d43d335d 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/dialect/DataSourceDialect.java @@ -17,6 +17,7 @@ package com.ververica.cdc.connectors.base.dialect; import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.state.CheckpointListener; import com.ververica.cdc.connectors.base.config.SourceConfig; import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; @@ -36,7 +37,8 @@ * @param The source config of data source. */ @Experimental -public interface DataSourceDialect extends Serializable { +public interface DataSourceDialect + extends Serializable, CheckpointListener { /** Get the name of dialect. */ String getName(); @@ -68,4 +70,13 @@ public interface DataSourceDialect extends Serializable /** The task context used for fetch task to fetch data from external systems. */ FetchTask.Context createFetchTaskContext(SourceSplitBase sourceSplitBase, C sourceConfig); + + /** + * We have an empty default implementation here because most dialects do not have to implement + * the method. + * + * @see CheckpointListener#notifyCheckpointComplete(long) + */ + @Override + default void notifyCheckpointComplete(long checkpointId) throws Exception {} } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java index ac0f8ad5fa..d567d3d785 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/options/SourceOptions.java @@ -35,9 +35,8 @@ public class SourceOptions { + "(1) source can be parallel during snapshot reading, \n" + "(2) source can perform checkpoints in the chunk granularity during snapshot reading, \n" + "(3) source doesn't need to acquire global read lock (FLUSH TABLES WITH READ LOCK) before snapshot reading.\n" - + "If you would like the source run in parallel, each parallel reader should have an unique server id, " + + "For MySQL, if you would like the source run in parallel, each parallel reader should have an unique server id, " + "so the 'server-id' must be a range like '5400-6400', and the range must be larger than the parallelism."); - public static final ConfigOption SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE = ConfigOptions.key("scan.incremental.snapshot.chunk.size") .intType() diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java index 51e9787440..0f4ad56749 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/relational/connection/JdbcConnectionPoolFactory.java @@ -40,9 +40,10 @@ public HikariDataSource createPooledDataSource(JdbcSourceConfig sourceConfig) { config.setMinimumIdle(MINIMUM_POOL_SIZE); config.setMaximumPoolSize(sourceConfig.getConnectionPoolSize()); config.setConnectionTimeout(sourceConfig.getConnectTimeout().toMillis()); - config.addDataSourceProperty(SERVER_TIMEZONE_KEY, sourceConfig.getServerTimeZone()); config.setDriverClassName(sourceConfig.getDriverClassName()); + // note: the following properties should be optional (only applied to MySQL) + config.addDataSourceProperty(SERVER_TIMEZONE_KEY, sourceConfig.getServerTimeZone()); // optional optimization configurations for pooled DataSource config.addDataSourceProperty("cachePrepStmts", "true"); config.addDataSourceProperty("prepStmtCacheSize", "250"); diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java index d47184d975..bd993de8e2 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/HybridSplitAssigner.java @@ -113,15 +113,23 @@ public Optional getNext() { // stream split assigning if (isStreamSplitAssigned) { // no more splits for the assigner + LOG.trace( + "No more splits for the SnapshotSplitAssigner. StreamSplit is already assigned."); return Optional.empty(); } else if (snapshotSplitAssigner.isFinished()) { // we need to wait snapshot-assigner to be finished before // assigning the stream split. Otherwise, records emitted from stream split // might be out-of-order in terms of same primary key with snapshot splits. isStreamSplitAssigned = true; - return Optional.of(createStreamSplit()); + StreamSplit streamSplit = createStreamSplit(); + LOG.trace( + "SnapshotSplitAssigner is finished: creating a new stream split {}", + streamSplit); + return Optional.of(streamSplit); } else { // stream split is not ready by now + LOG.trace( + "Waiting for SnapshotSplitAssigner to be finished before assigning a new stream split."); return Optional.empty(); } } else { diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java index 7acd3ea3b2..4ab00a5e67 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/assigner/state/PendingSplitsStateSerializer.java @@ -25,6 +25,7 @@ import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitSerializer; +import com.ververica.cdc.connectors.base.utils.SerializerUtils; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; @@ -41,7 +42,7 @@ /** The {@link SimpleVersionedSerializer Serializer} for the {@link PendingSplitsState}. */ public class PendingSplitsStateSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 4; + private static final int VERSION = 5; private static final ThreadLocal SERIALIZER_CACHE = ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); @@ -100,6 +101,7 @@ public PendingSplitsState deserialize(int version, byte[] serialized) throws IOE return deserializeLegacyPendingSplitsState(serialized); case 3: case 4: + case 5: return deserializePendingSplitsState(version, serialized); default: throw new IOException("Unknown version: " + version); @@ -173,7 +175,7 @@ private void serializeStreamPendingSplitsState( private SnapshotPendingSplitsState deserializeLegacySnapshotPendingSplitsState( int splitVersion, DataInputDeserializer in) throws IOException { - List alreadyProcessedTables = readTableIds(in); + List alreadyProcessedTables = readTableIds(2, in); List remainingSplits = readSnapshotSplits(splitVersion, in); Map assignedSnapshotSplits = readAssignedSnapshotSplits(splitVersion, in); @@ -219,13 +221,13 @@ private HybridPendingSplitsState deserializeLegacyHybridPendingSplitsState( private SnapshotPendingSplitsState deserializeSnapshotPendingSplitsState( int version, int splitVersion, DataInputDeserializer in) throws IOException { - List alreadyProcessedTables = readTableIds(in); + List alreadyProcessedTables = readTableIds(version, in); List remainingSplits = readSnapshotSplits(splitVersion, in); Map assignedSnapshotSplits = readAssignedSnapshotSplits(splitVersion, in); Map finishedOffsets = readFinishedOffsets(splitVersion, in); boolean isAssignerFinished = in.readBoolean(); - List remainingTableIds = readTableIds(in); + List remainingTableIds = readTableIds(version, in); boolean isTableIdCaseSensitive = in.readBoolean(); final List remainingSchemalessSplits = new ArrayList<>(); final Map assignedSchemalessSnapshotSplits = @@ -358,16 +360,22 @@ private void writeTableIds(Collection tableIds, DataOutputSerializer ou final int size = tableIds.size(); out.writeInt(size); for (TableId tableId : tableIds) { + boolean useCatalogBeforeSchema = SerializerUtils.shouldUseCatalogBeforeSchema(tableId); + out.writeBoolean(useCatalogBeforeSchema); out.writeUTF(tableId.toString()); } } - private List readTableIds(DataInputDeserializer in) throws IOException { + private List readTableIds(int version, DataInputDeserializer in) throws IOException { List tableIds = new ArrayList<>(); final int size = in.readInt(); for (int i = 0; i < size; i++) { + boolean useCatalogBeforeSchema = true; + if (version >= 5) { + useCatalogBeforeSchema = in.readBoolean(); + } String tableIdStr = in.readUTF(); - tableIds.add(TableId.parse(tableIdStr)); + tableIds.add(TableId.parse(tableIdStr, useCatalogBeforeSchema)); } return tableIds; } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java index c05837f6a0..d5262ac2da 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/offset/OffsetDeserializerSerializer.java @@ -47,6 +47,7 @@ default Offset readOffsetPosition(int offsetVersion, DataInputDeserializer in) : null; case 2: case 3: + case 4: return readOffsetPosition(in); default: throw new IOException("Unknown version: " + offsetVersion); @@ -82,12 +83,17 @@ default OffsetDeserializer createOffsetDeserializer() { default FinishedSnapshotSplitInfo deserialize(byte[] serialized) { try { final DataInputDeserializer in = new DataInputDeserializer(serialized); - TableId tableId = TableId.parse(in.readUTF()); + String tableIdStr = in.readUTF(); String splitId = in.readUTF(); Object[] splitStart = serializedStringToRow(in.readUTF()); Object[] splitEnd = serializedStringToRow(in.readUTF()); OffsetFactory offsetFactory = (OffsetFactory) serializedStringToObject(in.readUTF()); Offset highWatermark = readOffsetPosition(in); + boolean useCatalogBeforeSchema = true; + if (in.available() > 0) { + useCatalogBeforeSchema = in.readBoolean(); + } + TableId tableId = TableId.parse(tableIdStr, useCatalogBeforeSchema); in.releaseArrays(); return new FinishedSnapshotSplitInfo( diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java index 6b62b98df6..e86b5c8d73 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/FinishedSnapshotSplitInfo.java @@ -143,6 +143,9 @@ public byte[] serialize(final DataOutputSerializer out) throws IOException { out.writeUTF(SerializerUtils.rowToSerializedString(this.getSplitEnd())); out.writeUTF(SerializerUtils.rowToSerializedString(this.offsetFactory)); writeOffsetPosition(this.getHighWatermark(), out); + boolean useCatalogBeforeSchema = + SerializerUtils.shouldUseCatalogBeforeSchema(this.getTableId()); + out.writeBoolean(useCatalogBeforeSchema); return out.getCopyOfBuffer(); } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java index 36b5cc166c..962e7204f3 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/meta/split/SourceSplitSerializer.java @@ -44,7 +44,7 @@ public abstract class SourceSplitSerializer implements SimpleVersionedSerializer, OffsetDeserializerSerializer { - private static final int VERSION = 3; + private static final int VERSION = 4; private static final ThreadLocal SERIALIZER_CACHE = ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); @@ -67,6 +67,9 @@ public byte[] serialize(SourceSplitBase split) throws IOException { final DataOutputSerializer out = SERIALIZER_CACHE.get(); out.writeInt(SNAPSHOT_SPLIT_FLAG); + boolean useCatalogBeforeSchema = + SerializerUtils.shouldUseCatalogBeforeSchema(snapshotSplit.getTableId()); + out.writeBoolean(useCatalogBeforeSchema); out.writeUTF(snapshotSplit.getTableId().toString()); out.writeUTF(snapshotSplit.splitId()); out.writeUTF(snapshotSplit.getSplitKeyType().asSerializableString()); @@ -114,6 +117,7 @@ public SourceSplitBase deserialize(int version, byte[] serialized) throws IOExce case 1: case 2: case 3: + case 4: return deserializeSplit(version, serialized); default: throw new IOException("Unknown version: " + version); @@ -125,7 +129,11 @@ public SourceSplitBase deserializeSplit(int version, byte[] serialized) throws I int splitKind = in.readInt(); if (splitKind == SNAPSHOT_SPLIT_FLAG) { - TableId tableId = TableId.parse(in.readUTF()); + boolean useCatalogBeforeSchema = true; + if (version >= 4) { + useCatalogBeforeSchema = in.readBoolean(); + } + TableId tableId = TableId.parse(in.readUTF(), useCatalogBeforeSchema); String splitId = in.readUTF(); RowType splitKeyType = (RowType) LogicalTypeParser.parse(in.readUTF()); Object[] splitBoundaryStart = SerializerUtils.serializedStringToRow(in.readUTF()); @@ -174,6 +182,9 @@ public static void writeTableSchemas( final int size = tableSchemas.size(); out.writeInt(size); for (Map.Entry entry : tableSchemas.entrySet()) { + boolean useCatalogBeforeSchema = + SerializerUtils.shouldUseCatalogBeforeSchema(entry.getKey()); + out.writeBoolean(useCatalogBeforeSchema); out.writeUTF(entry.getKey().toString()); final String tableChangeStr = documentWriter.write(jsonSerializer.toDocument(entry.getValue())); @@ -189,7 +200,8 @@ public static Map readTableSchemas(int version, DataInputD Map tableSchemas = new HashMap<>(); final int size = in.readInt(); for (int i = 0; i < size; i++) { - TableId tableId = TableId.parse(in.readUTF()); + boolean useCatalogBeforeSchema = in.readBoolean(); + TableId tableId = TableId.parse(in.readUTF(), useCatalogBeforeSchema); final String tableChangeStr; switch (version) { case 1: @@ -197,6 +209,7 @@ public static Map readTableSchemas(int version, DataInputD break; case 2: case 3: + case 4: final int len = in.readInt(); final byte[] bytes = new byte[len]; in.read(bytes); @@ -227,13 +240,18 @@ private List readFinishedSplitsInfo( List finishedSplitsInfo = new ArrayList<>(); final int size = in.readInt(); for (int i = 0; i < size; i++) { - TableId tableId = TableId.parse(in.readUTF()); + String tableIdStr = in.readUTF(); String splitId = in.readUTF(); Object[] splitStart = SerializerUtils.serializedStringToRow(in.readUTF()); Object[] splitEnd = SerializerUtils.serializedStringToRow(in.readUTF()); OffsetFactory offsetFactory = (OffsetFactory) SerializerUtils.serializedStringToObject(in.readUTF()); Offset highWatermark = readOffsetPosition(version, in); + boolean useCatalogBeforeSchema = true; + if (version >= 4) { + useCatalogBeforeSchema = in.readBoolean(); + } + TableId tableId = TableId.parse(tableIdStr, useCatalogBeforeSchema); finishedSplitsInfo.add( new FinishedSnapshotSplitInfo( diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java index 921c61c74d..fd02dcd644 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceReader.java @@ -132,6 +132,11 @@ public List snapshotState(long checkpointId) { return stateSplits; } + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + dialect.notifyCheckpointComplete(checkpointId); + } + @Override protected void onSplitFinished(Map finishedSplitIds) { for (SourceSplitState splitState : finishedSplitIds.values()) { diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java index 82bc56f7dd..36b210626f 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/IncrementalSourceRecordEmitter.java @@ -44,6 +44,7 @@ import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getHistoryRecord; import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.getMessageTimestamp; import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isDataChangeRecord; +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isHeartbeatEvent; import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.isSchemaChangeEvent; /** @@ -91,11 +92,14 @@ protected void processElement( SourceRecord element, SourceOutput output, SourceSplitState splitState) throws Exception { if (isWatermarkEvent(element)) { + LOG.trace("Process WatermarkEvent: {}; splitState = {}", element, splitState); Offset watermark = getWatermark(element); if (isHighWatermarkEvent(element) && splitState.isSnapshotSplitState()) { + LOG.trace("Set HighWatermark {} for {}", watermark, splitState); splitState.asSnapshotSplitState().setHighWatermark(watermark); } } else if (isSchemaChangeEvent(element) && splitState.isStreamSplitState()) { + LOG.trace("Process SchemaChangeEvent: {}; splitState = {}", element, splitState); HistoryRecord historyRecord = getHistoryRecord(element); Array tableChanges = historyRecord.document().getArray(HistoryRecord.Fields.TABLE_CHANGES); @@ -107,15 +111,24 @@ protected void processElement( emitElement(element, output); } } else if (isDataChangeRecord(element)) { - if (splitState.isStreamSplitState()) { - Offset position = getOffsetPosition(element); - splitState.asStreamSplitState().setStartingOffset(position); - } + LOG.trace("Process DataChangeRecord: {}; splitState = {}", element, splitState); + updateStreamSplitState(splitState, element); reportMetrics(element); emitElement(element, output); + } else if (isHeartbeatEvent(element)) { + LOG.trace("Process Heartbeat: {}; splitState = {}", element, splitState); + updateStreamSplitState(splitState, element); } else { // unknown element - LOG.info("Meet unknown element {}, just skip.", element); + LOG.info( + "Meet unknown element {} for splitState = {}, just skip.", element, splitState); + } + } + + private void updateStreamSplitState(SourceSplitState splitState, SourceRecord element) { + if (splitState.isStreamSplitState()) { + Offset position = getOffsetPosition(element); + splitState.asStreamSplitState().setStartingOffset(position); } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java index ba66fbd53a..3e53eccc3f 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/FetchTask.java @@ -44,6 +44,9 @@ public interface FetchTask { /** Returns the split that the task used. */ Split getSplit(); + /** Stops current task, most of the implementations don't need this. */ + default void stop() {} + /** Base context used in the execution of fetch task. */ interface Context { void configure(SourceSplitBase sourceSplitBase); diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java index 2d1e3e2147..df06e78d04 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/IncrementalSourceStreamFetcher.java @@ -35,6 +35,7 @@ import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -55,6 +56,7 @@ public class IncrementalSourceStreamFetcher implements Fetcher pureStreamPhaseTables; private volatile ChangeEventQueue queue; + private volatile boolean currentTaskRunning; private volatile Throwable readException; private FetchTask streamFetchTask; @@ -70,6 +72,7 @@ public IncrementalSourceStreamFetcher(FetchTask.Context taskContext, int subTask ThreadFactory threadFactory = new ThreadFactoryBuilder().setNameFormat("debezium-reader-" + subTaskId).build(); this.executorService = Executors.newSingleThreadExecutor(threadFactory); + this.currentTaskRunning = true; this.pureStreamPhaseTables = new HashSet<>(); } @@ -85,6 +88,7 @@ public void submitTask(FetchTask fetchTask) { try { streamFetchTask.execute(taskContext); } catch (Exception e) { + this.currentTaskRunning = false; LOG.error( String.format( "Execute stream read task for stream split %s fail", @@ -97,7 +101,7 @@ public void submitTask(FetchTask fetchTask) { @Override public boolean isFinished() { - return currentStreamSplit == null || !streamFetchTask.isRunning(); + return currentStreamSplit == null || !currentTaskRunning; } @Nullable @@ -105,7 +109,7 @@ public boolean isFinished() { public Iterator pollSplitRecords() throws InterruptedException { checkReadException(); final List sourceRecords = new ArrayList<>(); - if (streamFetchTask.isRunning()) { + if (currentTaskRunning) { List batch = queue.poll(); for (DataChangeEvent event : batch) { if (shouldEmit(event.getRecord())) { @@ -114,10 +118,12 @@ public Iterator pollSplitRecords() throws InterruptedException { LOG.debug("{} data change event should not emit", event); } } + List sourceRecordsSet = new ArrayList<>(); + sourceRecordsSet.add(new SourceRecords(sourceRecords)); + return sourceRecordsSet.iterator(); + } else { + return Collections.emptyIterator(); } - List sourceRecordsSet = new ArrayList<>(); - sourceRecordsSet.add(new SourceRecords(sourceRecords)); - return sourceRecordsSet.iterator(); } private void checkReadException() { @@ -133,6 +139,8 @@ private void checkReadException() { @Override public void close() { try { + // gracefully stop streamFetchTask, e.g. during shutdown + stopReadTask(); if (executorService != null) { executorService.shutdown(); if (!executorService.awaitTermination( @@ -235,4 +243,11 @@ private void configureFilter() { this.maxSplitHighWatermarkMap = tableIdOffsetPositionMap; this.pureStreamPhaseTables.clear(); } + + public void stopReadTask() { + this.currentTaskRunning = false; + if (streamFetchTask != null) { + streamFetchTask.stop(); + } + } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java index 6dda90c4d7..c98d259a65 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/source/reader/external/JdbcSourceFetchTaskContext.java @@ -58,18 +58,17 @@ public JdbcSourceFetchTaskContext( @Override public TableId getTableId(SourceRecord record) { - return null; + return SourceRecordUtils.getTableId(record); } @Override public boolean isDataChangeRecord(SourceRecord record) { - return false; + return SourceRecordUtils.isDataChangeRecord(record); } @Override public boolean isRecordBetween(SourceRecord record, Object[] splitStart, Object[] splitEnd) { - RowType splitKeyType = - getSplitType(getDatabaseSchema().tableFor(SourceRecordUtils.getTableId(record))); + RowType splitKeyType = getSplitType(getDatabaseSchema().tableFor(this.getTableId(record))); Object[] key = SourceRecordUtils.getSplitKey(splitKeyType, record, getSchemaNameAdjuster()); return SourceRecordUtils.splitKeyRangeContains(key, splitStart, splitEnd); } @@ -157,7 +156,7 @@ public CommonConnectorConfig getDbzConnectorConfig() { } public SchemaNameAdjuster getSchemaNameAdjuster() { - return schemaNameAdjuster; + return SchemaNameAdjuster.create(); } public abstract RelationalDatabaseSchema getDatabaseSchema(); diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java index b475fdb71f..6a69de91fa 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SerializerUtils.java @@ -17,6 +17,7 @@ package com.ververica.cdc.connectors.base.utils; import io.debezium.DebeziumException; +import io.debezium.relational.TableId; import io.debezium.util.HexConverter; import java.io.ByteArrayInputStream; @@ -77,4 +78,14 @@ public static Object serializedStringToObject(String serialized) { e); } } + + /** + * Check whether the catalog should be used before the schema. + * + * @param tableId the table id + * @return false if catalog is not defined but the schema is defined; otherwise return true + */ + public static boolean shouldUseCatalogBeforeSchema(TableId tableId) { + return !(tableId.catalog() == null && tableId.schema() != null); + } } diff --git a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java index 9087e520ca..aa006a4621 100644 --- a/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java +++ b/flink-cdc-base/src/main/java/com/ververica/cdc/connectors/base/utils/SourceRecordUtils.java @@ -46,6 +46,8 @@ private SourceRecordUtils() {} public static final String SCHEMA_CHANGE_EVENT_KEY_NAME = "io.debezium.connector.mysql.SchemaChangeKey"; + public static final String SCHEMA_HEARTBEAT_EVENT_KEY_NAME = + "io.debezium.connector.common.Heartbeat"; private static final DocumentReader DOCUMENT_READER = DocumentReader.defaultReader(); /** Converts a {@link ResultSet} row to an array of Objects. */ @@ -99,11 +101,18 @@ public static boolean isSchemaChangeEvent(SourceRecord sourceRecord) { public static boolean isDataChangeRecord(SourceRecord record) { Schema valueSchema = record.valueSchema(); Struct value = (Struct) record.value(); - return valueSchema != null + return value != null + && valueSchema != null && valueSchema.field(Envelope.FieldName.OPERATION) != null && value.getString(Envelope.FieldName.OPERATION) != null; } + public static boolean isHeartbeatEvent(SourceRecord record) { + Schema valueSchema = record.valueSchema(); + return valueSchema != null + && SCHEMA_HEARTBEAT_EVENT_KEY_NAME.equalsIgnoreCase(valueSchema.name()); + } + public static TableId getTableId(SourceRecord dataRecord) { Struct value = (Struct) dataRecord.value(); Struct source = value.getStruct(Envelope.FieldName.SOURCE); diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChunkSplitter.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChunkSplitter.java index 1a281fbc81..16126e286f 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChunkSplitter.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/MySqlChunkSplitter.java @@ -343,7 +343,7 @@ private static String splitId(TableId tableId, int chunkId) { } private static void maySleep(int count, TableId tableId) { - // every 100 queries to sleep 1s + // every 10 queries to sleep 100ms if (count % 10 == 0) { try { Thread.sleep(100); diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java index 4e79dd0c62..d55606770b 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/config/MySqlSourceConfig.java @@ -61,6 +61,7 @@ public MySqlSourceConfig( super( startupOptions, databaseList, + null, tableList, splitSize, splitMetaGroupSize, diff --git a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java index 2564a258a2..04993f1f2f 100644 --- a/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java +++ b/flink-cdc-base/src/test/java/com/ververica/cdc/connectors/base/experimental/fetch/MySqlScanFetchTask.java @@ -276,7 +276,7 @@ protected SnapshotResult doExecute( "Snapshot step 3 - Determining high watermark {} for split {}", highWatermark, snapshotSplit); - ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(lowWatermark); + ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); dispatcher.dispatchWatermarkEvent( offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); return SnapshotResult.completed(ctx.offset); diff --git a/flink-cdc-e2e-tests/pom.xml b/flink-cdc-e2e-tests/pom.xml index 5a63d3b43d..620c4c018c 100644 --- a/flink-cdc-e2e-tests/pom.xml +++ b/flink-cdc-e2e-tests/pom.xml @@ -35,6 +35,7 @@ under the License. 1.17.0 3.1.0-1.17 8.0.27 + 42.5.1 @@ -54,7 +55,8 @@ under the License. org.postgresql postgresql - 42.2.26 + ${postgresql.driver.version} + test com.ibm.db2.jcc @@ -211,6 +213,16 @@ under the License. + + org.postgresql + postgresql + ${postgresql.driver.version} + postgresql-driver.jar + jar + ${project.build.directory}/dependencies + + + org.apache.flink flink-connector-jdbc_2.11 diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java index dabae3453e..111284e6bc 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/PostgresE2eITCase.java @@ -43,6 +43,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.junit.Assert.assertNotNull; import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT; @@ -56,12 +57,24 @@ public class PostgresE2eITCase extends FlinkContainerTestEnvironment { protected static final String PG_DRIVER_CLASS = "org.postgresql.Driver"; private static final String INTER_CONTAINER_PG_ALIAS = "postgres"; private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + + // debezium/postgres:9.6-alpine supports both ARM and AMD architectures private static final DockerImageName PG_IMAGE = - DockerImageName.parse("debezium/postgres:9.6").asCompatibleSubstituteFor("postgres"); + DockerImageName.parse("debezium/postgres:9.6-alpine") + .asCompatibleSubstituteFor("postgres"); private static final Path postgresCdcJar = TestUtils.getResource("postgres-cdc-connector.jar"); private static final Path mysqlDriverJar = TestUtils.getResource("mysql-driver.jar"); + private static final String FLINK_PROPERTIES = + String.join( + "\n", + Arrays.asList( + "jobmanager.rpc.address: jobmanager", + "taskmanager.numberOfTaskSlots: 10", + "parallelism.default: 1", + "execution.checkpointing.interval: 10000")); + @ClassRule public static final PostgreSQLContainer POSTGRES = new PostgreSQLContainer<>(PG_IMAGE) @@ -76,6 +89,7 @@ public class PostgresE2eITCase extends FlinkContainerTestEnvironment { public void before() { super.before(); initializePostgresTable("postgres_inventory"); + overrideFlinkProperties(FLINK_PROPERTIES); } @After @@ -83,54 +97,105 @@ public void after() { super.after(); } + List sourceSql = + Arrays.asList( + "CREATE TABLE products_source (", + " `id` INT NOT NULL,", + " name STRING,", + " description STRING,", + " weight DECIMAL(10,3),", + " primary key (`id`) not enforced", + ") WITH (", + " 'connector' = 'postgres-cdc',", + " 'hostname' = '" + INTER_CONTAINER_PG_ALIAS + "',", + " 'port' = '" + POSTGRESQL_PORT + "',", + " 'username' = '" + PG_TEST_USER + "',", + " 'password' = '" + PG_TEST_PASSWORD + "',", + " 'database-name' = '" + POSTGRES.getDatabaseName() + "',", + " 'schema-name' = 'inventory',", + " 'table-name' = 'products',", + " 'slot.name' = 'flink',", // dropping the slot allows WAL segments to be + // discarded by the database + " 'debezium.slot.drop_on_stop' = 'true'", + ");"); + + List sourceSqlWithIncrementalSnapshot = + Arrays.asList( + "CREATE TABLE products_source (", + " `id` INT NOT NULL,", + " name STRING,", + " description STRING,", + " weight DECIMAL(10,3),", + " primary key (`id`) not enforced", + ") WITH (", + " 'connector' = 'postgres-cdc',", + " 'hostname' = '" + INTER_CONTAINER_PG_ALIAS + "',", + " 'port' = '" + POSTGRESQL_PORT + "',", + " 'username' = '" + PG_TEST_USER + "',", + " 'password' = '" + PG_TEST_PASSWORD + "',", + " 'database-name' = '" + POSTGRES.getDatabaseName() + "',", + " 'schema-name' = 'inventory',", + " 'table-name' = 'products',", + " 'slot.name' = 'flink_incremental',", + " 'scan.incremental.snapshot.chunk.size' = '5',", + " 'scan.incremental.snapshot.enabled' = 'true',", + " 'scan.startup.mode' = 'initial'", + ");"); + List sinkSql = + Arrays.asList( + "CREATE TABLE products_sink (", + " `id` INT NOT NULL,", + " name STRING,", + " description STRING,", + " weight DECIMAL(10,3),", + " primary key (`id`) not enforced", + ") WITH (", + " 'connector' = 'jdbc',", + String.format( + " 'url' = 'jdbc:mysql://%s:3306/%s',", + INTER_CONTAINER_MYSQL_ALIAS, mysqlInventoryDatabase.getDatabaseName()), + " 'table-name' = 'products_sink',", + " 'username' = '" + MYSQL_TEST_USER + "',", + " 'password' = '" + MYSQL_TEST_PASSWORD + "'", + ");", + "INSERT INTO products_sink", + "SELECT * FROM products_source;"); + @Test - public void testPostgresCDC() throws Exception { + public void testPostgresCdcIncremental() throws Exception { + try (Connection conn = getPgJdbcConnection(); + Statement statement = conn.createStatement()) { + // gather the initial statistics of the table for splitting + statement.execute("ANALYZE;"); + } + List sqlLines = - Arrays.asList( - "CREATE TABLE products_source (", - " `id` INT NOT NULL,", - " name STRING,", - " description STRING,", - " weight DECIMAL(10,3),", - " primary key (`id`) not enforced", - ") WITH (", - " 'connector' = 'postgres-cdc',", - " 'hostname' = '" + INTER_CONTAINER_PG_ALIAS + "',", - " 'port' = '" + POSTGRESQL_PORT + "',", - " 'username' = '" + PG_TEST_USER + "',", - " 'password' = '" + PG_TEST_PASSWORD + "',", - " 'database-name' = '" + POSTGRES.getDatabaseName() + "',", - " 'schema-name' = 'inventory',", - " 'table-name' = 'products',", - " 'slot.name' = 'flink',", - // dropping the slot allows WAL segments to be discarded by the database - " 'debezium.slot.drop_on_stop' = 'true'", - ");", - "CREATE TABLE products_sink (", - " `id` INT NOT NULL,", - " name STRING,", - " description STRING,", - " weight DECIMAL(10,3),", - " primary key (`id`) not enforced", - ") WITH (", - " 'connector' = 'jdbc',", - String.format( - " 'url' = 'jdbc:mysql://%s:3306/%s',", - INTER_CONTAINER_MYSQL_ALIAS, - mysqlInventoryDatabase.getDatabaseName()), - " 'table-name' = 'products_sink',", - " 'username' = '" + MYSQL_TEST_USER + "',", - " 'password' = '" + MYSQL_TEST_PASSWORD + "'", - ");", - "INSERT INTO products_sink", - "SELECT * FROM products_source;"); + Stream.concat(sourceSqlWithIncrementalSnapshot.stream(), sinkSql.stream()) + .collect(Collectors.toList()); + testPostgresCDC(sqlLines); + } + + @Test + public void testPostgresCdcNonIncremental() throws Exception { + + List sqlLines = + Stream.concat(sourceSql.stream(), sinkSql.stream()).collect(Collectors.toList()); + testPostgresCDC(sqlLines); + } + + public void testPostgresCDC(List sqlLines) throws Exception { submitSQLJob(sqlLines, postgresCdcJar, jdbcJar, mysqlDriverJar); waitUntilJobRunning(Duration.ofSeconds(30)); + // wait a bit to make sure the replication slot is ready + Thread.sleep(30000); - // generate binlogs + // generate WAL try (Connection conn = getPgJdbcConnection(); Statement statement = conn.createStatement()) { + + // at this point, the replication slot 'flink' should already be created; otherwise, the + // test will fail statement.execute( "UPDATE inventory.products SET description='18oz carpenter hammer' WHERE id=106;"); statement.execute("UPDATE inventory.products SET weight='5.1' WHERE id=107;"); diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java index 52ae9e04f9..d087563326 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/FlinkContainerTestEnvironment.java @@ -27,6 +27,7 @@ import org.apache.flink.util.TestLogger; import com.ververica.cdc.connectors.mysql.testutils.MySqlContainer; +import com.ververica.cdc.connectors.mysql.testutils.MySqlVersion; import com.ververica.cdc.connectors.mysql.testutils.UniqueDatabase; import org.junit.After; import org.junit.Before; @@ -81,6 +82,7 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger { "jobmanager.rpc.address: jobmanager", "taskmanager.numberOfTaskSlots: 10", "parallelism.default: 4", + "execution.checkpointing.interval: 10000", // this is needed for oracle-cdc tests. // see https://stackoverflow.com/a/47062742/4915129 "env.java.opts: -Doracle.jdbc.timezoneAsRegion=false")); @@ -102,7 +104,8 @@ public abstract class FlinkContainerTestEnvironment extends TestLogger { @ClassRule public static final MySqlContainer MYSQL = (MySqlContainer) - new MySqlContainer() + new MySqlContainer( + MySqlVersion.V8_0) // v8 support both ARM and AMD architectures .withConfigurationOverride("docker/mysql/my.cnf") .withSetupSQL("docker/mysql/setup.sql") .withDatabaseName("flink-test") @@ -169,6 +172,12 @@ public void after() { mysqlInventoryDatabase.dropDatabase(); } + /** Allow overriding the default flink properties. */ + public void overrideFlinkProperties(String properties) { + jobManager.withEnv("FLINK_PROPERTIES", properties); + taskManager.withEnv("FLINK_PROPERTIES", properties); + } + /** * Submits a SQL job to the running cluster. * diff --git a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/JdbcProxy.java b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/JdbcProxy.java index 1d579dcbba..911dd97371 100644 --- a/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/JdbcProxy.java +++ b/flink-cdc-e2e-tests/src/test/java/com/ververica/cdc/connectors/tests/utils/JdbcProxy.java @@ -27,7 +27,7 @@ import java.util.Collections; import java.util.List; -import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertArrayEquals; /** Proxy to communicate with database using JDBC protocol. */ public class JdbcProxy { @@ -66,7 +66,8 @@ public void checkResult(List expectedResult, String table, String[] fiel } Collections.sort(results); Collections.sort(expectedResult); - assertEquals(expectedResult, results); + // make it easier to check the result + assertArrayEquals(expectedResult.toArray(), results.toArray()); } } diff --git a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java index 98bfa663b5..401a943aba 100644 --- a/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java +++ b/flink-connector-mongodb-cdc/src/main/java/com/ververica/cdc/connectors/mongodb/source/reader/fetch/MongoDBStreamFetchTask.java @@ -48,6 +48,7 @@ import org.slf4j.LoggerFactory; import java.time.Instant; +import java.util.Map; import java.util.Optional; import static com.ververica.cdc.connectors.mongodb.internal.MongoDBEnvelope.CLUSTER_TIME_FIELD; @@ -223,8 +224,8 @@ public StreamSplit getSplit() { private MongoChangeStreamCursor openChangeStreamCursor( ChangeStreamDescriptor changeStreamDescriptor) { ChangeStreamOffset offset = - new ChangeStreamOffset(streamSplit.getStartingOffset().getOffset()); - + new ChangeStreamOffset( + (Map) streamSplit.getStartingOffset().getOffset()); ChangeStreamIterable changeStreamIterable = getChangeStreamIterable(sourceConfig, changeStreamDescriptor); diff --git a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializer.java b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializer.java index 2445dc3861..91c6076f29 100644 --- a/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializer.java +++ b/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/assigners/state/PendingSplitsStateSerializer.java @@ -49,6 +49,7 @@ */ public class PendingSplitsStateSerializer implements SimpleVersionedSerializer { + // TODO: need proper implementation of the new version private static final int VERSION = 5; private static final ThreadLocal SERIALIZER_CACHE = ThreadLocal.withInitial(() -> new DataOutputSerializer(64)); diff --git a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/config/OracleSourceConfig.java b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/config/OracleSourceConfig.java index 478611d8cc..f9cab21cc1 100644 --- a/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/config/OracleSourceConfig.java +++ b/flink-connector-oracle-cdc/src/main/java/com/ververica/cdc/connectors/oracle/source/config/OracleSourceConfig.java @@ -65,6 +65,7 @@ public OracleSourceConfig( super( startupOptions, databaseList, + null, tableList, splitSize, splitMetaGroupSize, diff --git a/flink-connector-postgres-cdc/pom.xml b/flink-connector-postgres-cdc/pom.xml index 1f50573806..865f765c2f 100644 --- a/flink-connector-postgres-cdc/pom.xml +++ b/flink-connector-postgres-cdc/pom.xml @@ -30,6 +30,18 @@ under the License. + + com.ververica + flink-cdc-base + ${project.version} + + + kafka-log4j-appender + org.apache.kafka + + + + com.ververica @@ -56,10 +68,10 @@ under the License. - + org.postgresql postgresql - 42.2.27 + 42.5.1 @@ -151,6 +163,7 @@ under the License. test + diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresChunkSplitter.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresChunkSplitter.java new file mode 100644 index 0000000000..3da43aa816 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresChunkSplitter.java @@ -0,0 +1,359 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkRange; +import com.ververica.cdc.connectors.base.source.assigner.splitter.JdbcSourceChunkSplitter; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.utils.ObjectUtils; +import com.ververica.cdc.connectors.postgres.source.utils.ChunkUtils; +import com.ververica.cdc.connectors.postgres.source.utils.PostgresQueryUtils; +import com.ververica.cdc.connectors.postgres.source.utils.PostgresTypeUtils; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.history.TableChanges.TableChange; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.math.BigDecimal; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static com.ververica.cdc.connectors.base.utils.ObjectUtils.doubleCompare; +import static java.math.BigDecimal.ROUND_CEILING; + +/** + * The splitter to split the table into chunks using primary-key (by default) or a given split key. + */ +public class PostgresChunkSplitter implements JdbcSourceChunkSplitter { + private static final Logger LOG = LoggerFactory.getLogger(PostgresChunkSplitter.class); + + private final JdbcSourceConfig sourceConfig; + private final PostgresDialect dialect; + + public PostgresChunkSplitter(JdbcSourceConfig sourceConfig, PostgresDialect postgresDialect) { + this.sourceConfig = sourceConfig; + this.dialect = postgresDialect; + } + + private static String splitId(TableId tableId, int chunkId) { + return tableId.toString() + ":" + chunkId; + } + + private static void maySleep(int count, TableId tableId) { + // every 10 queries to sleep 100ms + if (count % 10 == 0) { + try { + Thread.sleep(100); + } catch (InterruptedException e) { + // nothing to do + } + LOG.info("JdbcSourceChunkSplitter has split {} chunks for table {}", count, tableId); + } + } + + @Override + public Collection generateSplits(TableId tableId) { + try (JdbcConnection jdbc = dialect.openJdbcConnection(sourceConfig)) { + + LOG.info("Start splitting table {} into chunks...", tableId); + long start = System.currentTimeMillis(); + + Table table = + Objects.requireNonNull(dialect.queryTableSchema(jdbc, tableId)).getTable(); + Column splitColumn = ChunkUtils.getSplitColumn(table, sourceConfig.getChunkKeyColumn()); + final List chunks; + try { + chunks = splitTableIntoChunks(jdbc, tableId, splitColumn); + } catch (SQLException e) { + throw new FlinkRuntimeException("Failed to split chunks for table " + tableId, e); + } + + // convert chunks into splits + List splits = new ArrayList<>(); + RowType splitType = getSplitType(splitColumn); + for (int i = 0; i < chunks.size(); i++) { + ChunkRange chunk = chunks.get(i); + SnapshotSplit split = + createSnapshotSplit( + jdbc, + tableId, + i, + splitType, + chunk.getChunkStart(), + chunk.getChunkEnd()); + splits.add(split); + } + + long end = System.currentTimeMillis(); + LOG.info( + "Split table {} into {} chunks, time cost: {}ms.", + tableId, + splits.size(), + end - start); + return splits; + } catch (Exception e) { + throw new FlinkRuntimeException( + String.format("Generate Splits for table %s error", tableId), e); + } + } + + @Override + public Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) + throws SQLException { + return PostgresQueryUtils.queryMinMax(jdbc, tableId, columnName); + } + + @Override + public Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException { + return PostgresQueryUtils.queryMin(jdbc, tableId, columnName, excludedLowerBound); + } + + @Override + public Object queryNextChunkMax( + JdbcConnection jdbc, + TableId tableId, + String columnName, + int chunkSize, + Object includedLowerBound) + throws SQLException { + return PostgresQueryUtils.queryNextChunkMax( + jdbc, tableId, columnName, chunkSize, includedLowerBound); + } + + // -------------------------------------------------------------------------------------------- + // Utilities + // -------------------------------------------------------------------------------------------- + + @Override + public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { + return PostgresQueryUtils.queryApproximateRowCnt(jdbc, tableId); + } + + @Override + public String buildSplitScanQuery( + TableId tableId, RowType splitKeyType, boolean isFirstSplit, boolean isLastSplit) { + return PostgresQueryUtils.buildSplitScanQuery( + tableId, splitKeyType, isFirstSplit, isLastSplit); + } + + @Override + public DataType fromDbzColumn(Column splitColumn) { + return PostgresTypeUtils.fromDbzColumn(splitColumn); + } + + /** + * We can use evenly-sized chunks or unevenly-sized chunks when split table into chunks, using + * evenly-sized chunks which is much efficient, using unevenly-sized chunks which will request + * many queries and is not efficient. + */ + private List splitTableIntoChunks( + JdbcConnection jdbc, TableId tableId, Column splitColumn) throws SQLException { + final String splitColumnName = splitColumn.name(); + final Object[] minMax = queryMinMax(jdbc, tableId, splitColumnName); + final Object min = minMax[0]; + final Object max = minMax[1]; + if (min == null || max == null || min.equals(max)) { + // empty table, or only one row, return full table scan as a chunk + return Collections.singletonList(ChunkRange.all()); + } + + final int chunkSize = sourceConfig.getSplitSize(); + final double distributionFactorUpper = sourceConfig.getDistributionFactorUpper(); + final double distributionFactorLower = sourceConfig.getDistributionFactorLower(); + + if (isEvenlySplitColumn(splitColumn)) { + long approximateRowCnt = queryApproximateRowCnt(jdbc, tableId); + double distributionFactor = + calculateDistributionFactor(tableId, min, max, approximateRowCnt); + + boolean dataIsEvenlyDistributed = + doubleCompare(distributionFactor, distributionFactorLower) >= 0 + && doubleCompare(distributionFactor, distributionFactorUpper) <= 0; + + if (dataIsEvenlyDistributed) { + // the minimum dynamic chunk size is at least 1 + final int dynamicChunkSize = Math.max((int) (distributionFactor * chunkSize), 1); + return splitEvenlySizedChunks( + tableId, min, max, approximateRowCnt, chunkSize, dynamicChunkSize); + } else { + return splitUnevenlySizedChunks( + jdbc, tableId, splitColumnName, min, max, chunkSize); + } + } else { + return splitUnevenlySizedChunks(jdbc, tableId, splitColumnName, min, max, chunkSize); + } + } + + /** + * Split table into evenly sized chunks based on the numeric min and max value of split column, + * and tumble chunks in step size. + */ + private List splitEvenlySizedChunks( + TableId tableId, + Object min, + Object max, + long approximateRowCnt, + int chunkSize, + int dynamicChunkSize) { + LOG.info( + "Use evenly-sized chunk optimization for table {}, the approximate row count is {}, the chunk size is {}, the dynamic chunk size is {}", + tableId, + approximateRowCnt, + chunkSize, + dynamicChunkSize); + if (approximateRowCnt <= chunkSize) { + // there is no more than one chunk, return full table as a chunk + return Collections.singletonList(ChunkRange.all()); + } + + final List splits = new ArrayList<>(); + Object chunkStart = null; + Object chunkEnd = ObjectUtils.plus(min, dynamicChunkSize); + while (ObjectUtils.compare(chunkEnd, max) <= 0) { + splits.add(ChunkRange.of(chunkStart, chunkEnd)); + chunkStart = chunkEnd; + try { + chunkEnd = ObjectUtils.plus(chunkEnd, dynamicChunkSize); + } catch (ArithmeticException e) { + // Stop chunk split to avoid dead loop when number overflows. + break; + } + } + // add the ending split + splits.add(ChunkRange.of(chunkStart, null)); + return splits; + } + + // ------------------------------------------------------------------------------------------ + + /** Split table into unevenly sized chunks by continuously calculating next chunk max value. */ + private List splitUnevenlySizedChunks( + JdbcConnection jdbc, + TableId tableId, + String splitColumnName, + Object min, + Object max, + int chunkSize) + throws SQLException { + LOG.info( + "Use unevenly-sized chunks for table {}, the chunk size is {}", tableId, chunkSize); + final List splits = new ArrayList<>(); + Object chunkStart = null; + Object chunkEnd = nextChunkEnd(jdbc, min, tableId, splitColumnName, max, chunkSize); + int count = 0; + while (chunkEnd != null && ObjectUtils.compare(chunkEnd, max) <= 0) { + // we start from [null, min + chunk_size) and avoid [null, min) + splits.add(ChunkRange.of(chunkStart, chunkEnd)); + // may sleep a while to avoid DDOS on PostgreSQL server + maySleep(count++, tableId); + chunkStart = chunkEnd; + chunkEnd = nextChunkEnd(jdbc, chunkEnd, tableId, splitColumnName, max, chunkSize); + } + // add the ending split + splits.add(ChunkRange.of(chunkStart, null)); + return splits; + } + + private Object nextChunkEnd( + JdbcConnection jdbc, + Object previousChunkEnd, + TableId tableId, + String splitColumnName, + Object max, + int chunkSize) + throws SQLException { + // chunk end might be null when max values are removed + Object chunkEnd = + queryNextChunkMax(jdbc, tableId, splitColumnName, chunkSize, previousChunkEnd); + if (Objects.equals(previousChunkEnd, chunkEnd)) { + // we don't allow equal chunk start and end, + // should query the next one larger than chunkEnd + chunkEnd = queryMin(jdbc, tableId, splitColumnName, chunkEnd); + } + if (ObjectUtils.compare(chunkEnd, max) >= 0) { + return null; + } else { + return chunkEnd; + } + } + + private SnapshotSplit createSnapshotSplit( + JdbcConnection jdbc, + TableId tableId, + int chunkId, + RowType splitKeyType, + Object chunkStart, + Object chunkEnd) { + // currently, we only support single split column + Object[] splitStart = chunkStart == null ? null : new Object[] {chunkStart}; + Object[] splitEnd = chunkEnd == null ? null : new Object[] {chunkEnd}; + Map schema = new HashMap<>(); + schema.put(tableId, dialect.queryTableSchema(jdbc, tableId)); + return new SnapshotSplit( + tableId, + splitId(tableId, chunkId), + splitKeyType, + splitStart, + splitEnd, + null, + schema); + } + + /** Returns the distribution factor of the given table. */ + private double calculateDistributionFactor( + TableId tableId, Object min, Object max, long approximateRowCnt) { + + if (!min.getClass().equals(max.getClass())) { + throw new IllegalStateException( + String.format( + "Unsupported operation type, the MIN value type %s is different with MAX value type %s.", + min.getClass().getSimpleName(), max.getClass().getSimpleName())); + } + if (approximateRowCnt == 0) { + return Double.MAX_VALUE; + } + BigDecimal difference = ObjectUtils.minus(max, min); + // factor = (max - min + 1) / rowCount + final BigDecimal subRowCnt = difference.add(BigDecimal.valueOf(1)); + double distributionFactor = + subRowCnt.divide(new BigDecimal(approximateRowCnt), 4, ROUND_CEILING).doubleValue(); + LOG.info( + "The distribution factor of table {} is {} according to the min split key {}, max split key {} and approximate row count {}", + tableId, + distributionFactor, + min, + max, + approximateRowCnt); + return distributionFactor; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java new file mode 100644 index 0000000000..09ea548ad8 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresConnectionPoolFactory.java @@ -0,0 +1,39 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; +import com.zaxxer.hikari.HikariDataSource; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A connection pool factory to create pooled Postgres {@link HikariDataSource}. */ +public class PostgresConnectionPoolFactory extends JdbcConnectionPoolFactory { + public static final String JDBC_URL_PATTERN = "jdbc:postgresql://%s:%s/%s"; + + private static final Logger LOG = LoggerFactory.getLogger(PostgresConnectionPoolFactory.class); + + @Override + public String getJdbcUrl(JdbcSourceConfig sourceConfig) { + + String hostName = sourceConfig.getHostname(); + int port = sourceConfig.getPort(); + String database = sourceConfig.getDatabaseList().get(0); + return String.format(JDBC_URL_PATTERN, hostName, port, database); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java new file mode 100644 index 0000000000..4e018da2c0 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresDialect.java @@ -0,0 +1,183 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.dialect.JdbcDataSourceDialect; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionFactory; +import com.ververica.cdc.connectors.base.relational.connection.JdbcConnectionPoolFactory; +import com.ververica.cdc.connectors.base.source.assigner.splitter.ChunkSplitter; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; +import com.ververica.cdc.connectors.postgres.source.fetch.PostgresScanFetchTask; +import com.ververica.cdc.connectors.postgres.source.fetch.PostgresSourceFetchTaskContext; +import com.ververica.cdc.connectors.postgres.source.fetch.PostgresStreamFetchTask; +import com.ververica.cdc.connectors.postgres.source.utils.CustomPostgresSchema; +import com.ververica.cdc.connectors.postgres.source.utils.TableDiscoveryUtils; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.TableId; +import io.debezium.relational.history.TableChanges.TableChange; + +import javax.annotation.Nullable; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static io.debezium.connector.postgresql.PostgresObjectUtils.newPostgresValueConverterBuilder; +import static io.debezium.connector.postgresql.Utils.currentOffset; + +/** The dialect for Postgres. */ +public class PostgresDialect implements JdbcDataSourceDialect { + private static final long serialVersionUID = 1L; + private final PostgresSourceConfig sourceConfig; + + private transient CustomPostgresSchema schema; + + @Nullable private PostgresStreamFetchTask streamFetchTask; + + public PostgresDialect(PostgresSourceConfigFactory configFactory) { + this.sourceConfig = configFactory.create(0); + } + + @Override + public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { + PostgresSourceConfig postgresSourceConfig = (PostgresSourceConfig) sourceConfig; + PostgresConnectorConfig dbzConfig = postgresSourceConfig.getDbzConnectorConfig(); + + PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = + newPostgresValueConverterBuilder(dbzConfig); + PostgresConnection jdbc = + new PostgresConnection( + dbzConfig.getJdbcConfig(), + valueConverterBuilder, + new JdbcConnectionFactory(sourceConfig, getPooledDataSourceFactory())); + + try { + jdbc.connect(); + } catch (Exception e) { + throw new FlinkRuntimeException(e); + } + return jdbc; + } + + @Override + public String getName() { + return "PostgreSQL"; + } + + @Override + public Offset displayCurrentOffset(JdbcSourceConfig sourceConfig) { + + try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) { + return currentOffset((PostgresConnection) jdbc); + + } catch (SQLException e) { + throw new FlinkRuntimeException(e); + } + } + + @Override + public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { + // from Postgres docs: + // + // SQL is case insensitive about key words and identifiers, + // except when identifiers are double-quoted to preserve the case + return true; + } + + @Override + public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { + return new PostgresChunkSplitter(sourceConfig, this); + } + + @Override + public List discoverDataCollections(JdbcSourceConfig sourceConfig) { + try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) { + return TableDiscoveryUtils.listTables( + // there is always a single database provided + sourceConfig.getDatabaseList().get(0), + jdbc, + ((PostgresSourceConfig) sourceConfig).getTableFilters()); + } catch (SQLException e) { + throw new FlinkRuntimeException("Error to discover tables: " + e.getMessage(), e); + } + } + + @Override + public Map discoverDataCollectionSchemas(JdbcSourceConfig sourceConfig) { + final List capturedTableIds = discoverDataCollections(sourceConfig); + + try (JdbcConnection jdbc = openJdbcConnection(sourceConfig)) { + // fetch table schemas + Map tableSchemas = new HashMap<>(); + for (TableId tableId : capturedTableIds) { + TableChange tableSchema = queryTableSchema(jdbc, tableId); + tableSchemas.put(tableId, tableSchema); + } + return tableSchemas; + } catch (Exception e) { + throw new FlinkRuntimeException( + "Error to discover table schemas: " + e.getMessage(), e); + } + } + + @Override + public JdbcConnectionPoolFactory getPooledDataSourceFactory() { + return new PostgresConnectionPoolFactory(); + } + + @Override + public TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { + if (schema == null) { + schema = new CustomPostgresSchema((PostgresConnection) jdbc, sourceConfig); + } + return schema.getTableSchema(tableId); + } + + @Override + public FetchTask createFetchTask(SourceSplitBase sourceSplitBase) { + if (sourceSplitBase.isSnapshotSplit()) { + return new PostgresScanFetchTask(sourceSplitBase.asSnapshotSplit()); + } else { + this.streamFetchTask = new PostgresStreamFetchTask(sourceSplitBase.asStreamSplit()); + return this.streamFetchTask; + } + } + + @Override + public JdbcSourceFetchTaskContext createFetchTaskContext( + SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig) { + return new PostgresSourceFetchTaskContext(taskSourceConfig, this); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + if (streamFetchTask != null) { + streamFetchTask.commitCurrentOffset(); + } + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java new file mode 100644 index 0000000000..e6bb092752 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceBuilder.java @@ -0,0 +1,247 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.annotation.Experimental; + +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfigFactory; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffsetFactory; +import com.ververica.cdc.debezium.DebeziumDeserializationSchema; + +import java.time.Duration; +import java.util.Properties; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** The source builder for PostgresIncrementalSource. */ +@Experimental +public class PostgresSourceBuilder { + + private final PostgresSourceConfigFactory configFactory = new PostgresSourceConfigFactory(); + private DebeziumDeserializationSchema deserializer; + + private PostgresSourceBuilder() {} + + /** + * The name of the Postgres logical decoding plug-in installed on the server. Supported values + * are decoderbufs, wal2json, wal2json_rds, wal2json_streaming, wal2json_rds_streaming and + * pgoutput. + */ + public PostgresSourceBuilder decodingPluginName(String name) { + this.configFactory.decodingPluginName(name); + return this; + } + + /** The hostname of the database to monitor for changes. */ + public PostgresSourceBuilder hostname(String hostname) { + this.configFactory.hostname(hostname); + return this; + } + + /** Integer port number of the Postgres database server. */ + public PostgresSourceBuilder port(int port) { + this.configFactory.port(port); + return this; + } + + /** The name of the PostgreSQL database from which to stream the changes. */ + public PostgresSourceBuilder database(String database) { + this.configFactory.database(database); + return this; + } + + /** + * An required list of regular expressions that match database names to be monitored; any + * database name not included in the whitelist will be excluded from monitoring. + */ + public PostgresSourceBuilder schemaList(String... schemaList) { + this.configFactory.schemaList(schemaList); + return this; + } + + /** + * An required list of regular expressions that match fully-qualified table identifiers for + * tables to be monitored; any table not included in the list will be excluded from monitoring. + * Each identifier is of the form {@code .}. + */ + public PostgresSourceBuilder tableList(String... tableList) { + this.configFactory.tableList(tableList); + return this; + } + + /** Name of the Postgres database to use when connecting to the Postgres database server. */ + public PostgresSourceBuilder username(String username) { + this.configFactory.username(username); + return this; + } + + /** Password to use when connecting to the Postgres database server. */ + public PostgresSourceBuilder password(String password) { + this.configFactory.password(password); + return this; + } + + /** + * The name of the PostgreSQL logical decoding slot that was created for streaming changes from + * a particular plug-in for a particular database/schema. The server uses this slot to stream + * events to the connector that you are configuring. Default is "flink". + * + *

    Slot names must conform to PostgreSQL + * replication slot naming rules, which state: "Each replication slot has a name, which can + * contain lower-case letters, numbers, and the underscore character." + */ + public PostgresSourceBuilder slotName(String slotName) { + this.configFactory.slotName(slotName); + return this; + } + + /** + * The split size (number of rows) of table snapshot, captured tables are split into multiple + * splits when read the snapshot of table. + */ + public PostgresSourceBuilder splitSize(int splitSize) { + this.configFactory.splitSize(splitSize); + return this; + } + + /** + * The group size of split meta, if the meta size exceeds the group size, the meta will be + * divided into multiple groups. + */ + public PostgresSourceBuilder splitMetaGroupSize(int splitMetaGroupSize) { + this.configFactory.splitMetaGroupSize(splitMetaGroupSize); + return this; + } + + /** + * The upper bound of split key evenly distribution factor, the factor is used to determine + * whether the table is evenly distribution or not. + */ + public PostgresSourceBuilder distributionFactorUpper(double distributionFactorUpper) { + this.configFactory.distributionFactorUpper(distributionFactorUpper); + return this; + } + + /** + * The lower bound of split key evenly distribution factor, the factor is used to determine + * whether the table is evenly distribution or not. + */ + public PostgresSourceBuilder distributionFactorLower(double distributionFactorLower) { + this.configFactory.distributionFactorLower(distributionFactorLower); + return this; + } + + /** The maximum fetch size for per poll when read table snapshot. */ + public PostgresSourceBuilder fetchSize(int fetchSize) { + this.configFactory.fetchSize(fetchSize); + return this; + } + + /** + * The maximum time that the connector should wait after trying to connect to the Postgres + * database server before timing out. + */ + public PostgresSourceBuilder connectTimeout(Duration connectTimeout) { + this.configFactory.connectTimeout(connectTimeout); + return this; + } + + /** The max retry times to get connection. */ + public PostgresSourceBuilder connectMaxRetries(int connectMaxRetries) { + this.configFactory.connectMaxRetries(connectMaxRetries); + return this; + } + + /** The connection pool size. */ + public PostgresSourceBuilder connectionPoolSize(int connectionPoolSize) { + this.configFactory.connectionPoolSize(connectionPoolSize); + return this; + } + + /** Whether the {@link PostgresIncrementalSource} should output the schema changes or not. */ + public PostgresSourceBuilder includeSchemaChanges(boolean includeSchemaChanges) { + this.configFactory.includeSchemaChanges(includeSchemaChanges); + return this; + } + + /** Specifies the startup options. */ + public PostgresSourceBuilder startupOptions(StartupOptions startupOptions) { + this.configFactory.startupOptions(startupOptions); + return this; + } + + /** + * The chunk key of table snapshot, captured tables are split into multiple chunks by the chunk + * key column when read the snapshot of table. + */ + public PostgresSourceBuilder chunkKeyColumn(String chunkKeyColumn) { + this.configFactory.chunkKeyColumn(chunkKeyColumn); + return this; + } + + /** The Debezium Postgres connector properties. For example, "snapshot.mode". */ + public PostgresSourceBuilder debeziumProperties(Properties properties) { + this.configFactory.debeziumProperties(properties); + return this; + } + + /** + * The deserializer used to convert from consumed {@link + * org.apache.kafka.connect.source.SourceRecord}. + */ + public PostgresSourceBuilder deserializer(DebeziumDeserializationSchema deserializer) { + this.deserializer = deserializer; + return this; + } + + /** The heartbeat interval for the Postgres server. */ + public PostgresSourceBuilder heartbeatInterval(Duration heartbeatInterval) { + this.configFactory.heartbeatInterval(heartbeatInterval); + return this; + } + + /** + * Build the {@link PostgresIncrementalSource}. + * + * @return a PostgresParallelSource with the settings made for this builder. + */ + public PostgresIncrementalSource build() { + PostgresOffsetFactory offsetFactory = new PostgresOffsetFactory(); + PostgresDialect dialect = new PostgresDialect(configFactory); + return new PostgresIncrementalSource<>( + configFactory, checkNotNull(deserializer), offsetFactory, dialect); + } + + /** The Postgres source based on the incremental snapshot framework. */ + @Experimental + public static class PostgresIncrementalSource extends JdbcIncrementalSource { + public PostgresIncrementalSource( + PostgresSourceConfigFactory configFactory, + DebeziumDeserializationSchema deserializationSchema, + PostgresOffsetFactory offsetFactory, + PostgresDialect dataSourceDialect) { + super(configFactory, deserializationSchema, offsetFactory, dataSourceDialect); + } + + public static PostgresSourceBuilder builder() { + return new PostgresSourceBuilder<>(); + } + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfig.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfig.java new file mode 100644 index 0000000000..330c2b0143 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfig.java @@ -0,0 +1,102 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.config; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.options.StartupOptions; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.relational.RelationalTableFilters; + +import javax.annotation.Nullable; + +import java.time.Duration; +import java.util.List; +import java.util.Properties; + +/** The configuration for Postgres CDC source. */ +public class PostgresSourceConfig extends JdbcSourceConfig { + + private static final long serialVersionUID = 1L; + + private final int subtaskId; + + public PostgresSourceConfig( + int subtaskId, + StartupOptions startupOptions, + List databaseList, + List schemaList, + List tableList, + int splitSize, + int splitMetaGroupSize, + double distributionFactorUpper, + double distributionFactorLower, + boolean includeSchemaChanges, + boolean closeIdleReaders, + Properties dbzProperties, + Configuration dbzConfiguration, + String driverClassName, + String hostname, + int port, + String username, + String password, + int fetchSize, + String serverTimeZone, + Duration connectTimeout, + int connectMaxRetries, + int connectionPoolSize, + @Nullable String chunkKeyColumn) { + super( + startupOptions, + databaseList, + schemaList, + tableList, + splitSize, + splitMetaGroupSize, + distributionFactorUpper, + distributionFactorLower, + includeSchemaChanges, + closeIdleReaders, + dbzProperties, + dbzConfiguration, + driverClassName, + hostname, + port, + username, + password, + fetchSize, + serverTimeZone, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + chunkKeyColumn); + this.subtaskId = subtaskId; + } + + public int getSubtaskId() { + return subtaskId; + } + + @Override + public PostgresConnectorConfig getDbzConnectorConfig() { + return new PostgresConnectorConfig(getDbzConfiguration()); + } + + public RelationalTableFilters getTableFilters() { + return getDbzConnectorConfig().getTableFilters(); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java new file mode 100644 index 0000000000..07530d5154 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceConfigFactory.java @@ -0,0 +1,160 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.config; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfigFactory; +import com.ververica.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PostgresConnector; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.UUID; + +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** Factory to create Configuration for Postgres source. */ +public class PostgresSourceConfigFactory extends JdbcSourceConfigFactory { + + private Duration heartbeatInterval = HEARTBEAT_INTERVAL.defaultValue(); + + private static final String JDBC_DRIVER = "org.postgresql.Driver"; + + private String pluginName = "decoderbufs"; + + private String slotName = "flink"; + + private String database; + + private List schemaList; + + /** Creates a new {@link PostgresSourceConfig} for the given subtask {@code subtaskId}. */ + @Override + public PostgresSourceConfig create(int subtaskId) { + Properties props = new Properties(); + props.setProperty("connector.class", PostgresConnector.class.getCanonicalName()); + props.setProperty("plugin.name", pluginName); + // hard code server name, because we don't need to distinguish it, docs: + // Logical name that identifies and provides a namespace for the particular PostgreSQL + // database server/cluster being monitored. The logical name should be unique across + // all other connectors, since it is used as a prefix for all Kafka topic names coming + // from this connector. Only alphanumeric characters and underscores should be used. + props.setProperty("database.server.name", "postgres_cdc_source"); + props.setProperty("database.hostname", checkNotNull(hostname)); + props.setProperty("database.dbname", checkNotNull(database)); + props.setProperty("database.user", checkNotNull(username)); + props.setProperty("database.password", checkNotNull(password)); + props.setProperty("database.port", String.valueOf(port)); + props.setProperty("slot.name", checkNotNull(slotName)); + // database history + props.setProperty( + "database.history", EmbeddedFlinkDatabaseHistory.class.getCanonicalName()); + props.setProperty("database.history.instance.name", UUID.randomUUID() + "_" + subtaskId); + props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true)); + props.setProperty("database.history.refer.ddl", String.valueOf(true)); + // we have to enable heartbeat for PG to make sure DebeziumChangeConsumer#handleBatch + // is invoked after job restart + props.setProperty("heartbeat.interval.ms", String.valueOf(heartbeatInterval.toMillis())); + props.setProperty("include.schema.changes", String.valueOf(includeSchemaChanges)); + + if (schemaList != null) { + props.setProperty("schema.include.list", String.join(",", schemaList)); + } + + if (tableList != null) { + props.setProperty("table.include.list", String.join(",", tableList)); + } + + // override the user-defined debezium properties + if (dbzProperties != null) { + props.putAll(dbzProperties); + } + + Configuration dbzConfiguration = Configuration.from(props); + return new PostgresSourceConfig( + subtaskId, + startupOptions, + Collections.singletonList(database), + schemaList, + tableList, + splitSize, + splitMetaGroupSize, + distributionFactorUpper, + distributionFactorLower, + includeSchemaChanges, + closeIdleReaders, + props, + dbzConfiguration, + JDBC_DRIVER, + hostname, + port, + username, + password, + fetchSize, + serverTimeZone, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + chunkKeyColumn); + } + + /** + * An optional list of regular expressions that match schema names to be monitored; any schema + * name not included in the whitelist will be excluded from monitoring. By default all + * non-system schemas will be monitored. + */ + public void schemaList(String[] schemaList) { + this.schemaList = Arrays.asList(schemaList); + } + + /** + * The name of the Postgres logical decoding plug-in installed on the server. Supported values + * are decoderbufs, wal2json, wal2json_rds, wal2json_streaming, wal2json_rds_streaming and + * pgoutput. + */ + public void decodingPluginName(String name) { + this.pluginName = name; + } + + /** The name of the PostgreSQL database from which to stream the changes. */ + public void database(String database) { + this.database = database; + } + + /** + * The name of the PostgreSQL logical decoding slot that was created for streaming changes from + * a particular plug-in for a particular database/schema. The server uses this slot to stream + * events to the connector that you are configuring. Default is "flink". + * + *

    Slot names must conform to PostgreSQL + * replication slot naming rules, which state: "Each replication slot has a name, which can + * contain lower-case letters, numbers, and the underscore character." + */ + public void slotName(String slotName) { + this.slotName = slotName; + } + + /** The interval of heartbeat events. */ + public void heartbeatInterval(Duration heartbeatInterval) { + this.heartbeatInterval = heartbeatInterval; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceOptions.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceOptions.java new file mode 100644 index 0000000000..7a4d4ed408 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/config/PostgresSourceOptions.java @@ -0,0 +1,70 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.config; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; + +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; +import com.ververica.cdc.connectors.postgres.source.PostgresSourceBuilder; +import com.ververica.cdc.debezium.table.DebeziumChangelogMode; + +import java.time.Duration; + +/** Configurations for {@link PostgresSourceBuilder.PostgresIncrementalSource}. */ +public class PostgresSourceOptions extends JdbcSourceOptions { + + public static final ConfigOption PORT = + ConfigOptions.key("port") + .intType() + .defaultValue(5432) + .withDescription("Integer port number of the PostgreSQL database server."); + + public static final ConfigOption DECODING_PLUGIN_NAME = + ConfigOptions.key("decoding.plugin.name") + .stringType() + .defaultValue("decoderbufs") + .withDescription( + "The name of the Postgres logical decoding plug-in installed on the server.\n" + + "Supported values are decoderbufs, wal2json, wal2json_rds, wal2json_streaming,\n" + + "wal2json_rds_streaming and pgoutput."); + + public static final ConfigOption SLOT_NAME = + ConfigOptions.key("slot.name") + .stringType() + .defaultValue("flink") + .withDescription( + "The name of the PostgreSQL logical decoding slot that was created for streaming changes " + + "from a particular plug-in for a particular database/schema. The server uses this slot " + + "to stream events to the connector that you are configuring. Default is \"flink\"."); + + public static final ConfigOption CHANGELOG_MODE = + ConfigOptions.key("changelog-mode") + .enumType(DebeziumChangelogMode.class) + .defaultValue(DebeziumChangelogMode.ALL) + .withDescription( + "The changelog mode used for encoding streaming changes.\n" + + "\"all\": Encodes changes as retract stream using all RowKinds. This is the default mode.\n" + + "\"upsert\": Encodes changes as upsert stream that describes idempotent updates on a key. It can be used for tables with primary keys when replica identity FULL is not an option."); + + public static final ConfigOption HEARTBEAT_INTERVAL = + ConfigOptions.key("heartbeat.interval.ms") + .durationType() + .defaultValue(Duration.ofSeconds(30)) + .withDescription( + "Optional interval of sending heartbeat event for tracing the latest available replication slot offsets"); +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java new file mode 100644 index 0000000000..2df2e7cbe8 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresScanFetchTask.java @@ -0,0 +1,406 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.fetch; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.meta.split.SnapshotSplit; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind; +import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffsetUtils; +import com.ververica.cdc.connectors.postgres.source.utils.PostgresQueryUtils; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.heartbeat.Heartbeat; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.AbstractSnapshotChangeEventSource; +import io.debezium.pipeline.source.spi.ChangeEventSource; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.SnapshotResult; +import io.debezium.relational.RelationalSnapshotChangeEventSource; +import io.debezium.relational.SnapshotChangeRecordEmitter; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.util.Clock; +import io.debezium.util.ColumnUtils; +import io.debezium.util.Strings; +import io.debezium.util.Threads; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Objects; + +import static io.debezium.connector.postgresql.Utils.currentOffset; +import static io.debezium.connector.postgresql.Utils.refreshSchema; +import static io.debezium.relational.RelationalSnapshotChangeEventSource.LOG_INTERVAL; + +/** A {@link FetchTask} implementation for Postgres to read snapshot split. */ +public class PostgresScanFetchTask implements FetchTask { + + private static final Logger LOG = LoggerFactory.getLogger(PostgresScanFetchTask.class); + + private final SnapshotSplit split; + private volatile boolean taskRunning = false; + + public PostgresScanFetchTask(SnapshotSplit split) { + this.split = split; + } + + @Override + public SourceSplitBase getSplit() { + return split; + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void execute(Context context) throws Exception { + LOG.info("Execute ScanFetchTask for split: {}", split); + PostgresSourceFetchTaskContext ctx = (PostgresSourceFetchTaskContext) context; + taskRunning = true; + + PostgresSnapshotSplitReadTask snapshotSplitReadTask = + new PostgresSnapshotSplitReadTask( + ctx.getConnection(), + ctx.getDbzConnectorConfig(), + ctx.getDatabaseSchema(), + ctx.getOffsetContext(), + ctx.getDispatcher(), + ctx.getSnapshotChangeEventSourceMetrics(), + split); + + SnapshotSplitChangeEventSourceContext changeEventSourceContext = + new SnapshotSplitChangeEventSourceContext(); + SnapshotResult snapshotResult = + snapshotSplitReadTask.execute(changeEventSourceContext, ctx.getOffsetContext()); + + if (!snapshotResult.isCompletedOrSkipped()) { + taskRunning = false; + throw new IllegalStateException( + String.format("Read snapshot for postgres split %s fail", split)); + } + + executeBackfillTask(ctx, changeEventSourceContext); + } + + private void executeBackfillTask( + PostgresSourceFetchTaskContext ctx, + SnapshotSplitChangeEventSourceContext changeEventSourceContext) + throws InterruptedException { + final StreamSplit backfillSplit = + new StreamSplit( + split.splitId(), + changeEventSourceContext.getLowWatermark(), + changeEventSourceContext.getHighWatermark(), + new ArrayList<>(), + split.getTableSchemas(), + 0); + + // optimization that skip the WAL read when the low watermark >= high watermark + final boolean backfillRequired = + backfillSplit.getEndingOffset().isAfter(backfillSplit.getStartingOffset()); + if (!backfillRequired) { + LOG.info( + "Skip the backfill {} for split {}: low watermark >= high watermark", + backfillSplit, + split); + ctx.getDispatcher() + .dispatchWatermarkEvent( + ctx.getOffsetContext().getPartition(), + backfillSplit, + backfillSplit.getEndingOffset(), + WatermarkKind.END); + + taskRunning = false; + return; + } + + final PostgresOffsetContext.Loader loader = + new PostgresOffsetContext.Loader(ctx.getDbzConnectorConfig()); + final PostgresOffsetContext postgresOffsetContext = + PostgresOffsetUtils.getPostgresOffsetContext( + loader, backfillSplit.getStartingOffset()); + + // we should only capture events for the current table, + // otherwise, we may not find corresponding schema + PostgresSourceConfig pgSourceConfig = (PostgresSourceConfig) ctx.getSourceConfig(); + Configuration dbzConf = + ctx.getDbzConnectorConfig() + .getConfig() + .edit() + .with("table.include.list", split.getTableId().toString()) + .with( + "slot.name", + pgSourceConfig.getDbzProperties().getProperty("slot.name") + + "_" + + pgSourceConfig.getSubtaskId()) + // Disable heartbeat event in snapshot split fetcher + .with(Heartbeat.HEARTBEAT_INTERVAL, 0) + .build(); + + final PostgresStreamFetchTask.StreamSplitReadTask backfillReadTask = + new PostgresStreamFetchTask.StreamSplitReadTask( + new PostgresConnectorConfig(dbzConf), + ctx.getSnapShotter(), + ctx.getConnection(), + ctx.getDispatcher(), + ctx.getErrorHandler(), + ctx.getTaskContext().getClock(), + ctx.getDatabaseSchema(), + ctx.getTaskContext(), + ctx.getReplicationConnection(), + backfillSplit); + LOG.info("Execute backfillReadTask for split {}", split); + LOG.info("Slot name {}", dbzConf.getString("slot.name")); + backfillReadTask.execute(new PostgresChangeEventSourceContext(), postgresOffsetContext); + } + + static class SnapshotSplitChangeEventSourceContext + implements ChangeEventSource.ChangeEventSourceContext { + + private PostgresOffset lowWatermark; + private PostgresOffset highWatermark; + + public PostgresOffset getLowWatermark() { + return lowWatermark; + } + + public void setLowWatermark(PostgresOffset lowWatermark) { + this.lowWatermark = lowWatermark; + } + + public PostgresOffset getHighWatermark() { + return highWatermark; + } + + public void setHighWatermark(PostgresOffset highWatermark) { + this.highWatermark = highWatermark; + } + + @Override + public boolean isRunning() { + return lowWatermark != null && highWatermark != null; + } + } + + class PostgresChangeEventSourceContext implements ChangeEventSource.ChangeEventSourceContext { + + public void finished() { + taskRunning = false; + } + + @Override + public boolean isRunning() { + return taskRunning; + } + } + + /** A SnapshotChangeEventSource implementation for Postgres to read snapshot split. */ + public static class PostgresSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + private static final Logger LOG = + LoggerFactory.getLogger(PostgresSnapshotSplitReadTask.class); + + private final PostgresConnection jdbcConnection; + private final PostgresConnectorConfig connectorConfig; + private final JdbcSourceEventDispatcher dispatcher; + private final SnapshotSplit snapshotSplit; + private final PostgresOffsetContext offsetContext; + private final PostgresSchema databaseSchema; + private final SnapshotProgressListener snapshotProgressListener; + private final Clock clock; + + public PostgresSnapshotSplitReadTask( + PostgresConnection jdbcConnection, + PostgresConnectorConfig connectorConfig, + PostgresSchema databaseSchema, + PostgresOffsetContext previousOffset, + JdbcSourceEventDispatcher dispatcher, + SnapshotProgressListener snapshotProgressListener, + SnapshotSplit snapshotSplit) { + super(connectorConfig, snapshotProgressListener); + this.jdbcConnection = jdbcConnection; + this.connectorConfig = connectorConfig; + this.snapshotProgressListener = snapshotProgressListener; + this.databaseSchema = databaseSchema; + this.dispatcher = dispatcher; + this.snapshotSplit = snapshotSplit; + this.offsetContext = previousOffset; + this.clock = Clock.SYSTEM; + } + + @Override + protected SnapshotResult doExecute( + ChangeEventSourceContext context, + OffsetContext previousOffset, + SnapshotContext snapshotContext, + SnapshottingTask snapshottingTask) + throws Exception { + final RelationalSnapshotChangeEventSource.RelationalSnapshotContext ctx = + (RelationalSnapshotChangeEventSource.RelationalSnapshotContext) snapshotContext; + ctx.offset = offsetContext; + refreshSchema(databaseSchema, jdbcConnection, false); + + final PostgresOffset lowWatermark = currentOffset(jdbcConnection); + LOG.info( + "Snapshot step 1 - Determining low watermark {} for split {}", + lowWatermark, + snapshotSplit); + ((SnapshotSplitChangeEventSourceContext) (context)).setLowWatermark(lowWatermark); + dispatcher.dispatchWatermarkEvent( + offsetContext.getPartition(), snapshotSplit, lowWatermark, WatermarkKind.LOW); + + LOG.info("Snapshot step 2 - Snapshotting data"); + createDataEvents(ctx, snapshotSplit.getTableId()); + + final PostgresOffset highWatermark = currentOffset(jdbcConnection); + LOG.info( + "Snapshot step 3 - Determining high watermark {} for split {}", + highWatermark, + snapshotSplit); + ((SnapshotSplitChangeEventSourceContext) (context)).setHighWatermark(highWatermark); + dispatcher.dispatchWatermarkEvent( + offsetContext.getPartition(), snapshotSplit, highWatermark, WatermarkKind.HIGH); + return SnapshotResult.completed(ctx.offset); + } + + private void createDataEvents( + RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, + TableId tableId) + throws InterruptedException { + EventDispatcher.SnapshotReceiver snapshotReceiver = + dispatcher.getSnapshotChangeEventReceiver(); + LOG.info("Snapshotting table {}", tableId); + createDataEventsForTable( + snapshotContext, + snapshotReceiver, + Objects.requireNonNull(databaseSchema.tableFor(tableId))); + snapshotReceiver.completeSnapshot(); + } + + /** Dispatches the data change events for the records of a single table. */ + private void createDataEventsForTable( + RelationalSnapshotChangeEventSource.RelationalSnapshotContext snapshotContext, + EventDispatcher.SnapshotReceiver snapshotReceiver, + Table table) + throws InterruptedException { + + long exportStart = clock.currentTimeInMillis(); + LOG.info( + "Exporting data from split '{}' of table {}", + snapshotSplit.splitId(), + table.id()); + + final String selectSql = + PostgresQueryUtils.buildSplitScanQuery( + snapshotSplit.getTableId(), + snapshotSplit.getSplitKeyType(), + snapshotSplit.getSplitStart() == null, + snapshotSplit.getSplitEnd() == null); + LOG.debug( + "For split '{}' of table {} using select statement: '{}'", + snapshotSplit.splitId(), + table.id(), + selectSql); + + try (PreparedStatement selectStatement = + PostgresQueryUtils.readTableSplitDataStatement( + jdbcConnection, + selectSql, + snapshotSplit.getSplitStart() == null, + snapshotSplit.getSplitEnd() == null, + snapshotSplit.getSplitStart(), + snapshotSplit.getSplitEnd(), + snapshotSplit.getSplitKeyType().getFieldCount(), + connectorConfig.getQueryFetchSize()); + ResultSet rs = selectStatement.executeQuery()) { + + ColumnUtils.ColumnArray columnArray = ColumnUtils.toArray(rs, table); + long rows = 0; + Threads.Timer logTimer = getTableScanLogTimer(); + + while (rs.next()) { + rows++; + final Object[] row = new Object[columnArray.getGreatestColumnPosition()]; + for (int i = 0; i < columnArray.getColumns().length; i++) { + row[columnArray.getColumns()[i].position() - 1] = rs.getObject(i + 1); + } + if (logTimer.expired()) { + long stop = clock.currentTimeInMillis(); + LOG.info( + "Exported {} records for split '{}' after {}", + rows, + snapshotSplit.splitId(), + Strings.duration(stop - exportStart)); + snapshotProgressListener.rowsScanned(table.id(), rows); + logTimer = getTableScanLogTimer(); + } + snapshotContext.offset.event(table.id(), clock.currentTime()); + SnapshotChangeRecordEmitter emitter = + new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + dispatcher.dispatchSnapshotEvent(table.id(), emitter, snapshotReceiver); + } + LOG.info( + "Finished exporting {} records for split '{}', total duration '{}'", + rows, + snapshotSplit.splitId(), + Strings.duration(clock.currentTimeInMillis() - exportStart)); + } catch (SQLException e) { + throw new FlinkRuntimeException( + "Snapshotting of table " + table.id() + " failed", e); + } + } + + private Threads.Timer getTableScanLogTimer() { + return Threads.timer(clock, LOG_INTERVAL); + } + + @Override + protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + return new SnapshottingTask(false, true); + } + + @Override + protected SnapshotContext prepare(ChangeEventSourceContext changeEventSourceContext) + throws Exception { + return new PostgresSnapshotContext(); + } + + private static class PostgresSnapshotContext + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + + public PostgresSnapshotContext() throws SQLException { + super(""); + } + } + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java new file mode 100644 index 0000000000..e9fd5cd7b5 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContext.java @@ -0,0 +1,263 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.fetch; + +import org.apache.flink.table.types.logical.RowType; + +import com.ververica.cdc.connectors.base.config.JdbcSourceConfig; +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.EmbeddedFlinkDatabaseHistory; +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.reader.external.JdbcSourceFetchTaskContext; +import com.ververica.cdc.connectors.postgres.source.PostgresDialect; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffsetFactory; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffsetUtils; +import com.ververica.cdc.connectors.postgres.source.utils.ChunkUtils; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresErrorHandler; +import io.debezium.connector.postgresql.PostgresObjectUtils; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.PostgresTaskContext; +import io.debezium.connector.postgresql.PostgresTopicSelector; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.connector.postgresql.spi.Snapshotter; +import io.debezium.data.Envelope; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.metrics.DefaultChangeEventSourceMetricsFactory; +import io.debezium.pipeline.metrics.SnapshotChangeEventSourceMetrics; +import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetrics; +import io.debezium.pipeline.metrics.spi.ChangeEventSourceMetricsFactory; +import io.debezium.pipeline.source.spi.EventMetadataProvider; +import io.debezium.relational.Column; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.schema.TopicSelector; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.util.concurrent.atomic.AtomicReference; + +import static io.debezium.connector.AbstractSourceInfo.SCHEMA_NAME_KEY; +import static io.debezium.connector.AbstractSourceInfo.TABLE_NAME_KEY; +import static io.debezium.connector.postgresql.PostgresConnectorConfig.SNAPSHOT_MODE; +import static io.debezium.connector.postgresql.PostgresObjectUtils.createReplicationConnection; +import static io.debezium.connector.postgresql.PostgresObjectUtils.newPostgresValueConverterBuilder; + +/** The context of {@link PostgresScanFetchTask} and {@link PostgresStreamFetchTask}. */ +public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { + private static final Logger LOG = LoggerFactory.getLogger(PostgresSourceFetchTaskContext.class); + + private PostgresTaskContext taskContext; + private ChangeEventQueue queue; + private PostgresConnection jdbcConnection; + private final AtomicReference replicationConnection = + new AtomicReference<>(); + private PostgresOffsetContext offsetContext; + private PostgresSchema schema; + private ErrorHandler errorHandler; + private JdbcSourceEventDispatcher dispatcher; + private EventMetadataProvider metadataProvider; + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + private StreamingChangeEventSourceMetrics streamingChangeEventSourceMetrics; + private Snapshotter snapShotter; + + public PostgresSourceFetchTaskContext( + JdbcSourceConfig sourceConfig, PostgresDialect dataSourceDialect) { + super(sourceConfig, dataSourceDialect); + } + + @Override + public PostgresConnectorConfig getDbzConnectorConfig() { + return (PostgresConnectorConfig) super.getDbzConnectorConfig(); + } + + private PostgresOffsetContext loadStartingOffsetState( + PostgresOffsetContext.Loader loader, SourceSplitBase sourceSplitBase) { + Offset offset = + sourceSplitBase.isSnapshotSplit() + ? new PostgresOffsetFactory() + .createInitialOffset() // get an offset for starting snapshot + : sourceSplitBase.asStreamSplit().getStartingOffset(); + + return PostgresOffsetUtils.getPostgresOffsetContext(loader, offset); + } + + @Override + public void configure(SourceSplitBase sourceSplitBase) { + LOG.debug("Configuring PostgresSourceFetchTaskContext for split: {}", sourceSplitBase); + PostgresConnectorConfig dbzConfig = getDbzConnectorConfig(); + + PostgresConnectorConfig.SnapshotMode snapshotMode = + PostgresConnectorConfig.SnapshotMode.parse( + dbzConfig.getConfig().getString(SNAPSHOT_MODE)); + this.snapShotter = snapshotMode.getSnapshotter(dbzConfig.getConfig()); + + PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = + newPostgresValueConverterBuilder(dbzConfig); + this.jdbcConnection = + new PostgresConnection(dbzConfig.getJdbcConfig(), valueConverterBuilder); + + TopicSelector topicSelector = PostgresTopicSelector.create(dbzConfig); + EmbeddedFlinkDatabaseHistory.registerHistory( + sourceConfig + .getDbzConfiguration() + .getString(EmbeddedFlinkDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), + sourceSplitBase.getTableSchemas().values()); + + try { + this.schema = + PostgresObjectUtils.newSchema( + jdbcConnection, + dbzConfig, + jdbcConnection.getTypeRegistry(), + topicSelector, + valueConverterBuilder.build(jdbcConnection.getTypeRegistry())); + } catch (SQLException e) { + throw new RuntimeException("Failed to initialize PostgresSchema", e); + } + + this.offsetContext = + loadStartingOffsetState( + new PostgresOffsetContext.Loader(dbzConfig), sourceSplitBase); + this.taskContext = PostgresObjectUtils.newTaskContext(dbzConfig, schema, topicSelector); + + this.replicationConnection.compareAndSet( + null, + createReplicationConnection( + this.taskContext, this.snapShotter.shouldSnapshot(), dbzConfig)); + + final int queueSize = + sourceSplitBase.isSnapshotSplit() ? Integer.MAX_VALUE : dbzConfig.getMaxQueueSize(); + this.queue = + new ChangeEventQueue.Builder() + .pollInterval(dbzConfig.getPollInterval()) + .maxBatchSize(dbzConfig.getMaxBatchSize()) + .maxQueueSize(queueSize) + .maxQueueSizeInBytes(dbzConfig.getMaxQueueSizeInBytes()) + .loggingContextSupplier( + () -> + taskContext.configureLoggingContext( + "postgres-cdc-connector-task")) + // do not buffer any element, we use signal event + // .buffering() + .build(); + + this.errorHandler = new PostgresErrorHandler(dbzConnectorConfig.getLogicalName(), queue); + this.metadataProvider = PostgresObjectUtils.newEventMetadataProvider(); + this.dispatcher = + new JdbcSourceEventDispatcher( + dbzConfig, + topicSelector, + schema, + queue, + dbzConfig.getTableFilters().dataCollectionFilter(), + DataChangeEvent::new, + metadataProvider, + schemaNameAdjuster); + + ChangeEventSourceMetricsFactory metricsFactory = + new DefaultChangeEventSourceMetricsFactory(); + this.snapshotChangeEventSourceMetrics = + metricsFactory.getSnapshotMetrics(taskContext, queue, metadataProvider); + this.streamingChangeEventSourceMetrics = + metricsFactory.getStreamingMetrics(taskContext, queue, metadataProvider); + } + + @Override + public PostgresSchema getDatabaseSchema() { + return schema; + } + + @Override + public RowType getSplitType(Table table) { + Column splitColumn = ChunkUtils.getSplitColumn(table, sourceConfig.getChunkKeyColumn()); + return ChunkUtils.getSplitType(splitColumn); + } + + @Override + public ErrorHandler getErrorHandler() { + return errorHandler; + } + + @Override + public JdbcSourceEventDispatcher getDispatcher() { + return dispatcher; + } + + @Override + public PostgresOffsetContext getOffsetContext() { + return offsetContext; + } + + @Override + public ChangeEventQueue getQueue() { + return queue; + } + + @Override + public Tables.TableFilter getTableFilter() { + return getDbzConnectorConfig().getTableFilters().dataCollectionFilter(); + } + + @Override + public TableId getTableId(SourceRecord record) { + Struct value = (Struct) record.value(); + Struct source = value.getStruct(Envelope.FieldName.SOURCE); + String schemaName = source.getString(SCHEMA_NAME_KEY); + String tableName = source.getString(TABLE_NAME_KEY); + return new TableId(null, schemaName, tableName); + } + + @Override + public Offset getStreamOffset(SourceRecord sourceRecord) { + return PostgresOffset.of(sourceRecord); + } + + public PostgresConnection getConnection() { + return jdbcConnection; + } + + public PostgresTaskContext getTaskContext() { + return taskContext; + } + + public ReplicationConnection getReplicationConnection() { + return replicationConnection.get(); + } + + public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + return snapshotChangeEventSourceMetrics; + } + + public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics() { + return streamingChangeEventSourceMetrics; + } + + public Snapshotter getSnapShotter() { + return snapShotter; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java new file mode 100644 index 0000000000..1c832e6274 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresStreamFetchTask.java @@ -0,0 +1,232 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.fetch; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.relational.JdbcSourceEventDispatcher; +import com.ververica.cdc.connectors.base.source.meta.split.SourceSplitBase; +import com.ververica.cdc.connectors.base.source.meta.split.StreamSplit; +import com.ververica.cdc.connectors.base.source.meta.wartermark.WatermarkKind; +import com.ververica.cdc.connectors.base.source.reader.external.FetchTask; +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource; +import io.debezium.connector.postgresql.PostgresTaskContext; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.connector.postgresql.spi.Snapshotter; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.source.spi.ChangeEventSource; +import io.debezium.util.Clock; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +import static com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset.NO_STOPPING_OFFSET; + +/** A {@link FetchTask} implementation for Postgres to read streaming changes. */ +public class PostgresStreamFetchTask implements FetchTask { + private static final Logger LOG = LoggerFactory.getLogger(PostgresStreamFetchTask.class); + + private final StreamSplit split; + private volatile boolean taskRunning = false; + private volatile boolean stopped = false; + + private StreamSplitReadTask streamSplitReadTask; + + private Long lastCommitLsn; + + public PostgresStreamFetchTask(StreamSplit streamSplit) { + this.split = streamSplit; + } + + @Override + public void execute(Context context) throws Exception { + if (stopped) { + LOG.debug( + "StreamFetchTask for split: {} is already stopped and can not be executed", + split); + return; + } else { + LOG.debug("execute StreamFetchTask for split: {}", split); + } + + PostgresSourceFetchTaskContext sourceFetchContext = + (PostgresSourceFetchTaskContext) context; + taskRunning = true; + streamSplitReadTask = + new StreamSplitReadTask( + sourceFetchContext.getDbzConnectorConfig(), + sourceFetchContext.getSnapShotter(), + sourceFetchContext.getConnection(), + sourceFetchContext.getDispatcher(), + sourceFetchContext.getErrorHandler(), + sourceFetchContext.getTaskContext().getClock(), + sourceFetchContext.getDatabaseSchema(), + sourceFetchContext.getTaskContext(), + sourceFetchContext.getReplicationConnection(), + split); + StreamSplitChangeEventSourceContext changeEventSourceContext = + new StreamSplitChangeEventSourceContext(); + streamSplitReadTask.execute( + changeEventSourceContext, sourceFetchContext.getOffsetContext()); + } + + @Override + public void stop() { + LOG.debug("stopping StreamFetchTask for split: {}", split); + if (streamSplitReadTask != null) { + ((StreamSplitChangeEventSourceContext) streamSplitReadTask.context).finished(); + } + stopped = true; + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public SourceSplitBase getSplit() { + return split; + } + + public void commitCurrentOffset() { + if (streamSplitReadTask != null && streamSplitReadTask.offsetContext != null) { + PostgresOffsetContext postgresOffsetContext = streamSplitReadTask.offsetContext; + + // only extracting and storing the lsn of the last commit + Long commitLsn = + (Long) + postgresOffsetContext + .getOffset() + .get(PostgresOffsetContext.LAST_COMMIT_LSN_KEY); + if (commitLsn != null + && (lastCommitLsn == null + || Lsn.valueOf(commitLsn).compareTo(Lsn.valueOf(lastCommitLsn)) > 0)) { + lastCommitLsn = commitLsn; + + Map offsets = new HashMap<>(); + offsets.put(PostgresOffsetContext.LAST_COMMIT_LSN_KEY, lastCommitLsn); + LOG.debug( + "Committing offset {} for {}", + Lsn.valueOf(lastCommitLsn), + streamSplitReadTask.streamSplit); + streamSplitReadTask.commitOffset(offsets); + } + } + } + + private class StreamSplitChangeEventSourceContext + implements ChangeEventSource.ChangeEventSourceContext { + + public void finished() { + taskRunning = false; + } + + @Override + public boolean isRunning() { + return taskRunning; + } + } + + /** A {@link ChangeEventSource} implementation for Postgres to read streaming changes. */ + public static class StreamSplitReadTask extends PostgresStreamingChangeEventSource { + private static final Logger LOG = LoggerFactory.getLogger(StreamSplitReadTask.class); + private final StreamSplit streamSplit; + private final JdbcSourceEventDispatcher dispatcher; + private final ErrorHandler errorHandler; + + public ChangeEventSourceContext context; + public PostgresOffsetContext offsetContext; + + public StreamSplitReadTask( + PostgresConnectorConfig connectorConfig, + Snapshotter snapshotter, + PostgresConnection connection, + JdbcSourceEventDispatcher dispatcher, + ErrorHandler errorHandler, + Clock clock, + PostgresSchema schema, + PostgresTaskContext taskContext, + ReplicationConnection replicationConnection, + StreamSplit streamSplit) { + + super( + connectorConfig, + snapshotter, + connection, + dispatcher, + errorHandler, + clock, + schema, + taskContext, + replicationConnection); + this.streamSplit = streamSplit; + this.dispatcher = dispatcher; + this.errorHandler = errorHandler; + } + + @Override + public void execute(ChangeEventSourceContext context, PostgresOffsetContext offsetContext) + throws InterruptedException { + this.context = context; + this.offsetContext = offsetContext; + + LOG.info("Execute StreamSplitReadTask for split: {}", streamSplit); + + offsetContext.setStreamingStoppingLsn( + ((PostgresOffset) streamSplit.getEndingOffset()).getLsn()); + super.execute(context, offsetContext); + if (isBoundedRead()) { + + LOG.debug("StreamSplit is bounded read: {}", streamSplit); + final PostgresOffset currentOffset = PostgresOffset.of(offsetContext.getOffset()); + + if (currentOffset.isAtOrAfter(streamSplit.getEndingOffset())) { + LOG.info("StreamSplitReadTask finished for {}", streamSplit); + + try { + dispatcher.dispatchWatermarkEvent( + offsetContext.getPartition(), + streamSplit, + currentOffset, + WatermarkKind.END); + } catch (InterruptedException e) { + LOG.error("Send signal event error.", e); + errorHandler.setProducerThrowable( + new FlinkRuntimeException("Error processing WAL signal event", e)); + } + + ((PostgresScanFetchTask.PostgresChangeEventSourceContext) context).finished(); + } + } + } + + private boolean isBoundedRead() { + return !NO_STOPPING_OFFSET + .getLsn() + .equals(((PostgresOffset) streamSplit.getEndingOffset()).getLsn()); + } + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffset.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffset.java new file mode 100644 index 0000000000..ccacaaef65 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffset.java @@ -0,0 +1,126 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.offset; + +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import io.debezium.connector.postgresql.SourceInfo; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.time.Conversions; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; + +import javax.annotation.Nullable; + +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; + +/** The offset for Postgres. */ +public class PostgresOffset extends Offset { + + private static final Logger LOG = org.slf4j.LoggerFactory.getLogger(PostgresOffset.class); + + public static final PostgresOffset INITIAL_OFFSET = + new PostgresOffset(Lsn.INVALID_LSN.asLong(), null, Instant.MIN); + public static final PostgresOffset NO_STOPPING_OFFSET = + new PostgresOffset(Lsn.valueOf("FFFFFFFF/FFFFFFFF").asLong(), null, Instant.MAX); + + // used by PostgresOffsetFactory + PostgresOffset(Map offset) { + this.offset = offset; + } + + PostgresOffset(Long lsn, Long txId, Instant lastCommitTs) { + Map offsetMap = new HashMap<>(); + // keys are from io.debezium.connector.postgresql.PostgresOffsetContext.Loader.load + offsetMap.put(SourceInfo.LSN_KEY, lsn.toString()); + if (txId != null) { + offsetMap.put(SourceInfo.TXID_KEY, txId.toString()); + } + if (lastCommitTs != null) { + offsetMap.put( + SourceInfo.TIMESTAMP_USEC_KEY, + String.valueOf(Conversions.toEpochMicros(lastCommitTs))); + } + this.offset = offsetMap; + } + + public static PostgresOffset of(SourceRecord dataRecord) { + return of(dataRecord.sourceOffset()); + } + + public static PostgresOffset of(Map offsetMap) { + Map offsetStrMap = new HashMap<>(); + for (Map.Entry entry : offsetMap.entrySet()) { + offsetStrMap.put( + entry.getKey(), entry.getValue() == null ? null : entry.getValue().toString()); + } + + return new PostgresOffset(offsetStrMap); + } + + public Lsn getLsn() { + return Lsn.valueOf(Long.valueOf(this.offset.get(SourceInfo.LSN_KEY))); + } + + @Nullable + public Long getTxid() { + String txid = this.offset.get(SourceInfo.TXID_KEY); + return txid == null ? null : Long.valueOf(txid); + } + + @Nullable + public Long getLastCommitTs() { + String lastCommitTs = this.offset.get(SourceInfo.TIMESTAMP_USEC_KEY); + return lastCommitTs == null ? null : Long.valueOf(lastCommitTs); + } + + @Override + public int compareTo(Offset o) { + PostgresOffset rhs = (PostgresOffset) o; + LOG.debug("comparing {} and {}", this, rhs); + return this.getLsn().compareTo(rhs.getLsn()); + } + + @Override + public String toString() { + return "Offset{lsn=" + + getLsn() + + ", txId=" + + (getTxid() == null ? "null" : getTxid()) + + ", lastCommitTs=" + + (getLastCommitTs() == null ? "null" : getLastCommitTs()) + + "]"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof PostgresOffset)) { + return false; + } + PostgresOffset that = (PostgresOffset) o; + return offset.equals(that.offset); + } + + @Override + public Map getOffset() { + return offset; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetFactory.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetFactory.java new file mode 100644 index 0000000000..54dba34802 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetFactory.java @@ -0,0 +1,58 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.offset; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import com.ververica.cdc.connectors.base.source.meta.offset.OffsetFactory; + +import java.util.Map; + +/** The factory to create {@link PostgresOffset}. */ +public class PostgresOffsetFactory extends OffsetFactory { + @Override + public Offset newOffset(Map offset) { + return new PostgresOffset(offset); + } + + @Override + public Offset newOffset(String filename, Long position) { + throw new FlinkRuntimeException( + "not supported create new Offset by Filename and Long position."); + } + + @Override + public Offset newOffset(Long position) { + throw new FlinkRuntimeException("not supported create new Offset by Long position."); + } + + @Override + public Offset createTimestampOffset(long timestampMillis) { + throw new FlinkRuntimeException("not supported create new Offset from timestamp."); + } + + @Override + public Offset createInitialOffset() { + return PostgresOffset.INITIAL_OFFSET; + } + + @Override + public Offset createNoStoppingOffset() { + return PostgresOffset.NO_STOPPING_OFFSET; + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetUtils.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetUtils.java new file mode 100644 index 0000000000..5d8a933dbe --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/offset/PostgresOffsetUtils.java @@ -0,0 +1,44 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.offset; + +import com.ververica.cdc.connectors.base.source.meta.offset.Offset; +import io.debezium.connector.postgresql.PostgresOffsetContext; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** Utils for handling {@link PostgresOffset}. */ +public class PostgresOffsetUtils { + public static PostgresOffsetContext getPostgresOffsetContext( + PostgresOffsetContext.Loader loader, Offset offset) { + + Map offsetStrMap = + Objects.requireNonNull(offset, "offset is null for the sourceSplitBase") + .getOffset(); + // all the keys happen to be long type for PostgresOffsetContext.Loader.load + Map offsetMap = new HashMap<>(); + for (String key : offsetStrMap.keySet()) { + String value = offsetStrMap.get(key); + if (value != null) { + offsetMap.put(key, Long.parseLong(value)); + } + } + return loader.load(offsetMap); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/ChunkUtils.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/ChunkUtils.java new file mode 100644 index 0000000000..00889feed4 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/ChunkUtils.java @@ -0,0 +1,72 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.utils; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.RowType; + +import io.debezium.relational.Column; +import io.debezium.relational.Table; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; + +/** Utilities to split chunks of table. */ +public class ChunkUtils { + public static RowType getSplitType(Column splitColumn) { + return (RowType) + DataTypes.ROW( + DataTypes.FIELD( + splitColumn.name(), + PostgresTypeUtils.fromDbzColumn(splitColumn))) + .getLogicalType(); + } + + public static Column getSplitColumn(Table table, @Nullable String chunkKeyColumn) { + List primaryKeys = table.primaryKeyColumns(); + if (primaryKeys.isEmpty()) { + throw new ValidationException( + String.format( + "Incremental snapshot for tables requires primary key," + + " but table %s doesn't have primary key.", + table.id())); + } + + if (chunkKeyColumn != null) { + Optional targetPkColumn = + primaryKeys.stream() + .filter(col -> chunkKeyColumn.equals(col.name())) + .findFirst(); + if (targetPkColumn.isPresent()) { + return targetPkColumn.get(); + } + throw new ValidationException( + String.format( + "Chunk key column '%s' doesn't exist in the primary key [%s] of the table %s.", + chunkKeyColumn, + primaryKeys.stream().map(Column::name).collect(Collectors.joining(",")), + table.id())); + } + + // use first field in primary key as the split key + return primaryKeys.get(0); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java new file mode 100644 index 0000000000..6fa1579796 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/CustomPostgresSchema.java @@ -0,0 +1,106 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.utils; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.postgres.source.config.PostgresSourceConfig; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.relational.history.TableChanges; +import io.debezium.relational.history.TableChanges.TableChange; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.util.Clock; + +import java.sql.SQLException; +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** A CustomPostgresSchema similar to PostgresSchema with customization. */ +public class CustomPostgresSchema { + + // cache the schema for each table + private final Map schemasByTableId = new HashMap<>(); + private final PostgresConnection jdbcConnection; + private final PostgresConnectorConfig dbzConfig; + + public CustomPostgresSchema( + PostgresConnection jdbcConnection, PostgresSourceConfig sourceConfig) { + this.jdbcConnection = jdbcConnection; + this.dbzConfig = sourceConfig.getDbzConnectorConfig(); + } + + public TableChange getTableSchema(TableId tableId) { + // read schema from cache first + if (!schemasByTableId.containsKey(tableId)) { + try { + readTableSchema(tableId); + } catch (SQLException e) { + throw new FlinkRuntimeException("Failed to read table schema", e); + } + } + return schemasByTableId.get(tableId); + } + + private TableChange readTableSchema(TableId tableId) throws SQLException { + + final PostgresOffsetContext offsetContext = + PostgresOffsetContext.initialContext(dbzConfig, jdbcConnection, Clock.SYSTEM); + + // set the events to populate proper sourceInfo into offsetContext + offsetContext.event(tableId, Instant.now()); + + Tables tables = new Tables(); + try { + jdbcConnection.readSchema( + tables, + dbzConfig.databaseName(), + tableId.schema(), + dbzConfig.getTableFilters().dataCollectionFilter(), + null, + false); + } catch (SQLException e) { + throw new FlinkRuntimeException("Failed to read schema", e); + } + + Table table = Objects.requireNonNull(tables.forTable(tableId)); + + // TODO: check whether we always set isFromSnapshot = true + SchemaChangeEvent schemaChangeEvent = + new SchemaChangeEvent( + offsetContext.getPartition(), + offsetContext.getOffset(), + offsetContext.getSourceInfo(), + dbzConfig.databaseName(), + tableId.schema(), + null, + table, + SchemaChangeEvent.SchemaChangeEventType.CREATE, + true); + + for (TableChanges.TableChange tableChange : schemaChangeEvent.getTableChanges()) { + this.schemasByTableId.put(tableId, tableChange); + } + return this.schemasByTableId.get(tableId); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresQueryUtils.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresQueryUtils.java new file mode 100644 index 0000000000..8d091cee42 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresQueryUtils.java @@ -0,0 +1,329 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.utils; + +import org.apache.flink.table.types.logical.RowType; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.TableId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.util.Iterator; +import java.util.Optional; +import java.util.stream.Collectors; + +import static com.ververica.cdc.connectors.base.utils.SourceRecordUtils.rowToArray; + +/** Query-related Utilities for Postgres CDC source. */ +public class PostgresQueryUtils { + + private static final Logger LOG = LoggerFactory.getLogger(PostgresQueryUtils.class); + + private PostgresQueryUtils() {} + + public static Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) + throws SQLException { + final String minMaxQuery = + String.format( + "SELECT MIN(%s), MAX(%s) FROM %s", + quote(columnName), quote(columnName), quote(tableId)); + return jdbc.queryAndMap( + minMaxQuery, + rs -> { + if (!rs.next()) { + // this should never happen + throw new SQLException( + String.format( + "No result returned after running query [%s]", + minMaxQuery)); + } + return rowToArray(rs, 2); + }); + } + + public static long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) + throws SQLException { + // The statement used to get approximate row count which is less + // accurate than COUNT(*), but is more efficient for large table. + // https://stackoverflow.com/questions/7943233/fast-way-to-discover-the-row-count-of-a-table-in-postgresql + // NOTE: it requires ANALYZE or VACUUM to be run first in PostgreSQL. + final String query = + String.format( + "SELECT reltuples::bigint FROM pg_class WHERE oid = to_regclass('%s')", + tableId.toString()); + + return jdbc.queryAndMap( + query, + rs -> { + if (!rs.next()) { + throw new SQLException( + String.format( + "No result returned after running query [%s]", query)); + } + LOG.info("queryApproximateRowCnt: {} => {}", query, rs.getLong(1)); + return rs.getLong(1); + }); + } + + public static Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException { + final String query = + String.format( + "SELECT MIN(%s) FROM %s WHERE %s > ?", + quote(columnName), quote(tableId), quote(columnName)); + return jdbc.prepareQueryAndMap( + query, + ps -> ps.setObject(1, excludedLowerBound), + rs -> { + if (!rs.next()) { + // this should never happen + throw new SQLException( + String.format( + "No result returned after running query [%s]", query)); + } + LOG.info("{} => {}", query, rs.getObject(1)); + return rs.getObject(1); + }); + } + + public static Object queryNextChunkMax( + JdbcConnection jdbc, + TableId tableId, + String splitColumnName, + int chunkSize, + Object includedLowerBound) + throws SQLException { + String quotedColumn = quote(splitColumnName); + String query = + String.format( + "SELECT MAX(%s) FROM (" + + "SELECT %s FROM %s WHERE %s >= ? ORDER BY %s ASC LIMIT %s" + + ") AS T", + quotedColumn, + quotedColumn, + quote(tableId), + quotedColumn, + quotedColumn, + chunkSize); + return jdbc.prepareQueryAndMap( + query, + ps -> ps.setObject(1, includedLowerBound), + rs -> { + if (!rs.next()) { + // this should never happen + throw new SQLException( + String.format( + "No result returned after running query [%s]", query)); + } + return rs.getObject(1); + }); + } + + public static String buildSplitScanQuery( + TableId tableId, RowType pkRowType, boolean isFirstSplit, boolean isLastSplit) { + return buildSplitQuery(tableId, pkRowType, isFirstSplit, isLastSplit, -1, true); + } + + private static String buildSplitQuery( + TableId tableId, + RowType pkRowType, + boolean isFirstSplit, + boolean isLastSplit, + int limitSize, + boolean isScanningData) { + final String condition; + + if (isFirstSplit && isLastSplit) { + condition = null; + } else if (isFirstSplit) { + final StringBuilder sql = new StringBuilder(); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?"); + if (isScanningData) { + sql.append(" AND NOT ("); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?"); + sql.append(")"); + } + condition = sql.toString(); + } else if (isLastSplit) { + final StringBuilder sql = new StringBuilder(); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?"); + condition = sql.toString(); + } else { + final StringBuilder sql = new StringBuilder(); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " >= ?"); + if (isScanningData) { + sql.append(" AND NOT ("); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " = ?"); + sql.append(")"); + } + sql.append(" AND "); + addPrimaryKeyColumnsToCondition(pkRowType, sql, " <= ?"); + condition = sql.toString(); + } + + if (isScanningData) { + return buildSelectWithRowLimits( + tableId, limitSize, "*", Optional.ofNullable(condition), Optional.empty()); + } else { + final String orderBy = + pkRowType.getFieldNames().stream().collect(Collectors.joining(", ")); + return buildSelectWithBoundaryRowLimits( + tableId, + limitSize, + getPrimaryKeyColumnsProjection(pkRowType), + getMaxPrimaryKeyColumnsProjection(pkRowType), + Optional.ofNullable(condition), + orderBy); + } + } + + public static PreparedStatement readTableSplitDataStatement( + JdbcConnection jdbc, + String sql, + boolean isFirstSplit, + boolean isLastSplit, + Object[] splitStart, + Object[] splitEnd, + int primaryKeyNum, + int fetchSize) { + try { + final PreparedStatement statement = initStatement(jdbc, sql, fetchSize); + if (isFirstSplit && isLastSplit) { + return statement; + } + if (isFirstSplit) { + for (int i = 0; i < primaryKeyNum; i++) { + statement.setObject(i + 1, splitEnd[i]); + statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]); + } + } else if (isLastSplit) { + for (int i = 0; i < primaryKeyNum; i++) { + statement.setObject(i + 1, splitStart[i]); + } + } else { + for (int i = 0; i < primaryKeyNum; i++) { + statement.setObject(i + 1, splitStart[i]); + statement.setObject(i + 1 + primaryKeyNum, splitEnd[i]); + statement.setObject(i + 1 + 2 * primaryKeyNum, splitEnd[i]); + } + } + return statement; + } catch (Exception e) { + throw new RuntimeException("Failed to build the split data read statement.", e); + } + } + + public static String quote(String dbOrTableName) { + return "\"" + dbOrTableName + "\""; + } + + public static String quote(TableId tableId) { + return tableId.toQuotedString('"'); + } + + private static PreparedStatement initStatement(JdbcConnection jdbc, String sql, int fetchSize) + throws SQLException { + final Connection connection = jdbc.connection(); + connection.setAutoCommit(false); + final PreparedStatement statement = connection.prepareStatement(sql); + statement.setFetchSize(fetchSize); + return statement; + } + + private static void addPrimaryKeyColumnsToCondition( + RowType pkRowType, StringBuilder sql, String predicate) { + for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); + fieldNamesIt.hasNext(); ) { + sql.append(fieldNamesIt.next()).append(predicate); + if (fieldNamesIt.hasNext()) { + sql.append(" AND "); + } + } + } + + private static String getPrimaryKeyColumnsProjection(RowType pkRowType) { + StringBuilder sql = new StringBuilder(); + for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); + fieldNamesIt.hasNext(); ) { + sql.append(fieldNamesIt.next()); + if (fieldNamesIt.hasNext()) { + sql.append(" , "); + } + } + return sql.toString(); + } + + private static String getMaxPrimaryKeyColumnsProjection(RowType pkRowType) { + StringBuilder sql = new StringBuilder(); + for (Iterator fieldNamesIt = pkRowType.getFieldNames().iterator(); + fieldNamesIt.hasNext(); ) { + sql.append("MAX(" + fieldNamesIt.next() + ")"); + if (fieldNamesIt.hasNext()) { + sql.append(" , "); + } + } + return sql.toString(); + } + + private static String buildSelectWithRowLimits( + TableId tableId, + int limit, + String projection, + Optional condition, + Optional orderBy) { + final StringBuilder sql = new StringBuilder("SELECT "); + sql.append(projection).append(" FROM "); + sql.append(quote(tableId)); + if (condition.isPresent()) { + sql.append(" WHERE ").append(condition.get()); + } + if (orderBy.isPresent()) { + sql.append(" ORDER BY ").append(orderBy.get()); + } + if (limit > 0) { + sql.append(" LIMIT ").append(limit); + } + return sql.toString(); + } + + private static String buildSelectWithBoundaryRowLimits( + TableId tableId, + int limit, + String projection, + String maxColumnProjection, + Optional condition, + String orderBy) { + final StringBuilder sql = new StringBuilder("SELECT "); + sql.append(maxColumnProjection); + sql.append(" FROM ("); + sql.append("SELECT "); + sql.append(projection); + sql.append(" FROM "); + sql.append(quote(tableId)); + if (condition.isPresent()) { + sql.append(" WHERE ").append(condition.get()); + } + sql.append(" ORDER BY ").append(orderBy).append(" LIMIT ").append(limit); + sql.append(") T"); + return sql.toString(); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresTypeUtils.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresTypeUtils.java new file mode 100644 index 0000000000..0bea61717a --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/PostgresTypeUtils.java @@ -0,0 +1,162 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.utils; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.DecimalType; + +import io.debezium.relational.Column; + +/** A utility class for converting Postgres types to Flink types. */ +public class PostgresTypeUtils { + private static final String PG_SMALLSERIAL = "smallserial"; + private static final String PG_SERIAL = "serial"; + private static final String PG_BIGSERIAL = "bigserial"; + private static final String PG_BYTEA = "bytea"; + private static final String PG_BYTEA_ARRAY = "_bytea"; + private static final String PG_SMALLINT = "int2"; + private static final String PG_SMALLINT_ARRAY = "_int2"; + private static final String PG_INTEGER = "int4"; + private static final String PG_INTEGER_ARRAY = "_int4"; + private static final String PG_BIGINT = "int8"; + private static final String PG_BIGINT_ARRAY = "_int8"; + private static final String PG_REAL = "float4"; + private static final String PG_REAL_ARRAY = "_float4"; + private static final String PG_DOUBLE_PRECISION = "float8"; + private static final String PG_DOUBLE_PRECISION_ARRAY = "_float8"; + private static final String PG_NUMERIC = "numeric"; + private static final String PG_NUMERIC_ARRAY = "_numeric"; + private static final String PG_BOOLEAN = "bool"; + private static final String PG_BOOLEAN_ARRAY = "_bool"; + private static final String PG_TIMESTAMP = "timestamp"; + private static final String PG_TIMESTAMP_ARRAY = "_timestamp"; + private static final String PG_TIMESTAMPTZ = "timestamptz"; + private static final String PG_TIMESTAMPTZ_ARRAY = "_timestamptz"; + private static final String PG_DATE = "date"; + private static final String PG_DATE_ARRAY = "_date"; + private static final String PG_TIME = "time"; + private static final String PG_TIME_ARRAY = "_time"; + private static final String PG_TEXT = "text"; + private static final String PG_TEXT_ARRAY = "_text"; + private static final String PG_CHAR = "bpchar"; + private static final String PG_CHAR_ARRAY = "_bpchar"; + private static final String PG_CHARACTER = "character"; + private static final String PG_CHARACTER_ARRAY = "_character"; + private static final String PG_CHARACTER_VARYING = "varchar"; + private static final String PG_CHARACTER_VARYING_ARRAY = "_varchar"; + + /** Returns a corresponding Flink data type from a debezium {@link Column}. */ + public static DataType fromDbzColumn(Column column) { + DataType dataType = convertFromColumn(column); + if (column.isOptional()) { + return dataType; + } else { + return dataType.notNull(); + } + } + + /** + * Returns a corresponding Flink data type from a debezium {@link Column} with nullable always + * be true. + */ + private static DataType convertFromColumn(Column column) { + String typeName = column.typeName(); + + int precision = column.length(); + int scale = column.scale().orElse(0); + + switch (typeName) { + case PG_BOOLEAN: + return DataTypes.BOOLEAN(); + case PG_BOOLEAN_ARRAY: + return DataTypes.ARRAY(DataTypes.BOOLEAN()); + case PG_BYTEA: + return DataTypes.BYTES(); + case PG_BYTEA_ARRAY: + return DataTypes.ARRAY(DataTypes.BYTES()); + case PG_SMALLINT: + case PG_SMALLSERIAL: + return DataTypes.SMALLINT(); + case PG_SMALLINT_ARRAY: + return DataTypes.ARRAY(DataTypes.SMALLINT()); + case PG_INTEGER: + case PG_SERIAL: + return DataTypes.INT(); + case PG_INTEGER_ARRAY: + return DataTypes.ARRAY(DataTypes.INT()); + case PG_BIGINT: + case PG_BIGSERIAL: + return DataTypes.BIGINT(); + case PG_BIGINT_ARRAY: + return DataTypes.ARRAY(DataTypes.BIGINT()); + case PG_REAL: + return DataTypes.FLOAT(); + case PG_REAL_ARRAY: + return DataTypes.ARRAY(DataTypes.FLOAT()); + case PG_DOUBLE_PRECISION: + return DataTypes.DOUBLE(); + case PG_DOUBLE_PRECISION_ARRAY: + return DataTypes.ARRAY(DataTypes.DOUBLE()); + case PG_NUMERIC: + // see SPARK-26538: handle numeric without explicit precision and scale. + if (precision > 0) { + return DataTypes.DECIMAL(precision, scale); + } + return DataTypes.DECIMAL(DecimalType.MAX_PRECISION, 18); + case PG_NUMERIC_ARRAY: + // see SPARK-26538: handle numeric without explicit precision and scale. + if (precision > 0) { + return DataTypes.ARRAY(DataTypes.DECIMAL(precision, scale)); + } + return DataTypes.ARRAY(DataTypes.DECIMAL(DecimalType.MAX_PRECISION, 18)); + case PG_CHAR: + case PG_CHARACTER: + return DataTypes.CHAR(precision); + case PG_CHAR_ARRAY: + case PG_CHARACTER_ARRAY: + return DataTypes.ARRAY(DataTypes.CHAR(precision)); + case PG_CHARACTER_VARYING: + return DataTypes.VARCHAR(precision); + case PG_CHARACTER_VARYING_ARRAY: + return DataTypes.ARRAY(DataTypes.VARCHAR(precision)); + case PG_TEXT: + return DataTypes.STRING(); + case PG_TEXT_ARRAY: + return DataTypes.ARRAY(DataTypes.STRING()); + case PG_TIMESTAMP: + return DataTypes.TIMESTAMP(scale); + case PG_TIMESTAMP_ARRAY: + return DataTypes.ARRAY(DataTypes.TIMESTAMP(scale)); + case PG_TIMESTAMPTZ: + return DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(scale); + case PG_TIMESTAMPTZ_ARRAY: + return DataTypes.ARRAY(DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE(scale)); + case PG_TIME: + return DataTypes.TIME(scale); + case PG_TIME_ARRAY: + return DataTypes.ARRAY(DataTypes.TIME(scale)); + case PG_DATE: + return DataTypes.DATE(); + case PG_DATE_ARRAY: + return DataTypes.ARRAY(DataTypes.DATE()); + default: + throw new UnsupportedOperationException( + String.format("Doesn't support Postgres type '%s' yet", typeName)); + } + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/TableDiscoveryUtils.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/TableDiscoveryUtils.java new file mode 100644 index 0000000000..2a1738eb24 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/source/utils/TableDiscoveryUtils.java @@ -0,0 +1,52 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.utils; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.RelationalTableFilters; +import io.debezium.relational.TableId; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** A utility class for table discovery. */ +public class TableDiscoveryUtils { + private static final Logger LOG = LoggerFactory.getLogger(TableDiscoveryUtils.class); + + public static List listTables( + String database, JdbcConnection jdbc, RelationalTableFilters tableFilters) + throws SQLException { + + Set allTableIds = + jdbc.readTableNames(database, null, null, new String[] {"TABLE"}); + + Set capturedTables = + allTableIds.stream() + .filter(t -> tableFilters.dataCollectionFilter().isIncluded(t)) + .collect(Collectors.toSet()); + LOG.info( + "Postgres captured tables : {} .", + capturedTables.stream().map(TableId::toString).collect(Collectors.joining(","))); + + return new ArrayList<>(capturedTables); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java index 50c40d48fe..cc88669213 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactory.java @@ -20,22 +20,50 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.ValidationException; import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.factories.DynamicTableFactory; import org.apache.flink.table.factories.DynamicTableSourceFactory; import org.apache.flink.table.factories.FactoryUtil; +import com.ververica.cdc.connectors.base.options.JdbcSourceOptions; +import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.connectors.postgres.utils.OptionUtils; import com.ververica.cdc.debezium.table.DebeziumChangelogMode; +import java.time.Duration; import java.util.HashSet; import java.util.Set; +import static com.ververica.cdc.connectors.base.utils.ObjectUtils.doubleCompare; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CHANGELOG_MODE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CHUNK_META_GROUP_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECTION_POOL_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECT_MAX_RETRIES; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECT_TIMEOUT; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.DATABASE_NAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.DECODING_PLUGIN_NAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.HOSTNAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.PASSWORD; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.PORT; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_ENABLED; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_STARTUP_MODE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCHEMA_NAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SLOT_NAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.TABLE_NAME; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.USERNAME; import static com.ververica.cdc.debezium.table.DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX; import static com.ververica.cdc.debezium.table.DebeziumOptions.getDebeziumProperties; import static com.ververica.cdc.debezium.utils.ResolvedSchemaUtils.getPhysicalSchema; import static org.apache.flink.util.Preconditions.checkArgument; +import static org.apache.flink.util.Preconditions.checkState; /** Factory for creating configured instance of {@link PostgreSQLTableSource}. */ public class PostgreSQLTableFactory implements DynamicTableSourceFactory { @@ -137,6 +165,29 @@ public DynamicTableSource createDynamicTableSource(DynamicTableFactory.Context c physicalSchema.getPrimaryKey().isPresent(), "Primary key must be present when upsert mode is selected."); } + boolean enableParallelRead = config.get(SCAN_INCREMENTAL_SNAPSHOT_ENABLED); + StartupOptions startupOptions = getStartupOptions(config); + int splitSize = config.get(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE); + int splitMetaGroupSize = config.get(CHUNK_META_GROUP_SIZE); + int fetchSize = config.get(SCAN_SNAPSHOT_FETCH_SIZE); + Duration connectTimeout = config.get(CONNECT_TIMEOUT); + int connectMaxRetries = config.get(CONNECT_MAX_RETRIES); + int connectionPoolSize = config.get(CONNECTION_POOL_SIZE); + double distributionFactorUpper = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + double distributionFactorLower = config.get(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + Duration heartbeatInterval = config.get(HEARTBEAT_INTERVAL); + String chunkKeyColumn = + config.getOptional(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN).orElse(null); + + if (enableParallelRead) { + validateIntegerOption(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE, splitSize, 1); + validateIntegerOption(SCAN_SNAPSHOT_FETCH_SIZE, fetchSize, 1); + validateIntegerOption(CHUNK_META_GROUP_SIZE, splitMetaGroupSize, 1); + validateIntegerOption(JdbcSourceOptions.CONNECTION_POOL_SIZE, connectionPoolSize, 1); + validateIntegerOption(JdbcSourceOptions.CONNECT_MAX_RETRIES, connectMaxRetries, 0); + validateDistributionFactorUpper(distributionFactorUpper); + validateDistributionFactorLower(distributionFactorLower); + } OptionUtils.printOptions(IDENTIFIER, ((Configuration) config).toMap()); @@ -152,7 +203,19 @@ public DynamicTableSource createDynamicTableSource(DynamicTableFactory.Context c pluginName, slotName, changelogMode, - getDebeziumProperties(context.getCatalogTable().getOptions())); + getDebeziumProperties(context.getCatalogTable().getOptions()), + enableParallelRead, + splitSize, + splitMetaGroupSize, + fetchSize, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + distributionFactorUpper, + distributionFactorLower, + heartbeatInterval, + startupOptions, + chunkKeyColumn); } @Override @@ -179,6 +242,76 @@ public Set> optionalOptions() { options.add(PORT); options.add(DECODING_PLUGIN_NAME); options.add(CHANGELOG_MODE); + options.add(SCAN_STARTUP_MODE); + options.add(SCAN_INCREMENTAL_SNAPSHOT_ENABLED); + options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE); + options.add(SCAN_INCREMENTAL_SNAPSHOT_CHUNK_KEY_COLUMN); + options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND); + options.add(SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND); + options.add(CHUNK_META_GROUP_SIZE); + options.add(SCAN_SNAPSHOT_FETCH_SIZE); + options.add(CONNECT_TIMEOUT); + options.add(CONNECT_MAX_RETRIES); + options.add(CONNECTION_POOL_SIZE); + options.add(HEARTBEAT_INTERVAL); return options; } + + private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial"; + private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset"; + + private static StartupOptions getStartupOptions(ReadableConfig config) { + String modeString = config.get(SCAN_STARTUP_MODE); + + switch (modeString.toLowerCase()) { + case SCAN_STARTUP_MODE_VALUE_INITIAL: + return StartupOptions.initial(); + + case SCAN_STARTUP_MODE_VALUE_LATEST: + return StartupOptions.latest(); + + default: + throw new ValidationException( + String.format( + "Invalid value for option '%s'. Supported values are [%s, %s], but was: %s", + SCAN_STARTUP_MODE.key(), + SCAN_STARTUP_MODE_VALUE_INITIAL, + SCAN_STARTUP_MODE_VALUE_LATEST, + modeString)); + } + } + + /** Checks the value of given integer option is valid. */ + private void validateIntegerOption( + ConfigOption option, int optionValue, int exclusiveMin) { + checkState( + optionValue > exclusiveMin, + String.format( + "The value of option '%s' must larger than %d, but is %d", + option.key(), exclusiveMin, optionValue)); + } + + /** Checks the value of given evenly distribution factor upper bound is valid. */ + private void validateDistributionFactorUpper(double distributionFactorUpper) { + checkState( + doubleCompare(distributionFactorUpper, 1.0d) >= 0, + String.format( + "The value of option '%s' must larger than or equals %s, but is %s", + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.key(), + 1.0d, + distributionFactorUpper)); + } + + /** Checks the value of given evenly distribution factor lower bound is valid. */ + private void validateDistributionFactorLower(double distributionFactorLower) { + checkState( + doubleCompare(distributionFactorLower, 0.0d) >= 0 + && doubleCompare(distributionFactorLower, 1.0d) <= 0, + String.format( + "The value of option '%s' must between %s and %s inclusively, but is %s", + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.key(), + 0.0d, + 1.0d, + distributionFactorLower)); + } } diff --git a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableSource.java b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableSource.java index 50fb40a04c..412e4359fc 100644 --- a/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableSource.java +++ b/flink-connector-postgres-cdc/src/main/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableSource.java @@ -22,18 +22,25 @@ import org.apache.flink.table.connector.source.DynamicTableSource; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.connector.source.SourceFunctionProvider; +import org.apache.flink.table.connector.source.SourceProvider; import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.DataType; import org.apache.flink.table.types.logical.RowType; +import com.ververica.cdc.connectors.base.options.StartupOptions; +import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; import com.ververica.cdc.connectors.postgres.PostgreSQLSource; +import com.ververica.cdc.connectors.postgres.source.PostgresSourceBuilder; import com.ververica.cdc.debezium.DebeziumDeserializationSchema; import com.ververica.cdc.debezium.DebeziumSourceFunction; import com.ververica.cdc.debezium.table.DebeziumChangelogMode; import com.ververica.cdc.debezium.table.MetadataConverter; import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema; +import javax.annotation.Nullable; + +import java.time.Duration; import java.util.Collections; import java.util.List; import java.util.Map; @@ -62,6 +69,18 @@ public class PostgreSQLTableSource implements ScanTableSource, SupportsReadingMe private final String slotName; private final DebeziumChangelogMode changelogMode; private final Properties dbzProperties; + private final boolean enableParallelRead; + private final int splitSize; + private final int splitMetaGroupSize; + private final int fetchSize; + private final Duration connectTimeout; + private final int connectionPoolSize; + private final int connectMaxRetries; + private final double distributionFactorUpper; + private final double distributionFactorLower; + private final Duration heartbeatInterval; + private final StartupOptions startupOptions; + private final String chunkKeyColumn; // -------------------------------------------------------------------------------------------- // Mutable attributes @@ -85,7 +104,19 @@ public PostgreSQLTableSource( String pluginName, String slotName, DebeziumChangelogMode changelogMode, - Properties dbzProperties) { + Properties dbzProperties, + boolean enableParallelRead, + int splitSize, + int splitMetaGroupSize, + int fetchSize, + Duration connectTimeout, + int connectMaxRetries, + int connectionPoolSize, + double distributionFactorUpper, + double distributionFactorLower, + Duration heartbeatInterval, + StartupOptions startupOptions, + @Nullable String chunkKeyColumn) { this.physicalSchema = physicalSchema; this.port = port; this.hostname = checkNotNull(hostname); @@ -98,6 +129,19 @@ public PostgreSQLTableSource( this.slotName = slotName; this.changelogMode = changelogMode; this.dbzProperties = dbzProperties; + this.enableParallelRead = enableParallelRead; + this.splitSize = splitSize; + this.splitMetaGroupSize = splitMetaGroupSize; + this.fetchSize = fetchSize; + this.connectTimeout = connectTimeout; + this.connectMaxRetries = connectMaxRetries; + this.connectionPoolSize = connectionPoolSize; + this.distributionFactorUpper = distributionFactorUpper; + this.distributionFactorLower = distributionFactorLower; + this.heartbeatInterval = heartbeatInterval; + this.startupOptions = startupOptions; + this.chunkKeyColumn = chunkKeyColumn; + // Mutable attributes this.producedDataType = physicalSchema.toPhysicalRowDataType(); this.metadataKeys = Collections.emptyList(); } @@ -132,21 +176,51 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext scanContext) { .setValueValidator(new PostgresValueValidator(schemaName, tableName)) .setChangelogMode(changelogMode) .build(); - DebeziumSourceFunction sourceFunction = - PostgreSQLSource.builder() - .hostname(hostname) - .port(port) - .database(database) - .schemaList(schemaName) - .tableList(schemaName + "." + tableName) - .username(username) - .password(password) - .decodingPluginName(pluginName) - .slotName(slotName) - .debeziumProperties(dbzProperties) - .deserializer(deserializer) - .build(); - return SourceFunctionProvider.of(sourceFunction, false); + + if (enableParallelRead) { + JdbcIncrementalSource parallelSource = + PostgresSourceBuilder.PostgresIncrementalSource.builder() + .hostname(hostname) + .port(port) + .database(database) + .schemaList(schemaName) + .tableList(schemaName + "." + tableName) + .username(username) + .password(password) + .decodingPluginName(pluginName) + .slotName(slotName) + .debeziumProperties(dbzProperties) + .deserializer(deserializer) + .splitSize(splitSize) + .splitMetaGroupSize(splitMetaGroupSize) + .distributionFactorUpper(distributionFactorUpper) + .distributionFactorLower(distributionFactorLower) + .fetchSize(fetchSize) + .connectTimeout(connectTimeout) + .connectMaxRetries(connectMaxRetries) + .connectionPoolSize(connectionPoolSize) + .startupOptions(startupOptions) + .chunkKeyColumn(chunkKeyColumn) + .heartbeatInterval(heartbeatInterval) + .build(); + return SourceProvider.of(parallelSource); + } else { + DebeziumSourceFunction sourceFunction = + PostgreSQLSource.builder() + .hostname(hostname) + .port(port) + .database(database) + .schemaList(schemaName) + .tableList(schemaName + "." + tableName) + .username(username) + .password(password) + .decodingPluginName(pluginName) + .slotName(slotName) + .debeziumProperties(dbzProperties) + .deserializer(deserializer) + .build(); + return SourceFunctionProvider.of(sourceFunction, false); + } } private MetadataConverter[] getMetadataConverters() { @@ -180,7 +254,19 @@ public DynamicTableSource copy() { pluginName, slotName, changelogMode, - dbzProperties); + dbzProperties, + enableParallelRead, + splitSize, + splitMetaGroupSize, + fetchSize, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + distributionFactorUpper, + distributionFactorLower, + heartbeatInterval, + startupOptions, + chunkKeyColumn); source.metadataKeys = metadataKeys; source.producedDataType = producedDataType; return source; @@ -208,7 +294,19 @@ public boolean equals(Object o) { && Objects.equals(dbzProperties, that.dbzProperties) && Objects.equals(producedDataType, that.producedDataType) && Objects.equals(metadataKeys, that.metadataKeys) - && Objects.equals(changelogMode, that.changelogMode); + && Objects.equals(changelogMode, that.changelogMode) + && Objects.equals(enableParallelRead, that.enableParallelRead) + && Objects.equals(splitSize, that.splitSize) + && Objects.equals(splitMetaGroupSize, that.splitMetaGroupSize) + && Objects.equals(fetchSize, that.fetchSize) + && Objects.equals(connectTimeout, that.connectTimeout) + && Objects.equals(connectMaxRetries, that.connectMaxRetries) + && Objects.equals(connectionPoolSize, that.connectionPoolSize) + && Objects.equals(distributionFactorUpper, that.distributionFactorUpper) + && Objects.equals(distributionFactorLower, that.distributionFactorLower) + && Objects.equals(heartbeatInterval, that.heartbeatInterval) + && Objects.equals(startupOptions, that.startupOptions) + && Objects.equals(chunkKeyColumn, that.chunkKeyColumn); } @Override @@ -227,7 +325,19 @@ public int hashCode() { dbzProperties, producedDataType, metadataKeys, - changelogMode); + changelogMode, + enableParallelRead, + splitSize, + splitMetaGroupSize, + fetchSize, + connectTimeout, + connectMaxRetries, + connectionPoolSize, + distributionFactorUpper, + distributionFactorLower, + heartbeatInterval, + startupOptions, + chunkKeyColumn); } @Override diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java new file mode 100644 index 0000000000..b6e064961b --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/PostgresObjectUtils.java @@ -0,0 +1,128 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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 io.debezium.connector.postgresql; + +import org.apache.flink.util.FlinkRuntimeException; + +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.relational.TableId; +import io.debezium.schema.TopicSelector; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.sql.SQLException; +import java.time.Duration; + +/** + * A factory for creating various Debezium objects + * + *

    It is a hack to access package-private constructor in debezium. + */ +public class PostgresObjectUtils { + private static final Logger LOGGER = LoggerFactory.getLogger(PostgresObjectUtils.class); + + /** Create a new PostgresSchema and initialize the content of the schema. */ + public static PostgresSchema newSchema( + PostgresConnection connection, + PostgresConnectorConfig config, + TypeRegistry typeRegistry, + TopicSelector topicSelector, + PostgresValueConverter valueConverter) + throws SQLException { + PostgresSchema schema = + new PostgresSchema(config, typeRegistry, topicSelector, valueConverter); + schema.refresh(connection, false); + return schema; + } + + public static PostgresTaskContext newTaskContext( + PostgresConnectorConfig connectorConfig, + PostgresSchema schema, + TopicSelector topicSelector) { + return new PostgresTaskContext(connectorConfig, schema, topicSelector); + } + + public static PostgresEventMetadataProvider newEventMetadataProvider() { + return new PostgresEventMetadataProvider(); + } + + /** + * Create a new PostgresVauleConverterBuilder instance and offer type registry for JDBC + * connection. + * + *

    It is created in this package because some methods (e.g., includeUnknownDatatypes) of + * PostgresConnectorConfig is protected. + */ + public static PostgresConnection.PostgresValueConverterBuilder newPostgresValueConverterBuilder( + PostgresConnectorConfig config) { + return typeRegistry -> + PostgresValueConverter.of(config, StandardCharsets.UTF_8, typeRegistry); + } + + // modified from + // io.debezium.connector.postgresql.PostgresConnectorTask.createReplicationConnection. + // pass connectorConfig instead of maxRetries and retryDelay as parameters. + // - old: ReplicationConnection createReplicationConnection(PostgresTaskContext taskContext, + // boolean doSnapshot, int maxRetries, Duration retryDelay) + // - new: ReplicationConnection createReplicationConnection(PostgresTaskContext taskContext, + // boolean doSnapshot, PostgresConnectorConfig connectorConfig) + public static ReplicationConnection createReplicationConnection( + PostgresTaskContext taskContext, + boolean doSnapshot, + PostgresConnectorConfig connectorConfig) { + int maxRetries = connectorConfig.maxRetries(); + Duration retryDelay = connectorConfig.retryDelay(); + + final Metronome metronome = Metronome.parker(retryDelay, Clock.SYSTEM); + short retryCount = 0; + while (retryCount <= maxRetries) { + try { + LOGGER.info("Creating a new replication connection for {}", taskContext); + return taskContext.createReplicationConnection(doSnapshot); + } catch (SQLException ex) { + retryCount++; + if (retryCount > maxRetries) { + LOGGER.error( + "Too many errors connecting to server. All {} retries failed.", + maxRetries); + throw new FlinkRuntimeException(ex); + } + + LOGGER.warn( + "Error connecting to server; will attempt retry {} of {} after {} " + + "seconds. Exception message: {}", + retryCount, + maxRetries, + retryDelay.getSeconds(), + ex.getMessage()); + try { + metronome.pause(); + } catch (InterruptedException e) { + LOGGER.warn("Connection retry sleep interrupted by exception: " + e); + Thread.currentThread().interrupt(); + } + } + } + LOGGER.error("Failed to create replication connection after {} retries", maxRetries); + throw new FlinkRuntimeException( + "Failed to create replication connection for " + taskContext); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/Utils.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/Utils.java new file mode 100644 index 0000000000..992f1abcf3 --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/Utils.java @@ -0,0 +1,78 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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 io.debezium.connector.postgresql; + +import org.apache.flink.util.FlinkRuntimeException; + +import com.ververica.cdc.connectors.postgres.source.offset.PostgresOffset; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.time.Conversions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.SQLException; +import java.time.Instant; +import java.util.HashMap; +import java.util.Map; + +/** A utility class for accessing various Debezium package-private methods. */ +public final class Utils { + + private static final Logger LOGGER = LoggerFactory.getLogger(Utils.class); + + public static Lsn lastKnownLsn(PostgresOffsetContext ctx) { + return ctx.lsn(); + } + + public static PostgresOffset currentOffset(PostgresConnection jdbcConnection) { + Long lsn; + Long txId; + try { + lsn = jdbcConnection.currentXLogLocation(); + txId = jdbcConnection.currentTransactionId(); + LOGGER.trace("Read xlogStart at '{}' from transaction '{}'", Lsn.valueOf(lsn), txId); + } catch (SQLException e) { + throw new FlinkRuntimeException("Error getting current Lsn/txId " + e.getMessage(), e); + } + + try { + jdbcConnection.commit(); + } catch (SQLException e) { + throw new FlinkRuntimeException( + "JDBC connection fails to commit: " + e.getMessage(), e); + } + + Map offsetMap = new HashMap<>(); + offsetMap.put(SourceInfo.LSN_KEY, lsn.toString()); + if (txId != null) { + offsetMap.put(SourceInfo.TXID_KEY, txId.toString()); + } + offsetMap.put( + SourceInfo.TIMESTAMP_USEC_KEY, + String.valueOf(Conversions.toEpochMicros(Instant.MIN))); + return PostgresOffset.of(offsetMap); + } + + public static PostgresSchema refreshSchema( + PostgresSchema schema, + PostgresConnection pgConnection, + boolean printReplicaIdentityInfo) + throws SQLException { + return schema.refresh(pgConnection, printReplicaIdentityInfo); + } +} diff --git a/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java new file mode 100644 index 0000000000..a64d9a117b --- /dev/null +++ b/flink-connector-postgres-cdc/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java @@ -0,0 +1,781 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.postgresql.connection; + +import com.zaxxer.hikari.pool.HikariProxyConnection; +import io.debezium.DebeziumException; +import io.debezium.annotation.VisibleForTesting; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PgOid; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresSchema; +import io.debezium.connector.postgresql.PostgresType; +import io.debezium.connector.postgresql.PostgresValueConverter; +import io.debezium.connector.postgresql.TypeRegistry; +import io.debezium.connector.postgresql.spi.SlotState; +import io.debezium.data.SpecialValueDecimal; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.schema.DatabaseSchema; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import org.apache.kafka.connect.errors.ConnectException; +import org.postgresql.core.BaseConnection; +import org.postgresql.core.ConnectionFactory; +import org.postgresql.jdbc.PgConnection; +import org.postgresql.jdbc.TimestampUtils; +import org.postgresql.replication.LogSequenceNumber; +import org.postgresql.util.PGmoney; +import org.postgresql.util.PSQLState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.Charset; +import java.sql.*; +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; + +/** + * {@link JdbcConnection} connection extension used for connecting to Postgres instances. + * + * @author Horia Chiorean + *

    Copied from Debezium 1.6.4-Final with two additional methods: + *

      + *
    • Constructor PostgresConnection( Configuration config, PostgresValueConverterBuilder + * valueConverterBuilder, ConnectionFactory factory) to allow passing a custom + * ConnectionFactory + *
    • override connection() to return a unwrapped PgConnection (otherwise, it will complain + * about HikariProxyConnection cannot be cast to class org.postgresql.core.BaseConnection) + *
    + */ +public class PostgresConnection extends JdbcConnection { + + private static Logger LOGGER = LoggerFactory.getLogger(PostgresConnection.class); + + private static final String URL_PATTERN = + "jdbc:postgresql://${" + + JdbcConfiguration.HOSTNAME + + "}:${" + + JdbcConfiguration.PORT + + "}/${" + + JdbcConfiguration.DATABASE + + "}"; + protected static final ConnectionFactory FACTORY = + JdbcConnection.patternBasedFactory( + URL_PATTERN, + org.postgresql.Driver.class.getName(), + PostgresConnection.class.getClassLoader(), + JdbcConfiguration.PORT.withDefault( + PostgresConnectorConfig.PORT.defaultValueAsString())); + + /** + * Obtaining a replication slot may fail if there's a pending transaction. We're retrying to get + * a slot for 30 min. + */ + private static final int MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT = 900; + + private static final Duration PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS = + Duration.ofSeconds(2); + + private final TypeRegistry typeRegistry; + private final PostgresDefaultValueConverter defaultValueConverter; + + /** + * Creates a Postgres connection using the supplied configuration. If necessary this connection + * is able to resolve data type mappings. Such a connection requires a {@link + * PostgresValueConverter}, and will provide its own {@link TypeRegistry}. Usually only one such + * connection per connector is needed. + * + * @param config {@link Configuration} instance, may not be null. + * @param valueConverterBuilder supplies a configured {@link PostgresValueConverter} for a given + * {@link TypeRegistry} + */ + public PostgresConnection( + Configuration config, PostgresValueConverterBuilder valueConverterBuilder) { + this(config, valueConverterBuilder, FACTORY); + } + + /** Creates a Postgres connection using the supplied configuration with customized factory */ + public PostgresConnection( + Configuration config, + PostgresValueConverterBuilder valueConverterBuilder, + ConnectionFactory factory) { + super( + config, + factory, + PostgresConnection::validateServerVersion, + PostgresConnection::defaultSettings); + + if (Objects.isNull(valueConverterBuilder)) { + this.typeRegistry = null; + this.defaultValueConverter = null; + } else { + this.typeRegistry = new TypeRegistry(this); + + final PostgresValueConverter valueConverter = + valueConverterBuilder.build(this.typeRegistry); + this.defaultValueConverter = + new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); + } + } + + /** Return an unwrapped PgConnection instead of HikariProxyConnection */ + @Override + public synchronized Connection connection() throws SQLException { + Connection conn = connection(true); + if (conn instanceof HikariProxyConnection) { + // assuming HikariCP use org.postgresql.jdbc.PgConnection + return conn.unwrap(PgConnection.class); + } + return conn; + } + + /** + * Create a Postgres connection using the supplied configuration and {@link TypeRegistry} + * + * @param config {@link Configuration} instance, may not be null. + * @param typeRegistry an existing/already-primed {@link TypeRegistry} instance + */ + public PostgresConnection(Configuration config, TypeRegistry typeRegistry) { + super( + config, + FACTORY, + PostgresConnection::validateServerVersion, + PostgresConnection::defaultSettings); + if (Objects.isNull(typeRegistry)) { + this.typeRegistry = null; + this.defaultValueConverter = null; + } else { + this.typeRegistry = typeRegistry; + final PostgresValueConverter valueConverter = + PostgresValueConverter.of( + new PostgresConnectorConfig(config), + this.getDatabaseCharset(), + typeRegistry); + this.defaultValueConverter = + new PostgresDefaultValueConverter(valueConverter, this.getTimestampUtils()); + } + } + + /** + * Creates a Postgres connection using the supplied configuration. The connector is the regular + * one without datatype resolution capabilities. + * + * @param config {@link Configuration} instance, may not be null. + */ + public PostgresConnection(Configuration config) { + this(config, (TypeRegistry) null); + } + + /** + * Returns a JDBC connection string for the current configuration. + * + * @return a {@code String} where the variables in {@code urlPattern} are replaced with values + * from the configuration + */ + public String connectionString() { + return connectionString(URL_PATTERN); + } + + /** + * Prints out information about the REPLICA IDENTITY status of a table. This in turn determines + * how much information is available for UPDATE and DELETE operations for logical replication. + * + * @param tableId the identifier of the table + * @return the replica identity information; never null + * @throws SQLException if there is a problem obtaining the replica identity information for the + * given table + */ + public ServerInfo.ReplicaIdentity readReplicaIdentityInfo(TableId tableId) throws SQLException { + String statement = + "SELECT relreplident FROM pg_catalog.pg_class c " + + "LEFT JOIN pg_catalog.pg_namespace n ON c.relnamespace=n.oid " + + "WHERE n.nspname=? and c.relname=?"; + String schema = + tableId.schema() != null && tableId.schema().length() > 0 + ? tableId.schema() + : "public"; + StringBuilder replIdentity = new StringBuilder(); + prepareQuery( + statement, + stmt -> { + stmt.setString(1, schema); + stmt.setString(2, tableId.table()); + }, + rs -> { + if (rs.next()) { + replIdentity.append(rs.getString(1)); + } else { + LOGGER.warn( + "Cannot determine REPLICA IDENTITY information for table '{}'", + tableId); + } + }); + return ServerInfo.ReplicaIdentity.parseFromDB(replIdentity.toString()); + } + + /** + * Returns the current state of the replication slot + * + * @param slotName the name of the slot + * @param pluginName the name of the plugin used for the desired slot + * @return the {@link SlotState} or null, if no slot state is found + * @throws SQLException + */ + public SlotState getReplicationSlotState(String slotName, String pluginName) + throws SQLException { + ServerInfo.ReplicationSlot slot; + try { + slot = readReplicationSlotInfo(slotName, pluginName); + if (slot.equals(ServerInfo.ReplicationSlot.INVALID)) { + return null; + } else { + return slot.asSlotState(); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new ConnectException( + "Interrupted while waiting for valid replication slot info", e); + } + } + + /** + * Fetches the state of a replication stage given a slot name and plugin name + * + * @param slotName the name of the slot + * @param pluginName the name of the plugin used for the desired slot + * @return the {@link ServerInfo.ReplicationSlot} object or a {@link + * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid + * @throws SQLException is thrown by the underlying JDBC + */ + private ServerInfo.ReplicationSlot fetchReplicationSlotInfo(String slotName, String pluginName) + throws SQLException { + final String database = database(); + final ServerInfo.ReplicationSlot slot = + queryForSlot( + slotName, + database, + pluginName, + rs -> { + if (rs.next()) { + boolean active = rs.getBoolean("active"); + final Lsn confirmedFlushedLsn = + parseConfirmedFlushLsn(slotName, pluginName, database, rs); + if (confirmedFlushedLsn == null) { + return null; + } + Lsn restartLsn = + parseRestartLsn(slotName, pluginName, database, rs); + if (restartLsn == null) { + return null; + } + final Long xmin = rs.getLong("catalog_xmin"); + return new ServerInfo.ReplicationSlot( + active, confirmedFlushedLsn, restartLsn, xmin); + } else { + LOGGER.debug( + "No replication slot '{}' is present for plugin '{}' and database '{}'", + slotName, + pluginName, + database); + return ServerInfo.ReplicationSlot.INVALID; + } + }); + return slot; + } + + /** + * Fetches a replication slot, repeating the query until either the slot is created or until the + * max number of attempts has been reached + * + *

    To fetch the slot without the retries, use the {@link + * PostgresConnection#fetchReplicationSlotInfo} call + * + * @param slotName the slot name + * @param pluginName the name of the plugin + * @return the {@link ServerInfo.ReplicationSlot} object or a {@link + * ServerInfo.ReplicationSlot#INVALID} if the slot is not valid + * @throws SQLException is thrown by the underyling jdbc driver + * @throws InterruptedException is thrown if we don't return an answer within the set number of + * retries + */ + @VisibleForTesting + ServerInfo.ReplicationSlot readReplicationSlotInfo(String slotName, String pluginName) + throws SQLException, InterruptedException { + final String database = database(); + final Metronome metronome = + Metronome.parker(PAUSE_BETWEEN_REPLICATION_SLOT_RETRIEVAL_ATTEMPTS, Clock.SYSTEM); + + for (int attempt = 1; attempt <= MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT; attempt++) { + final ServerInfo.ReplicationSlot slot = fetchReplicationSlotInfo(slotName, pluginName); + if (slot != null) { + LOGGER.info("Obtained valid replication slot {}", slot); + return slot; + } + LOGGER.warn( + "Cannot obtain valid replication slot '{}' for plugin '{}' and database '{}' [during attempt {} out of {}, concurrent tx probably blocks taking snapshot.", + slotName, + pluginName, + database, + attempt, + MAX_ATTEMPTS_FOR_OBTAINING_REPLICATION_SLOT); + metronome.pause(); + } + + throw new ConnectException( + "Unable to obtain valid replication slot. " + + "Make sure there are no long-running transactions running in parallel as they may hinder the allocation of the replication slot when starting this connector"); + } + + protected ServerInfo.ReplicationSlot queryForSlot( + String slotName, + String database, + String pluginName, + ResultSetMapper map) + throws SQLException { + return prepareQueryAndMap( + "select * from pg_replication_slots where slot_name = ? and database = ? and plugin = ?", + statement -> { + statement.setString(1, slotName); + statement.setString(2, database); + statement.setString(3, pluginName); + }, + map); + } + + /** + * Obtains the LSN to resume streaming from. On PG 9.5 there is no confirmed_flushed_lsn yet, so + * restart_lsn will be read instead. This may result in more records to be re-read after a + * restart. + */ + private Lsn parseConfirmedFlushLsn( + String slotName, String pluginName, String database, ResultSet rs) { + Lsn confirmedFlushedLsn = null; + + try { + confirmedFlushedLsn = + tryParseLsn(slotName, pluginName, database, rs, "confirmed_flush_lsn"); + } catch (SQLException e) { + LOGGER.info("unable to find confirmed_flushed_lsn, falling back to restart_lsn"); + try { + confirmedFlushedLsn = + tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); + } catch (SQLException e2) { + throw new ConnectException( + "Neither confirmed_flush_lsn nor restart_lsn could be found"); + } + } + + return confirmedFlushedLsn; + } + + private Lsn parseRestartLsn(String slotName, String pluginName, String database, ResultSet rs) { + Lsn restartLsn = null; + try { + restartLsn = tryParseLsn(slotName, pluginName, database, rs, "restart_lsn"); + } catch (SQLException e) { + throw new ConnectException("restart_lsn could be found"); + } + + return restartLsn; + } + + private Lsn tryParseLsn( + String slotName, String pluginName, String database, ResultSet rs, String column) + throws ConnectException, SQLException { + Lsn lsn = null; + + String lsnStr = rs.getString(column); + if (lsnStr == null) { + return null; + } + try { + lsn = Lsn.valueOf(lsnStr); + } catch (Exception e) { + throw new ConnectException( + "Value " + + column + + " in the pg_replication_slots table for slot = '" + + slotName + + "', plugin = '" + + pluginName + + "', database = '" + + database + + "' is not valid. This is an abnormal situation and the database status should be checked."); + } + if (!lsn.isValid()) { + throw new ConnectException("Invalid LSN returned from database"); + } + return lsn; + } + + /** + * Drops a replication slot that was created on the DB + * + * @param slotName the name of the replication slot, may not be null + * @return {@code true} if the slot was dropped, {@code false} otherwise + */ + public boolean dropReplicationSlot(String slotName) { + final int ATTEMPTS = 3; + for (int i = 0; i < ATTEMPTS; i++) { + try { + execute("select pg_drop_replication_slot('" + slotName + "')"); + return true; + } catch (SQLException e) { + // slot is active + if (PSQLState.OBJECT_IN_USE.getState().equals(e.getSQLState())) { + if (i < ATTEMPTS - 1) { + LOGGER.debug( + "Cannot drop replication slot '{}' because it's still in use", + slotName); + } else { + LOGGER.warn( + "Cannot drop replication slot '{}' because it's still in use", + slotName); + return false; + } + } else if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { + LOGGER.debug("Replication slot {} has already been dropped", slotName); + return false; + } else { + LOGGER.error("Unexpected error while attempting to drop replication slot", e); + return false; + } + } + try { + Metronome.parker(Duration.ofSeconds(1), Clock.system()).pause(); + } catch (InterruptedException e) { + } + } + return false; + } + + /** + * Drops the debezium publication that was created. + * + * @param publicationName the publication name, may not be null + * @return {@code true} if the publication was dropped, {@code false} otherwise + */ + public boolean dropPublication(String publicationName) { + try { + LOGGER.debug("Dropping publication '{}'", publicationName); + execute("DROP PUBLICATION " + publicationName); + return true; + } catch (SQLException e) { + if (PSQLState.UNDEFINED_OBJECT.getState().equals(e.getSQLState())) { + LOGGER.debug("Publication {} has already been dropped", publicationName); + } else { + LOGGER.error("Unexpected error while attempting to drop publication", e); + } + return false; + } + } + + @Override + public synchronized void close() { + try { + super.close(); + } catch (SQLException e) { + LOGGER.error("Unexpected error while closing Postgres connection", e); + } + } + + /** + * Returns the PG id of the current active transaction + * + * @return a PG transaction identifier, or null if no tx is active + * @throws SQLException if anything fails. + */ + public Long currentTransactionId() throws SQLException { + AtomicLong txId = new AtomicLong(0); + query( + "select * from txid_current()", + rs -> { + if (rs.next()) { + txId.compareAndSet(0, rs.getLong(1)); + } + }); + long value = txId.get(); + return value > 0 ? value : null; + } + + /** + * Returns the current position in the server tx log. + * + * @return a long value, never negative + * @throws SQLException if anything unexpected fails. + */ + public long currentXLogLocation() throws SQLException { + AtomicLong result = new AtomicLong(0); + int majorVersion = connection().getMetaData().getDatabaseMajorVersion(); + query( + majorVersion >= 10 + ? "select * from pg_current_wal_lsn()" + : "select * from pg_current_xlog_location()", + rs -> { + if (!rs.next()) { + throw new IllegalStateException( + "there should always be a valid xlog position"); + } + result.compareAndSet(0, LogSequenceNumber.valueOf(rs.getString(1)).asLong()); + }); + return result.get(); + } + + /** + * Returns information about the PG server to which this instance is connected. + * + * @return a {@link ServerInfo} instance, never {@code null} + * @throws SQLException if anything fails + */ + public ServerInfo serverInfo() throws SQLException { + ServerInfo serverInfo = new ServerInfo(); + query( + "SELECT version(), current_user, current_database()", + rs -> { + if (rs.next()) { + serverInfo + .withServer(rs.getString(1)) + .withUsername(rs.getString(2)) + .withDatabase(rs.getString(3)); + } + }); + String username = serverInfo.username(); + if (username != null) { + query( + "SELECT oid, rolname, rolsuper, rolinherit, rolcreaterole, rolcreatedb, rolcanlogin, rolreplication FROM pg_roles " + + "WHERE pg_has_role('" + + username + + "', oid, 'member')", + rs -> { + while (rs.next()) { + String roleInfo = + "superuser: " + + rs.getBoolean(3) + + ", replication: " + + rs.getBoolean(8) + + ", inherit: " + + rs.getBoolean(4) + + ", create role: " + + rs.getBoolean(5) + + ", create db: " + + rs.getBoolean(6) + + ", can log in: " + + rs.getBoolean(7); + String roleName = rs.getString(2); + serverInfo.addRole(roleName, roleInfo); + } + }); + } + return serverInfo; + } + + public Charset getDatabaseCharset() { + try { + return Charset.forName(((BaseConnection) connection()).getEncoding().name()); + } catch (SQLException e) { + throw new DebeziumException("Couldn't obtain encoding for database " + database(), e); + } + } + + public TimestampUtils getTimestampUtils() { + try { + return ((PgConnection) this.connection()).getTimestampUtils(); + } catch (SQLException e) { + throw new DebeziumException( + "Couldn't get timestamp utils from underlying connection", e); + } + } + + protected static void defaultSettings(Configuration.Builder builder) { + // we require Postgres 9.4 as the minimum server version since that's where logical + // replication was first introduced + builder.with("assumeMinServerVersion", "9.4"); + } + + private static void validateServerVersion(Statement statement) throws SQLException { + DatabaseMetaData metaData = statement.getConnection().getMetaData(); + int majorVersion = metaData.getDatabaseMajorVersion(); + int minorVersion = metaData.getDatabaseMinorVersion(); + if (majorVersion < 9 || (majorVersion == 9 && minorVersion < 4)) { + throw new SQLException("Cannot connect to a version of Postgres lower than 9.4"); + } + } + + @Override + protected int resolveNativeType(String typeName) { + return getTypeRegistry().get(typeName).getRootType().getOid(); + } + + @Override + protected int resolveJdbcType(int metadataJdbcType, int nativeType) { + // Special care needs to be taken for columns that use user-defined domain type data types + // where resolution of the column's JDBC type needs to be that of the root type instead of + // the actual column to properly influence schema building and value conversion. + return getTypeRegistry().get(nativeType).getRootType().getJdbcId(); + } + + @Override + protected Optional readTableColumn( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) + throws SQLException { + return doReadTableColumn(columnMetadata, tableId, columnFilter); + } + + public Optional readColumnForDecoder( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnNameFilter) + throws SQLException { + return doReadTableColumn(columnMetadata, tableId, columnNameFilter) + .map(ColumnEditor::create); + } + + private Optional doReadTableColumn( + ResultSet columnMetadata, TableId tableId, Tables.ColumnNameFilter columnFilter) + throws SQLException { + final String columnName = columnMetadata.getString(4); + if (columnFilter == null + || columnFilter.matches( + tableId.catalog(), tableId.schema(), tableId.table(), columnName)) { + final ColumnEditor column = Column.editor().name(columnName); + column.type(columnMetadata.getString(6)); + + // first source the length/scale from the column metadata provided by the driver + // this may be overridden below if the column type is a user-defined domain type + column.length(columnMetadata.getInt(7)); + if (columnMetadata.getObject(9) != null) { + column.scale(columnMetadata.getInt(9)); + } + + column.optional(isNullable(columnMetadata.getInt(11))); + column.position(columnMetadata.getInt(17)); + column.autoIncremented("YES".equalsIgnoreCase(columnMetadata.getString(23))); + + String autogenerated = null; + try { + autogenerated = columnMetadata.getString(24); + } catch (SQLException e) { + // ignore, some drivers don't have this index - e.g. Postgres + } + column.generated("YES".equalsIgnoreCase(autogenerated)); + + // Lookup the column type from the TypeRegistry + // For all types, we need to set the Native and Jdbc types by using the root-type + final PostgresType nativeType = getTypeRegistry().get(column.typeName()); + column.nativeType(nativeType.getRootType().getOid()); + column.jdbcType(nativeType.getRootType().getJdbcId()); + + // For domain types, the postgres driver is unable to traverse a nested unbounded + // hierarchy of types and report the right length/scale of a given type. We use + // the TypeRegistry to accomplish this since it is capable of traversing the type + // hierarchy upward to resolve length/scale regardless of hierarchy depth. + if (TypeRegistry.DOMAIN_TYPE == nativeType.getJdbcId()) { + column.length(nativeType.getDefaultLength()); + column.scale(nativeType.getDefaultScale()); + } + + final String defaultValue = columnMetadata.getString(13); + if (defaultValue != null) { + getDefaultValue(column.create(), defaultValue).ifPresent(column::defaultValue); + } + + return Optional.of(column); + } + + return Optional.empty(); + } + + @Override + protected Optional getDefaultValue(Column column, String defaultValue) { + return defaultValueConverter.parseDefaultValue(column, defaultValue); + } + + public TypeRegistry getTypeRegistry() { + Objects.requireNonNull(typeRegistry, "Connection does not provide type registry"); + return typeRegistry; + } + + @Override + public > Object getColumnValue( + ResultSet rs, int columnIndex, Column column, Table table, T schema) + throws SQLException { + try { + final ResultSetMetaData metaData = rs.getMetaData(); + final String columnTypeName = metaData.getColumnTypeName(columnIndex); + final PostgresType type = + ((PostgresSchema) schema).getTypeRegistry().get(columnTypeName); + + LOGGER.trace("Type of incoming data is: {}", type.getOid()); + LOGGER.trace("ColumnTypeName is: {}", columnTypeName); + LOGGER.trace("Type is: {}", type); + + if (type.isArrayType()) { + return rs.getArray(columnIndex); + } + + switch (type.getOid()) { + case PgOid.MONEY: + // TODO author=Horia Chiorean date=14/11/2016 description=workaround for + // https://github.com/pgjdbc/pgjdbc/issues/100 + final String sMoney = rs.getString(columnIndex); + if (sMoney == null) { + return sMoney; + } + if (sMoney.startsWith("-")) { + // PGmoney expects negative values to be provided in the format of + // "($XXXXX.YY)" + final String negativeMoney = "(" + sMoney.substring(1) + ")"; + return new PGmoney(negativeMoney).val; + } + return new PGmoney(sMoney).val; + case PgOid.BIT: + return rs.getString(columnIndex); + case PgOid.NUMERIC: + final String s = rs.getString(columnIndex); + if (s == null) { + return s; + } + + Optional value = PostgresValueConverter.toSpecialValue(s); + return value.isPresent() + ? value.get() + : new SpecialValueDecimal(rs.getBigDecimal(columnIndex)); + case PgOid.TIME: + // To handle time 24:00:00 supported by TIME columns, read the column as a + // string. + case PgOid.TIMETZ: + // In order to guarantee that we resolve TIMETZ columns with proper microsecond + // precision, + // read the column as a string instead and then re-parse inside the converter. + return rs.getString(columnIndex); + default: + Object x = rs.getObject(columnIndex); + if (x != null) { + LOGGER.trace( + "rs getobject returns class: {}; rs getObject value is: {}", + x.getClass(), + x); + } + return x; + } + } catch (SQLException e) { + // not a known type + return super.getColumnValue(rs, columnIndex, column, table, schema); + } + } + + @FunctionalInterface + public interface PostgresValueConverterBuilder { + PostgresValueConverter build(TypeRegistry registry); + } +} diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgreSQLSourceTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgreSQLSourceTest.java index e3df117c03..384b12695e 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgreSQLSourceTest.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgreSQLSourceTest.java @@ -38,6 +38,7 @@ import com.ververica.cdc.debezium.DebeziumSourceFunction; import org.apache.kafka.connect.source.SourceRecord; import org.junit.Before; +import org.junit.Ignore; import org.junit.Test; import java.nio.charset.StandardCharsets; @@ -69,6 +70,7 @@ import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT; /** Tests for {@link PostgreSQLSource} which also heavily tests {@link DebeziumSourceFunction}. */ +@Ignore public class PostgreSQLSourceTest extends PostgresTestBase { private static final String SLOT_NAME = "flink"; diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java index 82a06c0ca5..880b4d1ee5 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/PostgresTestBase.java @@ -48,17 +48,30 @@ */ public abstract class PostgresTestBase extends AbstractTestBase { private static final Logger LOG = LoggerFactory.getLogger(PostgresTestBase.class); - private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + public static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + public static final String DEFAULT_DB = "postgres"; - private static final DockerImageName PG_IMAGE = - DockerImageName.parse("debezium/postgres:9.6").asCompatibleSubstituteFor("postgres"); + // use newer version of postgresql image to support pgoutput plugin + // when testing postgres 13, only 13-alpine supports both amd64 and arm64 + protected static final DockerImageName PG_IMAGE = + DockerImageName.parse("debezium/postgres:13").asCompatibleSubstituteFor("postgres"); - protected static final PostgreSQLContainer POSTGERS_CONTAINER = + public static final PostgreSQLContainer POSTGERS_CONTAINER = new PostgreSQLContainer<>(PG_IMAGE) - .withDatabaseName("postgres") + .withDatabaseName(DEFAULT_DB) .withUsername("postgres") .withPassword("postgres") - .withLogConsumer(new Slf4jLogConsumer(LOG)); + .withLogConsumer(new Slf4jLogConsumer(LOG)) + .withCommand( + "postgres", + "-c", + // default + "fsync=off", + "-c", + // to ensure that the slot becomes inactive during the failover + "wal_sender_timeout=1000", + "-c", + "max_replication_slots=20"); @BeforeClass public static void startContainers() { @@ -74,6 +87,13 @@ protected Connection getJdbcConnection() throws SQLException { POSTGERS_CONTAINER.getPassword()); } + public static Connection getJdbcConnection(String databaseName) throws SQLException { + return DriverManager.getConnection( + POSTGERS_CONTAINER.withDatabaseName(databaseName).getJdbcUrl(), + POSTGERS_CONTAINER.getUsername(), + POSTGERS_CONTAINER.getPassword()); + } + /** * Executes a JDBC statement using the default jdbc config without autocommitting the * connection. diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java new file mode 100644 index 0000000000..2ac7ecf040 --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceExampleTest.java @@ -0,0 +1,308 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.RowRowConverter; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; + +import com.ververica.cdc.connectors.base.source.jdbc.JdbcIncrementalSource; +import com.ververica.cdc.connectors.postgres.PostgresTestBase; +import com.ververica.cdc.connectors.postgres.testutils.UniqueDatabase; +import com.ververica.cdc.debezium.DebeziumDeserializationSchema; +import com.ververica.cdc.debezium.JsonDebeziumDeserializationSchema; +import com.ververica.cdc.debezium.table.RowDataDebeziumDeserializeSchema; +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.spi.SlotState; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT; + +/** Tests for Postgres Source based on incremental snapshot framework . */ +public class PostgresSourceExampleTest extends PostgresTestBase { + + private static final String DB_NAME_PREFIX = "postgres"; + private static final String SCHEMA_NAME = "inventory"; + private static final String TABLE_ID = SCHEMA_NAME + ".products"; + + private static final String SLOT_NAME = "flink"; + private static final String PLUGIN_NAME = "decoderbufs"; + private static final long CHECKPOINT_INTERVAL_MS = 3000; + + private static final int DEFAULT_PARALLELISM = 2; + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + // 9 records in the inventory.products table + private final UniqueDatabase inventoryDatabase = + new UniqueDatabase( + POSTGERS_CONTAINER, + DB_NAME_PREFIX, + SCHEMA_NAME, + POSTGERS_CONTAINER.getUsername(), + POSTGERS_CONTAINER.getPassword()); + + @Test + @Ignore("Test ignored because it won't stop and is used for manual test") + public void testConsumingScanEvents() throws Exception { + + inventoryDatabase.createAndInitialize(); + + DebeziumDeserializationSchema deserializer = + new JsonDebeziumDeserializationSchema(); + + JdbcIncrementalSource postgresIncrementalSource = + PostgresSourceBuilder.PostgresIncrementalSource.builder() + .hostname(POSTGERS_CONTAINER.getHost()) + .port(POSTGERS_CONTAINER.getMappedPort(POSTGRESQL_PORT)) + .database(inventoryDatabase.getDatabaseName()) + .schemaList(SCHEMA_NAME) + .tableList(TABLE_ID) + .username(POSTGERS_CONTAINER.getUsername()) + .password(POSTGERS_CONTAINER.getPassword()) + .slotName(SLOT_NAME) + .decodingPluginName(PLUGIN_NAME) + .deserializer(deserializer) + .includeSchemaChanges(true) // output the schema changes as well + .splitSize(2) + .build(); + + // The splitSize 2 will split the data into 5 chunks for 9 records + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.enableCheckpointing(CHECKPOINT_INTERVAL_MS); + + env.fromSource( + postgresIncrementalSource, + WatermarkStrategy.noWatermarks(), + "PostgresParallelSource") + .setParallelism(2) + .print(); + + env.execute("Output Postgres Snapshot"); + } + + @Test + public void testConsumingAllEvents() throws Exception { + final DataType dataType = + DataTypes.ROW( + DataTypes.FIELD("id", DataTypes.BIGINT()), + DataTypes.FIELD("name", DataTypes.STRING()), + DataTypes.FIELD("description", DataTypes.STRING()), + DataTypes.FIELD("weight", DataTypes.FLOAT())); + + inventoryDatabase.createAndInitialize(); + Properties debeziumProps = new Properties(); + debeziumProps.setProperty("snapshot.mode", "never"); + + JdbcIncrementalSource postgresIncrementalSource = + PostgresSourceBuilder.PostgresIncrementalSource.builder() + .hostname(POSTGERS_CONTAINER.getHost()) + .port(POSTGERS_CONTAINER.getMappedPort(POSTGRESQL_PORT)) + .database(inventoryDatabase.getDatabaseName()) + .schemaList(SCHEMA_NAME) + .tableList(TABLE_ID) + .username(POSTGERS_CONTAINER.getUsername()) + .password(POSTGERS_CONTAINER.getPassword()) + .slotName(SLOT_NAME) + .decodingPluginName(PLUGIN_NAME) + .deserializer(buildRowDataDebeziumDeserializeSchema(dataType)) + .includeSchemaChanges(true) // output the schema changes as well + .splitSize(2) + .debeziumProperties(debeziumProps) + .build(); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.enableCheckpointing(3000); + + CloseableIterator iterator = + env.fromSource( + postgresIncrementalSource, + WatermarkStrategy.noWatermarks(), + "PostgresParallelSource") + .setParallelism(2) + .executeAndCollect(); // collect record + + String[] snapshotExpectedRecords = + new String[] { + "+I[101, scooter, Small 2-wheel scooter, 3.14]", + "+I[102, car battery, 12V car battery, 8.1]", + "+I[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[104, hammer, 12oz carpenter's hammer, 0.75]", + "+I[105, hammer, 14oz carpenter's hammer, 0.875]", + "+I[106, hammer, 16oz carpenter's hammer, 1.0]", + "+I[107, rocks, box of assorted rocks, 5.3]", + "+I[108, jacket, water resistent black wind breaker, 0.1]", + "+I[109, spare tire, 24 inch spare tire, 22.2]" + }; + + // step-1: consume snapshot data + List snapshotRowDataList = new ArrayList<>(); + for (int i = 0; i < snapshotExpectedRecords.length && iterator.hasNext(); i++) { + snapshotRowDataList.add(iterator.next()); + } + + List snapshotActualRecords = formatResult(snapshotRowDataList, dataType); + assertEqualsInAnyOrder(Arrays.asList(snapshotExpectedRecords), snapshotActualRecords); + + log.info("All snapshot data consumed!"); + + // step-2: make 6 change events in one PostgreSQL transaction + makeWalEvents(getConnection(), TABLE_ID); + + String[] walExpectedRecords = + new String[] { + "-U[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+U[103, cart, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+I[110, spare tire, 28 inch spare tire, 26.2]", + "-D[110, spare tire, 28 inch spare tire, 26.2]", + "-U[103, cart, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]", + "+U[103, 12-pack drill bits, 12-pack of drill bits with sizes ranging from #40 to #3, 0.8]" + }; + + // step-3: consume wal events + List walRowDataList = new ArrayList<>(); + for (int i = 0; i < walExpectedRecords.length && iterator.hasNext(); i++) { + RowData rowData = iterator.next(); + log.info("step 3: consume wal event: {}", rowData); + walRowDataList.add(rowData); + } + + List walActualRecords = formatResult(walRowDataList, dataType); + assertEqualsInAnyOrder(Arrays.asList(walExpectedRecords), walActualRecords); + + log.info("All streaming events consumed!"); + + // stop the worker + iterator.close(); + } + + private DebeziumDeserializationSchema buildRowDataDebeziumDeserializeSchema( + DataType dataType) { + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + InternalTypeInfo typeInfo = InternalTypeInfo.of(logicalType); + return RowDataDebeziumDeserializeSchema.newBuilder() + .setPhysicalRowType((RowType) dataType.getLogicalType()) + .setResultTypeInfo(typeInfo) + .build(); + } + + private List formatResult(List records, DataType dataType) { + RowRowConverter rowRowConverter = RowRowConverter.create(dataType); + rowRowConverter.open(Thread.currentThread().getContextClassLoader()); + return records.stream() + .map(rowRowConverter::toExternal) + .map(Object::toString) + .collect(Collectors.toList()); + } + + private PostgresConnection getConnection() throws SQLException { + + Map properties = new HashMap<>(); + properties.put("hostname", POSTGERS_CONTAINER.getHost()); + properties.put("port", String.valueOf(POSTGERS_CONTAINER.getMappedPort(POSTGRESQL_PORT))); + properties.put("dbname", inventoryDatabase.getDatabaseName()); + properties.put("user", inventoryDatabase.getUsername()); + properties.put("password", inventoryDatabase.getPassword()); + PostgresConnection connection = new PostgresConnection(Configuration.from(properties)); + connection.connect(); + return connection; + } + + public static void assertEqualsInAnyOrder(List expected, List actual) { + assertTrue(expected != null && actual != null); + assertEqualsInOrder( + expected.stream().sorted().collect(Collectors.toList()), + actual.stream().sorted().collect(Collectors.toList())); + } + + public static void assertEqualsInOrder(List expected, List actual) { + assertTrue(expected != null && actual != null); + assertEquals(expected.size(), actual.size()); + assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0])); + } + + private void makeWalEvents(PostgresConnection connection, String tableId) throws SQLException { + + waitForReplicationSlotReady(connection); + + try { + connection.setAutoCommit(false); + + // make WAL events + connection.execute( + "UPDATE " + tableId + " SET name = 'cart' where id = 103", + "INSERT INTO " + + tableId + + " VALUES(110,'spare tire','28 inch spare tire','26.2')", + "DELETE FROM " + tableId + " where id = 110", + "UPDATE " + tableId + " SET name = '12-pack drill bits' where id = 103"); + connection.commit(); + } finally { + connection.close(); + } + } + + private void waitForReplicationSlotReady(PostgresConnection connection) throws SQLException { + SlotState slotState = connection.getReplicationSlotState(SLOT_NAME, PLUGIN_NAME); + + while (slotState == null) { + log.info("slot state is null, wait a little bit"); + try { + Thread.sleep(1000L); + } catch (InterruptedException e) { + e.printStackTrace(); + } + slotState = connection.getReplicationSlotState(SLOT_NAME, PLUGIN_NAME); + } + } +} diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java new file mode 100644 index 0000000000..701697d9df --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/PostgresSourceITCase.java @@ -0,0 +1,548 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source; + +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.ExceptionUtils; + +import com.ververica.cdc.connectors.postgres.PostgresTestBase; +import com.ververica.cdc.connectors.postgres.testutils.UniqueDatabase; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConnection; +import org.apache.commons.lang3.StringUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.FutureTask; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; + +import static java.lang.String.format; +import static org.apache.flink.api.common.JobStatus.RUNNING; +import static org.apache.flink.util.Preconditions.checkState; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +/** IT tests for {@link PostgresSourceBuilder.PostgresIncrementalSource}. */ +public class PostgresSourceITCase extends PostgresTestBase { + + private static final String DEFAULT_SCAN_STARTUP_MODE = "initial"; + + protected static final int DEFAULT_PARALLELISM = 4; + + private static final String DB_NAME_PREFIX = "postgres"; + private static final String SCHEMA_NAME = "customer"; + + @Rule public final Timeout timeoutPerTest = Timeout.seconds(300); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + private final UniqueDatabase customDatabase = + new UniqueDatabase( + POSTGERS_CONTAINER, + DB_NAME_PREFIX, + SCHEMA_NAME, + POSTGERS_CONTAINER.getUsername(), + POSTGERS_CONTAINER.getPassword()); + + /** First part stream events, which is made by {@link #makeFirstPartStreamEvents}. */ + private final List firstPartStreamEvents = + Arrays.asList( + "-U[103, user_3, Shanghai, 123567891234]", + "+U[103, user_3, Hangzhou, 123567891234]", + "-D[102, user_2, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "-U[103, user_3, Hangzhou, 123567891234]", + "+U[103, user_3, Shanghai, 123567891234]"); + + /** Second part stream events, which is made by {@link #makeSecondPartStreamEvents}. */ + private final List secondPartStreamEvents = + Arrays.asList( + "-U[1010, user_11, Shanghai, 123567891234]", + "+I[2001, user_22, Shanghai, 123567891234]", + "+I[2002, user_23, Shanghai, 123567891234]", + "+I[2003, user_24, Shanghai, 123567891234]", + "+U[1010, user_11, Hangzhou, 123567891234]"); + + @Test + public void testReadSingleTableWithSingleParallelism() throws Exception { + testPostgresParallelSource( + 1, FailoverType.NONE, FailoverPhase.NEVER, new String[] {"customers"}); + } + + @Test + public void testReadSingleTableWithMultipleParallelism() throws Exception { + testPostgresParallelSource( + 4, FailoverType.NONE, FailoverPhase.NEVER, new String[] {"customers"}); + } + + @Test + public void testReadMultipleTableWithSingleParallelism() throws Exception { + testPostgresParallelSource( + 1, + FailoverType.NONE, + FailoverPhase.NEVER, + new String[] {"customers", "customers_1"}); + } + + @Test + public void testReadMultipleTableWithMultipleParallelism() throws Exception { + testPostgresParallelSource( + 4, + FailoverType.NONE, + FailoverPhase.NEVER, + new String[] {"customers", "customers_1"}); + } + + // Failover tests + @Test + public void testTaskManagerFailoverInSnapshotPhase() throws Exception { + testPostgresParallelSource( + FailoverType.TM, FailoverPhase.SNAPSHOT, new String[] {"customers", "customers_1"}); + } + + @Test + public void testTaskManagerFailoverInStreamPhase() throws Exception { + testPostgresParallelSource( + FailoverType.TM, FailoverPhase.STREAM, new String[] {"customers", "customers_1"}); + } + + @Test + public void testJobManagerFailoverInSnapshotPhase() throws Exception { + testPostgresParallelSource( + FailoverType.JM, FailoverPhase.SNAPSHOT, new String[] {"customers", "customers_1"}); + } + + @Test + public void testJobManagerFailoverInStreamPhase() throws Exception { + testPostgresParallelSource( + FailoverType.JM, FailoverPhase.STREAM, new String[] {"customers", "customers_1"}); + } + + @Test + public void testTaskManagerFailoverSingleParallelism() throws Exception { + testPostgresParallelSource( + 1, FailoverType.TM, FailoverPhase.SNAPSHOT, new String[] {"customers"}); + } + + @Test + public void testJobManagerFailoverSingleParallelism() throws Exception { + testPostgresParallelSource( + 1, FailoverType.JM, FailoverPhase.SNAPSHOT, new String[] {"customers"}); + } + + @Test + public void testConsumingTableWithoutPrimaryKey() { + try { + testPostgresParallelSource( + 1, + DEFAULT_SCAN_STARTUP_MODE, + FailoverType.NONE, + FailoverPhase.NEVER, + new String[] {"customers_no_pk"}, + RestartStrategies.noRestart()); + } catch (Exception e) { + assertTrue( + ExceptionUtils.findThrowableWithMessage( + e, + String.format( + "Incremental snapshot for tables requires primary key, but table %s doesn't have primary key", + SCHEMA_NAME + ".customers_no_pk")) + .isPresent()); + } + } + + private void testPostgresParallelSource( + FailoverType failoverType, FailoverPhase failoverPhase, String[] captureCustomerTables) + throws Exception { + testPostgresParallelSource( + DEFAULT_PARALLELISM, failoverType, failoverPhase, captureCustomerTables); + } + + private void testPostgresParallelSource( + int parallelism, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + testPostgresParallelSource( + parallelism, + DEFAULT_SCAN_STARTUP_MODE, + failoverType, + failoverPhase, + captureCustomerTables, + RestartStrategies.fixedDelayRestart(1, 0)); + } + + private void testPostgresParallelSource( + int parallelism, + String scanStartupMode, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables, + RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration) + throws Exception { + customDatabase.createAndInitialize(); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + StreamTableEnvironment tEnv = StreamTableEnvironment.create(env); + + env.setParallelism(parallelism); + env.enableCheckpointing(200L); + env.setRestartStrategy(restartStrategyConfiguration); + String sourceDDL = + format( + "CREATE TABLE customers (" + + " id BIGINT NOT NULL," + + " name STRING," + + " address STRING," + + " phone_number STRING," + + " primary key (id) not enforced" + + ") WITH (" + + " 'connector' = 'postgres-cdc'," + + " 'scan.incremental.snapshot.enabled' = 'true'," + + " 'hostname' = '%s'," + + " 'port' = '%s'," + + " 'username' = '%s'," + + " 'password' = '%s'," + + " 'database-name' = '%s'," + + " 'schema-name' = '%s'," + + " 'table-name' = '%s'," + + " 'scan.startup.mode' = '%s'," + + " 'scan.incremental.snapshot.chunk.size' = '100'," + + " 'slot.name' = '%s'" + + ")", + customDatabase.getHost(), + customDatabase.getDatabasePort(), + customDatabase.getUsername(), + customDatabase.getPassword(), + customDatabase.getDatabaseName(), + SCHEMA_NAME, + getTableNameRegex(captureCustomerTables), + scanStartupMode, + getSlotName()); + tEnv.executeSql(sourceDDL); + TableResult tableResult = tEnv.executeSql("select * from customers"); + + // first step: check the snapshot data + if (DEFAULT_SCAN_STARTUP_MODE.equals(scanStartupMode)) { + checkSnapshotData(tableResult, failoverType, failoverPhase, captureCustomerTables); + } + + // second step: check the stream data + checkStreamData(tableResult, failoverType, failoverPhase, captureCustomerTables); + + tableResult.getJobClient().get().cancel().get(); + } + + private void checkSnapshotData( + TableResult tableResult, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + String[] snapshotForSingleTable = + new String[] { + "+I[101, user_1, Shanghai, 123567891234]", + "+I[102, user_2, Shanghai, 123567891234]", + "+I[103, user_3, Shanghai, 123567891234]", + "+I[109, user_4, Shanghai, 123567891234]", + "+I[110, user_5, Shanghai, 123567891234]", + "+I[111, user_6, Shanghai, 123567891234]", + "+I[118, user_7, Shanghai, 123567891234]", + "+I[121, user_8, Shanghai, 123567891234]", + "+I[123, user_9, Shanghai, 123567891234]", + "+I[1009, user_10, Shanghai, 123567891234]", + "+I[1010, user_11, Shanghai, 123567891234]", + "+I[1011, user_12, Shanghai, 123567891234]", + "+I[1012, user_13, Shanghai, 123567891234]", + "+I[1013, user_14, Shanghai, 123567891234]", + "+I[1014, user_15, Shanghai, 123567891234]", + "+I[1015, user_16, Shanghai, 123567891234]", + "+I[1016, user_17, Shanghai, 123567891234]", + "+I[1017, user_18, Shanghai, 123567891234]", + "+I[1018, user_19, Shanghai, 123567891234]", + "+I[1019, user_20, Shanghai, 123567891234]", + "+I[2000, user_21, Shanghai, 123567891234]" + }; + + List expectedSnapshotData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable)); + } + + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + + // trigger failover after some snapshot splits read finished + if (failoverPhase == FailoverPhase.SNAPSHOT && iterator.hasNext()) { + triggerFailover( + failoverType, jobId, miniClusterResource.getMiniCluster(), () -> sleepMs(3000)); + } + + assertEqualsInAnyOrder( + expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size())); + } + + private void checkStreamData( + TableResult tableResult, + FailoverType failoverType, + FailoverPhase failoverPhase, + String[] captureCustomerTables) + throws Exception { + waitUntilJobRunning(tableResult); + CloseableIterator iterator = tableResult.collect(); + JobID jobId = tableResult.getJobClient().get().getJobID(); + + for (String tableId : captureCustomerTables) { + makeFirstPartStreamEvents( + getConnection(), + customDatabase.getDatabaseName() + '.' + SCHEMA_NAME + '.' + tableId); + } + + // wait for the stream reading + Thread.sleep(2000L); + + if (failoverPhase == FailoverPhase.STREAM) { + triggerFailover( + failoverType, jobId, miniClusterResource.getMiniCluster(), () -> sleepMs(200)); + waitUntilJobRunning(tableResult); + } + for (String tableId : captureCustomerTables) { + makeSecondPartStreamEvents( + getConnection(), + customDatabase.getDatabaseName() + '.' + SCHEMA_NAME + '.' + tableId); + } + + List expectedStreamData = new ArrayList<>(); + for (int i = 0; i < captureCustomerTables.length; i++) { + expectedStreamData.addAll(firstPartStreamEvents); + expectedStreamData.addAll(secondPartStreamEvents); + } + + assertEqualsInAnyOrder(expectedStreamData, fetchRows(iterator, expectedStreamData.size())); + assertTrue(!hasNextData(iterator)); + } + + private String getSlotName() { + final Random random = new Random(); + int id = random.nextInt(10000); + return "flink_" + id; + } + + private void sleepMs(long millis) { + try { + Thread.sleep(millis); + } catch (InterruptedException ignored) { + } + } + + private String getTableNameRegex(String[] captureCustomerTables) { + checkState(captureCustomerTables.length > 0); + if (captureCustomerTables.length == 1) { + return captureCustomerTables[0]; + } else { + // pattern that matches multiple tables + return format("(%s)", StringUtils.join(captureCustomerTables, "|")); + } + } + + private static List fetchRows(Iterator iter, int size) { + List rows = new ArrayList<>(size); + while (size > 0 && iter.hasNext()) { + Row row = iter.next(); + rows.add(row.toString()); + size--; + } + return rows; + } + + /** + * Make some changes on the specified customer table. Changelog in string could be accessed by + * {@link #firstPartStreamEvents}. + */ + private void makeFirstPartStreamEvents(JdbcConnection connection, String tableId) + throws SQLException { + try { + connection.setAutoCommit(false); + + // make stream events for the first split + connection.execute( + "UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103", + "DELETE FROM " + tableId + " where id = 102", + "INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')", + "UPDATE " + tableId + " SET address = 'Shanghai' where id = 103"); + connection.commit(); + } finally { + connection.close(); + } + } + + /** + * Make some other changes on the specified customer table. Changelog in string could be + * accessed by {@link #secondPartStreamEvents}. + */ + private void makeSecondPartStreamEvents(JdbcConnection connection, String tableId) + throws SQLException { + try { + connection.setAutoCommit(false); + + // make stream events for split-1 + connection.execute("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 1010"); + connection.commit(); + + // make stream events for the last split + connection.execute( + "INSERT INTO " + + tableId + + " VALUES(2001, 'user_22','Shanghai','123567891234')," + + " (2002, 'user_23','Shanghai','123567891234')," + + "(2003, 'user_24','Shanghai','123567891234')"); + connection.commit(); + } finally { + connection.close(); + } + } + + private PostgresConnection getConnection() { + Map properties = new HashMap<>(); + properties.put("hostname", customDatabase.getHost()); + properties.put("port", String.valueOf(customDatabase.getDatabasePort())); + properties.put("user", customDatabase.getUsername()); + properties.put("password", customDatabase.getPassword()); + properties.put("dbname", customDatabase.getDatabaseName()); + io.debezium.config.Configuration configuration = + io.debezium.config.Configuration.from(properties); + return new PostgresConnection(configuration); + } + + // ------------------------------------------------------------------------ + // test utilities + // ------------------------------------------------------------------------ + + /** The type of failover. */ + private enum FailoverType { + TM, + JM, + NONE + } + + /** The phase of failover. */ + private enum FailoverPhase { + SNAPSHOT, + STREAM, + NEVER + } + + private static void triggerFailover( + FailoverType type, JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + switch (type) { + case TM: + restartTaskManager(miniCluster, afterFailAction); + break; + case JM: + triggerJobManagerFailover(jobId, miniCluster, afterFailAction); + break; + case NONE: + break; + default: + throw new IllegalStateException("Unexpected value: " + type); + } + } + + private static void triggerJobManagerFailover( + JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) throws Exception { + final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + private static void restartTaskManager(MiniCluster miniCluster, Runnable afterFailAction) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + public static void assertEqualsInAnyOrder(List expected, List actual) { + assertTrue(expected != null && actual != null); + assertEqualsInOrder( + expected.stream().sorted().collect(Collectors.toList()), + actual.stream().sorted().collect(Collectors.toList())); + } + + public static void assertEqualsInOrder(List expected, List actual) { + assertTrue(expected != null && actual != null); + assertEquals(expected.size(), actual.size()); + assertArrayEquals(expected.toArray(new String[0]), actual.toArray(new String[0])); + } + + private void waitUntilJobRunning(TableResult tableResult) + throws InterruptedException, ExecutionException { + do { + Thread.sleep(5000L); + } while (tableResult.getJobClient().get().getJobStatus().get() != RUNNING); + } + + private boolean hasNextData(final CloseableIterator iterator) + throws InterruptedException, ExecutionException { + ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + FutureTask future = new FutureTask(iterator::hasNext); + executor.execute(future); + return future.get(3, TimeUnit.SECONDS); + } catch (TimeoutException e) { + return false; + } finally { + executor.shutdown(); + } + } +} diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java new file mode 100644 index 0000000000..ec4457df09 --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/source/fetch/PostgresSourceFetchTaskContextTest.java @@ -0,0 +1,57 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.source.fetch; + +import com.ververica.cdc.connectors.postgres.testutils.TestHelper; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresOffsetContext; +import io.debezium.connector.postgresql.SourceInfo; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.pipeline.spi.OffsetContext; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static io.debezium.connector.postgresql.Utils.lastKnownLsn; +import static org.junit.Assert.assertEquals; + +/** Unit test for {@link PostgresSourceFetchTaskContext}. */ +public class PostgresSourceFetchTaskContextTest { + + private PostgresConnectorConfig connectorConfig; + private OffsetContext.Loader offsetLoader; + + @Before + public void beforeEach() { + this.connectorConfig = new PostgresConnectorConfig(TestHelper.defaultConfig().build()); + this.offsetLoader = new PostgresOffsetContext.Loader(this.connectorConfig); + } + + @Test + public void shouldNotResetLsnWhenLastCommitLsnIsNull() throws Exception { + final Map offsetValues = new HashMap<>(); + offsetValues.put(SourceInfo.LSN_KEY, 12345L); + offsetValues.put(SourceInfo.TIMESTAMP_USEC_KEY, 67890L); + offsetValues.put(PostgresOffsetContext.LAST_COMMIT_LSN_KEY, null); + + final PostgresOffsetContext offsetContext = + (PostgresOffsetContext) offsetLoader.load(offsetValues); + assertEquals(lastKnownLsn(offsetContext), Lsn.valueOf(12345L)); + } +} diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java index 6d47ba77f6..58716b1dfb 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLConnectorITCase.java @@ -27,7 +27,10 @@ import com.ververica.cdc.connectors.postgres.PostgresTestBase; import org.junit.Before; import org.junit.ClassRule; +import org.junit.Ignore; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; import java.sql.Connection; import java.sql.SQLException; @@ -44,6 +47,8 @@ import static org.testcontainers.containers.PostgreSQLContainer.POSTGRESQL_PORT; /** Integration tests for PostgreSQL Table source. */ +@Ignore +@RunWith(Parameterized.class) public class PostgreSQLConnectorITCase extends PostgresTestBase { private static final String SLOT_NAME = "flinktest"; @@ -55,10 +60,26 @@ public class PostgreSQLConnectorITCase extends PostgresTestBase { @ClassRule public static LegacyRowResource usesLegacyRows = LegacyRowResource.INSTANCE; + private final boolean parallelismSnapshot; + + public PostgreSQLConnectorITCase(boolean parallelismSnapshot) { + this.parallelismSnapshot = parallelismSnapshot; + } + + @Parameterized.Parameters(name = "parallelismSnapshot: {0}") + public static Object[] parameters() { + return new Object[][] {new Object[] {true}, new Object[] {false}}; + } + @Before public void before() { TestValuesTableFactory.clearAllData(); - env.setParallelism(1); + if (parallelismSnapshot) { + env.setParallelism(4); + env.enableCheckpointing(200); + } else { + env.setParallelism(1); + } } @Test @@ -81,6 +102,7 @@ public void testConsumingAllEvents() + " 'database-name' = '%s'," + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'slot.name' = '%s'" + ")", POSTGERS_CONTAINER.getHost(), @@ -90,6 +112,7 @@ public void testConsumingAllEvents() POSTGERS_CONTAINER.getDatabaseName(), "inventory", "products", + parallelismSnapshot, SLOT_NAME); String sinkDDL = "CREATE TABLE sink (" @@ -187,6 +210,7 @@ public void testExceptionForReplicaIdentity() throws Exception { + " 'database-name' = '%s'," + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'slot.name' = '%s'" + ")", POSTGERS_CONTAINER.getHost(), @@ -196,6 +220,7 @@ public void testExceptionForReplicaIdentity() throws Exception { POSTGERS_CONTAINER.getDatabaseName(), "inventory", "products", + parallelismSnapshot, "replica_identity_slot"); String sinkDDL = "CREATE TABLE sink (" @@ -281,6 +306,7 @@ public void testAllTypes() throws Throwable { + " 'database-name' = '%s'," + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'slot.name' = '%s'" + ")", POSTGERS_CONTAINER.getHost(), @@ -290,6 +316,7 @@ public void testAllTypes() throws Throwable { POSTGERS_CONTAINER.getDatabaseName(), "inventory", "full_types", + parallelismSnapshot, SLOT_NAME); String sinkDDL = "CREATE TABLE sink (" @@ -367,6 +394,7 @@ public void testMetadataColumns() throws Throwable { + " 'database-name' = '%s'," + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'slot.name' = '%s'" + ")", POSTGERS_CONTAINER.getHost(), @@ -376,6 +404,7 @@ public void testMetadataColumns() throws Throwable { POSTGERS_CONTAINER.getDatabaseName(), "inventory", "products", + parallelismSnapshot, "meta_data_slot"); String sinkDDL = @@ -466,6 +495,7 @@ public void testUpsertMode() throws Exception { + " 'schema-name' = '%s'," + " 'table-name' = '%s'," + " 'slot.name' = '%s'," + + " 'scan.incremental.snapshot.enabled' = '%s'," + " 'changelog-mode' = '%s'" + ")", POSTGERS_CONTAINER.getHost(), @@ -476,6 +506,7 @@ public void testUpsertMode() throws Exception { "inventory", "products", "replica_identity_slot", + parallelismSnapshot, "upsert"); String sinkDDL = "CREATE TABLE sink (" diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java index 38017dc06d..cc097f7e77 100644 --- a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/table/PostgreSQLTableFactoryTest.java @@ -35,11 +35,13 @@ import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext; import org.apache.flink.util.ExceptionUtils; +import com.ververica.cdc.connectors.base.options.StartupOptions; import com.ververica.cdc.debezium.DebeziumSourceFunction; import com.ververica.cdc.debezium.table.DebeziumChangelogMode; import com.ververica.cdc.debezium.utils.ResolvedSchemaUtils; import org.junit.Test; +import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -47,6 +49,15 @@ import java.util.Map; import java.util.Properties; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CHUNK_META_GROUP_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECTION_POOL_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECT_MAX_RETRIES; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.CONNECT_TIMEOUT; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.HEARTBEAT_INTERVAL; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SCAN_SNAPSHOT_FETCH_SIZE; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND; +import static com.ververica.cdc.connectors.postgres.source.config.PostgresSourceOptions.SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND; import static com.ververica.cdc.connectors.utils.AssertUtils.assertProducedTypeOfSourceFunction; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -119,7 +130,19 @@ public void testCommonProperties() { "decoderbufs", MY_SLOT_NAME, DebeziumChangelogMode.ALL, - PROPERTIES); + PROPERTIES, + false, + SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(), + CHUNK_META_GROUP_SIZE.defaultValue(), + SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(), + CONNECT_TIMEOUT.defaultValue(), + CONNECT_MAX_RETRIES.defaultValue(), + CONNECTION_POOL_SIZE.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(), + HEARTBEAT_INTERVAL.defaultValue(), + StartupOptions.initial(), + null); assertEquals(expectedSource, actualSource); } @@ -147,7 +170,19 @@ public void testOptionalProperties() { "wal2json", MY_SLOT_NAME, DebeziumChangelogMode.UPSERT, - dbzProperties); + dbzProperties, + false, + SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(), + CHUNK_META_GROUP_SIZE.defaultValue(), + SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(), + CONNECT_TIMEOUT.defaultValue(), + CONNECT_MAX_RETRIES.defaultValue(), + CONNECTION_POOL_SIZE.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(), + HEARTBEAT_INTERVAL.defaultValue(), + StartupOptions.initial(), + null); assertEquals(expectedSource, actualSource); } @@ -175,7 +210,19 @@ public void testMetadataColumns() { "decoderbufs", MY_SLOT_NAME, DebeziumChangelogMode.ALL, - new Properties()); + new Properties(), + false, + SCAN_INCREMENTAL_SNAPSHOT_CHUNK_SIZE.defaultValue(), + CHUNK_META_GROUP_SIZE.defaultValue(), + SCAN_SNAPSHOT_FETCH_SIZE.defaultValue(), + CONNECT_TIMEOUT.defaultValue(), + CONNECT_MAX_RETRIES.defaultValue(), + CONNECTION_POOL_SIZE.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(), + HEARTBEAT_INTERVAL.defaultValue(), + StartupOptions.initial(), + null); expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType(); expectedSource.metadataKeys = Arrays.asList("op_ts", "database_name", "schema_name", "table_name"); @@ -190,6 +237,85 @@ public void testMetadataColumns() { assertProducedTypeOfSourceFunction(debeziumSourceFunction, expectedSource.producedDataType); } + @Test + public void testEnableParallelReadSource() { + Map properties = getAllOptions(); + properties.put("scan.incremental.snapshot.enabled", "true"); + properties.put("scan.incremental.snapshot.chunk.size", "8000"); + properties.put("scan.snapshot.fetch.size", "100"); + properties.put("connect.timeout", "45s"); + + // validation for source + DynamicTableSource actualSource = createTableSource(SCHEMA, properties); + PostgreSQLTableSource expectedSource = + new PostgreSQLTableSource( + SCHEMA, + 5432, + MY_LOCALHOST, + MY_DATABASE, + MY_SCHEMA, + MY_TABLE, + MY_USERNAME, + MY_PASSWORD, + "decoderbufs", + MY_SLOT_NAME, + DebeziumChangelogMode.ALL, + PROPERTIES, + true, + 8000, + CHUNK_META_GROUP_SIZE.defaultValue(), + 100, + Duration.ofSeconds(45), + CONNECT_MAX_RETRIES.defaultValue(), + CONNECTION_POOL_SIZE.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(), + HEARTBEAT_INTERVAL.defaultValue(), + StartupOptions.initial(), + null); + assertEquals(expectedSource, actualSource); + } + + @Test + public void testStartupFromLatestOffset() { + Map properties = getAllOptions(); + properties.put("scan.incremental.snapshot.enabled", "true"); + properties.put("scan.incremental.snapshot.chunk.size", "8000"); + properties.put("scan.snapshot.fetch.size", "100"); + properties.put("connect.timeout", "45s"); + properties.put("scan.startup.mode", "latest-offset"); + + // validation for source + DynamicTableSource actualSource = createTableSource(properties); + PostgreSQLTableSource expectedSource = + new PostgreSQLTableSource( + SCHEMA, + 5432, + MY_LOCALHOST, + MY_DATABASE, + MY_SCHEMA, + MY_TABLE, + MY_USERNAME, + MY_PASSWORD, + "decoderbufs", + MY_SLOT_NAME, + DebeziumChangelogMode.ALL, + PROPERTIES, + true, + 8000, + CHUNK_META_GROUP_SIZE.defaultValue(), + 100, + Duration.ofSeconds(45), + CONNECT_MAX_RETRIES.defaultValue(), + CONNECTION_POOL_SIZE.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND.defaultValue(), + SPLIT_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue(), + HEARTBEAT_INTERVAL.defaultValue(), + StartupOptions.latest(), + null); + assertEquals(expectedSource, actualSource); + } + @Test public void testValidation() { // validate illegal port @@ -264,6 +390,7 @@ private Map getAllOptions() { options.put("username", MY_USERNAME); options.put("password", MY_PASSWORD); options.put("slot.name", MY_SLOT_NAME); + options.put("scan.incremental.snapshot.enabled", String.valueOf(false)); return options; } diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/TestHelper.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/TestHelper.java new file mode 100644 index 0000000000..7ff8efb709 --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/TestHelper.java @@ -0,0 +1,67 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.testutils; + +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.relational.RelationalDatabaseConnectorConfig; + +/** A helper class for testing. */ +public class TestHelper { + + private static final String TEST_SERVER = "test_server"; + + private static final String TEST_PROPERTY_PREFIX = "debezium.test."; + + private static JdbcConfiguration defaultJdbcConfig() { + return JdbcConfiguration.copy(Configuration.fromSystemProperties("database.")) + .withDefault(JdbcConfiguration.DATABASE, "postgres") + .withDefault(JdbcConfiguration.HOSTNAME, "localhost") + .withDefault(JdbcConfiguration.PORT, 5432) + .withDefault(JdbcConfiguration.USER, "postgres") + .withDefault(JdbcConfiguration.PASSWORD, "postgres") + .with(PostgresConnectorConfig.MAX_RETRIES, 2) + .with(PostgresConnectorConfig.RETRY_DELAY_MS, 2000) + .build(); + } + + /** + * Returns a default configuration for the PostgreSQL connector. Modified from Debezium + * project's postgres TestHelper. + */ + public static Configuration.Builder defaultConfig() { + JdbcConfiguration jdbcConfiguration = defaultJdbcConfig(); + Configuration.Builder builder = Configuration.create(); + jdbcConfiguration.forEach((field, value) -> builder.with("database." + field, value)); + builder.with(RelationalDatabaseConnectorConfig.SERVER_NAME, TEST_SERVER) + .with(PostgresConnectorConfig.DROP_SLOT_ON_STOP, true) + .with(PostgresConnectorConfig.STATUS_UPDATE_INTERVAL_MS, 100) + .with(PostgresConnectorConfig.PLUGIN_NAME, "decoderbufs") + .with( + PostgresConnectorConfig.SSL_MODE, + PostgresConnectorConfig.SecureConnectionMode.DISABLED); + final String testNetworkTimeout = + System.getProperty(TEST_PROPERTY_PREFIX + "network.timeout"); + if (testNetworkTimeout != null && testNetworkTimeout.length() != 0) { + builder.with( + PostgresConnectorConfig.STATUS_UPDATE_INTERVAL_MS, + Integer.parseInt(testNetworkTimeout)); + } + return builder; + } +} diff --git a/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/UniqueDatabase.java b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/UniqueDatabase.java new file mode 100644 index 0000000000..7f4d5e8c46 --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/java/com/ververica/cdc/connectors/postgres/testutils/UniqueDatabase.java @@ -0,0 +1,164 @@ +/* + * Copyright 2022 Ververica Inc. + * + * 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.ververica.cdc.connectors.postgres.testutils; + +import com.ververica.cdc.connectors.postgres.PostgresTestBase; +import org.testcontainers.containers.PostgreSQLContainer; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import static org.junit.Assert.assertNotNull; + +/** + * Create and populate a unique instance of a PostgreSQL database for each run of JUnit test. A user + * of class needs to provide a logical name for Debezium and database name. It is expected that + * there is a init file in src/test/resources/ddl/<database_name>.sql. The + * database name is enriched with a unique suffix that guarantees complete isolation between runs + * + * <database_name>_<suffix> + * + *

    This class is inspired from Debezium project. + */ +public class UniqueDatabase { + private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$"); + + private final PostgreSQLContainer container; + private final String databaseName; + + private final String schemaName; + private final String templateName; + private final String username; + private final String password; + + public UniqueDatabase( + PostgreSQLContainer container, + String databaseName, + String schemaName, + String username, + String password) { + this( + container, + databaseName, + schemaName, + Integer.toUnsignedString(new Random().nextInt(), 36), + username, + password); + } + + private UniqueDatabase( + PostgreSQLContainer container, + String databaseName, + String schemaName, + final String identifier, + String username, + String password) { + this.container = container; + this.databaseName = databaseName + "_" + identifier; + this.schemaName = schemaName; + this.templateName = schemaName; + this.username = username; + this.password = password; + } + + public String getHost() { + return container.getHost(); + } + + public int getDatabasePort() { + return container.getMappedPort(5432); + } + + public String getDatabaseName() { + return databaseName; + } + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } + + private void createDatabase(String databaseName) throws SQLException { + try (Connection connection = + PostgresTestBase.getJdbcConnection(PostgresTestBase.DEFAULT_DB)) { + try (Statement statement = connection.createStatement()) { + statement.execute("CREATE DATABASE " + databaseName); + } + } + } + + /** Creates the database and populates it with initialization SQL script. */ + public void createAndInitialize() { + final String ddlFile = String.format("ddl/%s.sql", templateName); + final URL ddlTestFile = UniqueDatabase.class.getClassLoader().getResource(ddlFile); + assertNotNull("Cannot locate " + ddlFile, ddlTestFile); + + try { + createDatabase(databaseName); + try (Connection connection = PostgresTestBase.getJdbcConnection(databaseName); + Statement statement = connection.createStatement()) { + final List statements = + Arrays.stream( + Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream() + .map(String::trim) + .filter(x -> !x.startsWith("--") && !x.isEmpty()) + .map( + x -> { + final Matcher m = + COMMENT_PATTERN.matcher(x); + return m.matches() ? m.group(1) : x; + }) + .map(this::convertSQL) + .collect(Collectors.joining("\n")) + .split(";")) + .map(x -> x.replace("$$", ";")) + .collect(Collectors.toList()); + for (String stmt : statements) { + statement.execute(stmt); + } + + // run an analyze to collect the statics about tables, used in estimating + // row count in chunk splitter (for auto-vacuum tables, we don't need to do it) + statement.execute("analyze"); + } + } catch (final Exception e) { + throw new IllegalStateException(e); + } + } + + public Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection(container.getJdbcUrl(), username, password); + } + + private String convertSQL(final String sql) { + return sql.replace("$DBNAME$", schemaName); + } +} diff --git a/flink-connector-postgres-cdc/src/test/resources/ddl/customer.sql b/flink-connector-postgres-cdc/src/test/resources/ddl/customer.sql new file mode 100644 index 0000000000..ae6d40de58 --- /dev/null +++ b/flink-connector-postgres-cdc/src/test/resources/ddl/customer.sql @@ -0,0 +1,110 @@ +-- Copyright 2022 Ververica Inc. +-- +-- 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. + +DROP SCHEMA IF EXISTS customer CASCADE; +CREATE SCHEMA customer; +SET search_path TO customer; + +-- Create and populate our users using a single insert with many rows +CREATE TABLE customers ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); +ALTER TABLE customers REPLICA IDENTITY FULL; + +INSERT INTO customers +VALUES (101,'user_1','Shanghai','123567891234'), + (102,'user_2','Shanghai','123567891234'), + (103,'user_3','Shanghai','123567891234'), + (109,'user_4','Shanghai','123567891234'), + (110,'user_5','Shanghai','123567891234'), + (111,'user_6','Shanghai','123567891234'), + (118,'user_7','Shanghai','123567891234'), + (121,'user_8','Shanghai','123567891234'), + (123,'user_9','Shanghai','123567891234'), + (1009,'user_10','Shanghai','123567891234'), + (1010,'user_11','Shanghai','123567891234'), + (1011,'user_12','Shanghai','123567891234'), + (1012,'user_13','Shanghai','123567891234'), + (1013,'user_14','Shanghai','123567891234'), + (1014,'user_15','Shanghai','123567891234'), + (1015,'user_16','Shanghai','123567891234'), + (1016,'user_17','Shanghai','123567891234'), + (1017,'user_18','Shanghai','123567891234'), + (1018,'user_19','Shanghai','123567891234'), + (1019,'user_20','Shanghai','123567891234'), + (2000,'user_21','Shanghai','123567891234'); + +-- table has same name prefix with 'customers.*' +CREATE TABLE customers_1 ( + id INTEGER NOT NULL PRIMARY KEY, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); +ALTER TABLE customers_1 REPLICA IDENTITY FULL; + +INSERT INTO customers_1 +VALUES (101,'user_1','Shanghai','123567891234'), + (102,'user_2','Shanghai','123567891234'), + (103,'user_3','Shanghai','123567891234'), + (109,'user_4','Shanghai','123567891234'), + (110,'user_5','Shanghai','123567891234'), + (111,'user_6','Shanghai','123567891234'), + (118,'user_7','Shanghai','123567891234'), + (121,'user_8','Shanghai','123567891234'), + (123,'user_9','Shanghai','123567891234'), + (1009,'user_10','Shanghai','123567891234'), + (1010,'user_11','Shanghai','123567891234'), + (1011,'user_12','Shanghai','123567891234'), + (1012,'user_13','Shanghai','123567891234'), + (1013,'user_14','Shanghai','123567891234'), + (1014,'user_15','Shanghai','123567891234'), + (1015,'user_16','Shanghai','123567891234'), + (1016,'user_17','Shanghai','123567891234'), + (1017,'user_18','Shanghai','123567891234'), + (1018,'user_19','Shanghai','123567891234'), + (1019,'user_20','Shanghai','123567891234'), + (2000,'user_21','Shanghai','123567891234'); + +CREATE TABLE customers_no_pk ( + id INTEGER NOT NULL, + name VARCHAR(255) NOT NULL DEFAULT 'flink', + address VARCHAR(1024), + phone_number VARCHAR(512) +); + +INSERT INTO customers_no_pk +VALUES (101,'user_1','Shanghai','123567891234'), + (102,'user_2','Shanghai','123567891234'), + (103,'user_3','Shanghai','123567891234'), + (109,'user_4','Shanghai','123567891234'), + (110,'user_5','Shanghai','123567891234'), + (111,'user_6','Shanghai','123567891234'), + (118,'user_7','Shanghai','123567891234'), + (121,'user_8','Shanghai','123567891234'), + (123,'user_9','Shanghai','123567891234'), + (1009,'user_10','Shanghai','123567891234'), + (1010,'user_11','Shanghai','123567891234'), + (1011,'user_12','Shanghai','123567891234'), + (1012,'user_13','Shanghai','123567891234'), + (1013,'user_14','Shanghai','123567891234'), + (1014,'user_15','Shanghai','123567891234'), + (1015,'user_16','Shanghai','123567891234'), + (1016,'user_17','Shanghai','123567891234'), + (1017,'user_18','Shanghai','123567891234'), + (1018,'user_19','Shanghai','123567891234'), + (1019,'user_20','Shanghai','123567891234'), + (2000,'user_21','Shanghai','123567891234'); diff --git a/flink-connector-postgres-cdc/src/test/resources/ddl/inventory.sql b/flink-connector-postgres-cdc/src/test/resources/ddl/inventory.sql index 12384233fd..7a76598f6b 100644 --- a/flink-connector-postgres-cdc/src/test/resources/ddl/inventory.sql +++ b/flink-connector-postgres-cdc/src/test/resources/ddl/inventory.sql @@ -11,7 +11,7 @@ -- specific language governing permissions and limitations -- under the License. --- Create the schema that we'll use to populate data and watch the effect in the binlog +-- Create the schema that we'll use to populate data and watch the effect in the WAL DROP SCHEMA IF EXISTS inventory CASCADE; CREATE SCHEMA inventory; SET search_path TO inventory; diff --git a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java index 8ee666272d..c7e2f59ca7 100644 --- a/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java +++ b/flink-connector-sqlserver-cdc/src/main/java/com/ververica/cdc/connectors/sqlserver/source/config/SqlServerSourceConfig.java @@ -58,6 +58,7 @@ public SqlServerSourceConfig( super( startupOptions, databaseList, + null, tableList, splitSize, splitMetaGroupSize, diff --git a/flink-sql-connector-postgres-cdc/pom.xml b/flink-sql-connector-postgres-cdc/pom.xml index 68cde8c1c8..8732667ad2 100644 --- a/flink-sql-connector-postgres-cdc/pom.xml +++ b/flink-sql-connector-postgres-cdc/pom.xml @@ -54,8 +54,10 @@ under the License. io.debezium:debezium-embedded io.debezium:debezium-core io.debezium:debezium-connector-postgres + com.ververica:flink-cdc-base com.ververica:flink-connector-debezium com.ververica:flink-connector-postgres-cdc + com.zaxxer:HikariCP com.google.protobuf:protobuf-java com.google.guava:* org.apache.kafka:* diff --git a/tools/maven/suppressions.xml b/tools/maven/suppressions.xml index f7b1fc0153..cabe214a0d 100644 --- a/tools/maven/suppressions.xml +++ b/tools/maven/suppressions.xml @@ -73,6 +73,11 @@ under the License. + + +