From 224fb7752d7dee8dfe0dc61313c12512f0464869 Mon Sep 17 00:00:00 2001 From: sunkang Date: Fri, 1 Sep 2023 15:36:55 +0800 Subject: [PATCH 1/3] [Feature][connector-v2][PostgreSQL CDC]Support source PostgreSQL CDC --- docs/en/connector-v2/source/Postgres-CDC.md | 190 +++ plugin-mapping.properties | 1 + .../connector-cdc-postgres/pom.xml | 69 + .../postgresql/PostgresConnectorConfig.java | 1277 +++++++++++++++++ .../PostgresEventMetadataProvider.java | 85 ++ .../postgresql/PostgresOffsetContext.java | 382 +++++ .../connector/postgresql/PostgresSchema.java | 310 ++++ .../postgresql/PostgresTaskContext.java | 141 ++ .../connector/postgresql/TypeRegistry.java | 487 +++++++ .../connection/PostgresConnection.java | 773 ++++++++++ .../postgres/config/PostgresSourceConfig.java | 89 ++ .../config/PostgresSourceConfigFactory.java | 118 ++ .../cdc/postgres/option/PostgresOptions.java | 49 + .../cdc/postgres/source/PostgresDialect.java | 121 ++ .../source/PostgresIncrementalSource.java | 134 ++ .../PostgresIncrementalSourceFactory.java | 121 ++ .../PostgresPooledDataSourceFactory.java | 35 + .../source/PostgresSourceOptions.java | 51 + .../enumerator/PostgresChunkSplitter.java | 93 ++ .../cdc/postgres/source/offset/LsnOffset.java | 124 ++ .../source/offset/LsnOffsetFactory.java | 79 + .../PostgresSourceFetchTaskContext.java | 364 +++++ .../snapshot/PostgresSnapshotFetchTask.java | 137 ++ .../PostgresSnapshotSplitReadTask.java | 269 ++++ ...SnapshotSplitChangeEventSourceContext.java | 55 + .../reader/wal/PostgresWalFetchTask.java | 84 ++ .../utils/PostgresConnectionUtils.java | 44 + .../cdc/postgres/utils/PostgresSchema.java | 86 ++ .../cdc/postgres/utils/PostgresTypeUtils.java | 93 ++ .../cdc/postgres/utils/PostgresUtils.java | 419 ++++++ .../postgres/utils/TableDiscoveryUtils.java | 91 ++ .../PostgresIncrementalSourceFactoryTest.java | 30 + seatunnel-connectors-v2/connector-cdc/pom.xml | 1 + .../connector-cdc-postgres-e2e/pom.xml | 60 + .../connector/cdc/postgres/PostgresCDCIT.java | 315 ++++ .../src/test/resources/ddl/postgres_cdc.sql | 199 +++ .../src/test/resources/docker/postgres.cnf | 704 +++++++++ .../resources/postgres_cdc_to_postgres.conf | 55 + .../seatunnel-connector-v2-e2e/pom.xml | 1 + 39 files changed, 7736 insertions(+) create mode 100644 docs/en/connector-v2/source/Postgres-CDC.md create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf create mode 100644 seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf diff --git a/docs/en/connector-v2/source/Postgres-CDC.md b/docs/en/connector-v2/source/Postgres-CDC.md new file mode 100644 index 00000000000..fd0ddad176d --- /dev/null +++ b/docs/en/connector-v2/source/Postgres-CDC.md @@ -0,0 +1,190 @@ +# Postgres CDC + +> Postgres CDC source connector + +## Description + +The Postgres CDC connector allows for reading snapshot data and incremental data from Postgres database. This document +describes how to set up the Postgres CDC connector to run SQL queries against Postgres databases. + +## Key features + +- [ ] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +## Options + +| name | type | required | default value | +| ---------------------------------------------- | -------- | -------- | ------------- | +| base-url | String | Yes | - | +| username | String | Yes | - | +| password | String | Yes | - | +| database-names | List | Yes | - | +| table-names | List | Yes | - | +| startup.mode | Enum | No | INITIAL | +| startup.timestamp | Long | No | - | +| startup.specific-offset.file | String | No | - | +| startup.specific-offset.pos | Long | No | - | +| stop.mode | Enum | No | NEVER | +| stop.timestamp | Long | No | - | +| stop.specific-offset.file | String | No | - | +| stop.specific-offset.pos | Long | No | - | +| incremental.parallelism | Integer | No | 1 | +| snapshot.split.size | Integer | No | 8096 | +| snapshot.fetch.size | Integer | No | 1024 | +| server-time-zone | String | No | UTC | +| connect.timeout.ms | Duration | No | 30000 | +| connect.max-retries | Integer | No | 3 | +| connection.pool.size | Integer | No | 20 | +| chunk-key.even-distribution.factor.upper-bound | Double | No | 1000 | +| chunk-key.even-distribution.factor.lower-bound | Double | No | 0.05 | +| debezium.* | config | No | - | +| format | Enum | No | DEFAULT | +| common-options | | no | - | + +### username [String] + +Name of the database to use when connecting to the database server. + +### password [String] + +Password to use when connecting to the database server. + +### database-name [String] + +Database name of the database to monitor. + +### schmea-list [List] + +Schmea name of the database to monitor. + +### table-names [List] + +Table name of the database to monitor. The table name needs to include the schema name, for example: schema_name.table_name + +### startup.mode [Enum] + +Optional startup mode for postgres CDC consumer, valid enumerations are "initial", "earliest", "latest" and "specific". + +### startup.timestamp [Long] + +Start from the specified epoch timestamp (in milliseconds). + +**Note, This option is required when the "startup.mode" option used `'timestamp'`.** + +### startup.specific-offset.file [String] + +Start from the specified binlog file name. + +**Note, This option is required when the "startup.mode" option used `'specific'`.** + +### startup.specific-offset.pos [Long] + +Start from the specified binlog file position. + +**Note, This option is required when the "startup.mode" option used `'specific'`.** + +### stop.mode [Enum] + +Optional stop mode for postgres CDC consumer, valid enumerations are "never". + +### stop.timestamp [Long] + +Stop from the specified epoch timestamp (in milliseconds). + +**Note, This option is required when the "stop.mode" option used `'timestamp'`.** + +### stop.specific-offset.file [String] + +Stop from the specified binlog file name. + +**Note, This option is required when the "stop.mode" option used `'specific'`.** + +### stop.specific-offset.pos [Long] + +Stop from the specified binlog file position. + +**Note, This option is required when the "stop.mode" option used `'specific'`.** + +### incremental.parallelism [Integer] + +The number of parallel readers in the incremental phase. + +### snapshot.split.size [Integer] + +The split size (number of rows) of table snapshot, captured tables are split into multiple splits when read the snapshot +of table. + +### snapshot.fetch.size [Integer] + +The maximum fetch size for per poll when read table snapshot. + +### server-time-zone [String] + +The session time zone in database server. + +### connect.timeout.ms [long] + +The maximum time that the connector should wait after trying to connect to the database server before timing out. + +### connect.max-retries [Integer] + +The max retry times that the connector should retry to build database server connection. + +### connection.pool.size [Integer] + +The connection pool size. + +### debezium [Config] + +Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from postgres server. + +See more about +the [Debezium's postgres Connector properties](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#postgresql-connector-properties) + +### format [Enum] + +Optional output format for Postgres CDC, valid enumerations are "DEFAULT"、"COMPATIBLE_DEBEZIUM_JSON". + +#### example + +```conf +source { + Postgres-CDC { + debezium { + snapshot.mode = "never" + decimal.handling.mode = "double" + } + } +} +``` + +### common options + +Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. + +## Example + +```Jdbc { +source { + Postgres-CDC { + base-url = "jdbc:postgresql://127.0.0.1:5432/postgres" + username = "postgres" + password = "postgres" + database-names = ["postgres"] + table-names = ["postgres.public.cdc_source_table"] + } + +} +``` + +## Changelog + +- Add Postgres CDC Source Connector + +### next version + diff --git a/plugin-mapping.properties b/plugin-mapping.properties index 551da2c7cec..80ef10d7832 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -113,3 +113,4 @@ seatunnel.source.Rocketmq = connector-rocketmq seatunnel.sink.Rocketmq = connector-rocketmq seatunnel.source.Paimon = connector-paimon seatunnel.sink.Paimon = connector-paimon +seatunnel.source.Postgres-CDC = connector-cdc-postgres diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml new file mode 100644 index 00000000000..540e29d3d61 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml @@ -0,0 +1,69 @@ + + + + 4.0.0 + + org.apache.seatunnel + connector-cdc + ${revision} + + connector-cdc-postgres + SeaTunnel : Connectors V2 : CDC : Postgres + + + + + org.apache.seatunnel + connector-cdc-base + ${project.version} + compile + + + + io.debezium + debezium-connector-postgres + ${debezium.version} + compile + + + + + + + + + org.apache.seatunnel + connector-cdc-base + + + + io.debezium + debezium-connector-postgres + + + + org.apache.seatunnel + connector-jdbc + ${project.version} + + + + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java new file mode 100644 index 00000000000..92462d0ba29 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java @@ -0,0 +1,1277 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Width; +import org.apache.kafka.common.config.ConfigValue; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.config.ConfigDefinition; +import io.debezium.config.Configuration; +import io.debezium.config.EnumeratedValue; +import io.debezium.config.Field; +import io.debezium.connector.AbstractSourceInfo; +import io.debezium.connector.SourceInfoStructMaker; +import io.debezium.connector.postgresql.connection.MessageDecoder; +import io.debezium.connector.postgresql.connection.MessageDecoderContext; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder; +import io.debezium.connector.postgresql.connection.pgproto.PgProtoMessageDecoder; +import io.debezium.connector.postgresql.connection.wal2json.NonStreamingWal2JsonMessageDecoder; +import io.debezium.connector.postgresql.connection.wal2json.StreamingWal2JsonMessageDecoder; +import io.debezium.connector.postgresql.snapshot.AlwaysSnapshotter; +import io.debezium.connector.postgresql.snapshot.InitialOnlySnapshotter; +import io.debezium.connector.postgresql.snapshot.InitialSnapshotter; +import io.debezium.connector.postgresql.snapshot.NeverSnapshotter; +import io.debezium.connector.postgresql.spi.Snapshotter; +import io.debezium.heartbeat.DatabaseHeartbeatImpl; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.relational.ColumnFilterMode; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables.TableFilter; +import io.debezium.util.Strings; + +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +/** The configuration properties for the {@link PostgresConnector} */ +@SuppressWarnings({"InnerTypeLast", "MagicNumber", "MethodName", "RegexpSinglelineJava"}) +public class PostgresConnectorConfig extends RelationalDatabaseConnectorConfig { + + private static final Logger LOGGER = LoggerFactory.getLogger(PostgresConnectorConfig.class); + + /** The set of predefined HStoreHandlingMode options or aliases */ + public enum HStoreHandlingMode implements EnumeratedValue { + + /** Represents HStore value as json */ + JSON("json"), + + /** Represents HStore value as map */ + MAP("map"); + + private final String value; + + HStoreHandlingMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied values is one of the predefined options + * + * @param value the configuration property value ; may not be null + * @return the matching option, or null if the match is not found + */ + public static HStoreHandlingMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (HStoreHandlingMode option : HStoreHandlingMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied values is one of the predefined options + * + * @param value the configuration property value ; may not be null + * @param defaultValue the default value ; may be null + * @return the matching option or null if the match is not found and non-null default is + * invalid + */ + public static HStoreHandlingMode parse(String value, String defaultValue) { + HStoreHandlingMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + /** Defines modes of representation of {@code interval} datatype */ + public enum IntervalHandlingMode implements EnumeratedValue { + + /** Represents interval as inexact microseconds count */ + NUMERIC("numeric"), + + /** Represents interval as ISO 8601 time interval */ + STRING("string"); + + private final String value; + + IntervalHandlingMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Convert mode name into the logical value + * + * @param value the configuration property value ; may not be null + * @return the matching option, or null if the match is not found + */ + public static IntervalHandlingMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (IntervalHandlingMode option : IntervalHandlingMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Convert mode name into the logical value + * + * @param value the configuration property value ; may not be null + * @param defaultValue the default value ; may be null + * @return the matching option or null if the match is not found and non-null default is + * invalid + */ + public static IntervalHandlingMode parse(String value, String defaultValue) { + IntervalHandlingMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + /** The set of predefined Snapshotter options or aliases. */ + public enum SnapshotMode implements EnumeratedValue { + + /** Always perform a snapshot when starting. */ + ALWAYS("always", c -> new AlwaysSnapshotter()), + + /** Perform a snapshot only upon initial startup of a connector. */ + INITIAL("initial", c -> new InitialSnapshotter()), + + /** Never perform a snapshot and only receive logical changes. */ + NEVER("never", c -> new NeverSnapshotter()), + + /** Perform a snapshot and then stop before attempting to receive any logical changes. */ + INITIAL_ONLY("initial_only", c -> new InitialOnlySnapshotter()), + + /** Perform an exported snapshot */ + @Deprecated + EXPORTED("exported", c -> new InitialSnapshotter()), + + /** Inject a custom snapshotter, which allows for more control over snapshots. */ + CUSTOM( + "custom", + c -> { + return c.getInstance(SNAPSHOT_MODE_CLASS, Snapshotter.class); + }); + + @FunctionalInterface + public interface SnapshotterBuilder { + Snapshotter buildSnapshotter(Configuration config); + } + + private final String value; + private final SnapshotterBuilder builderFunc; + + SnapshotMode(String value, SnapshotterBuilder buildSnapshotter) { + this.value = value; + this.builderFunc = buildSnapshotter; + } + + public Snapshotter getSnapshotter(Configuration config) { + return builderFunc.buildSnapshotter(config); + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static SnapshotMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (SnapshotMode option : SnapshotMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is + * invalid + */ + public static SnapshotMode parse(String value, String defaultValue) { + SnapshotMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + /** The set of predefined SecureConnectionMode options or aliases. */ + public enum SecureConnectionMode implements EnumeratedValue { + + /** + * Establish an unencrypted connection + * + *

see the {@code sslmode} Postgres JDBC driver option + */ + DISABLED("disable"), + + /** + * Establish a secure connection if the server supports secure connections. The connection + * attempt fails if a secure connection cannot be established + * + *

see the {@code sslmode} Postgres JDBC driver option + */ + REQUIRED("require"), + + /** + * Like REQUIRED, but additionally verify the server TLS certificate against the configured + * Certificate Authority (CA) certificates. The connection attempt fails if no valid + * matching CA certificates are found. + * + *

see the {@code sslmode} Postgres JDBC driver option + */ + VERIFY_CA("verify-ca"), + + /** + * Like VERIFY_CA, but additionally verify that the server certificate matches the host to + * which the connection is attempted. + * + *

see the {@code sslmode} Postgres JDBC driver option + */ + VERIFY_FULL("verify-full"); + + private final String value; + + SecureConnectionMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static SecureConnectionMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (SecureConnectionMode option : SecureConnectionMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is + * invalid + */ + public static SecureConnectionMode parse(String value, String defaultValue) { + SecureConnectionMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + public enum LogicalDecoder implements EnumeratedValue { + PGOUTPUT("pgoutput") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new PgOutputMessageDecoder(config); + } + + @Override + public String getPostgresPluginName() { + return getValue(); + } + + @Override + public boolean supportsTruncate() { + return true; + } + }, + DECODERBUFS("decoderbufs") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new PgProtoMessageDecoder(); + } + + @Override + public String getPostgresPluginName() { + return getValue(); + } + + @Override + public boolean supportsTruncate() { + return false; + } + }, + WAL2JSON_STREAMING("wal2json_streaming") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new StreamingWal2JsonMessageDecoder(); + } + + @Override + public String getPostgresPluginName() { + return "wal2json"; + } + + @Override + public boolean supportsTruncate() { + return false; + } + + @Override + public boolean hasUnchangedToastColumnMarker() { + return false; + } + + @Override + public boolean sendsNullToastedValuesInOld() { + return false; + } + }, + WAL2JSON_RDS_STREAMING("wal2json_rds_streaming") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new StreamingWal2JsonMessageDecoder(); + } + + @Override + public boolean forceRds() { + return true; + } + + @Override + public String getPostgresPluginName() { + return "wal2json"; + } + + @Override + public boolean supportsTruncate() { + return false; + } + + @Override + public boolean hasUnchangedToastColumnMarker() { + return false; + } + + @Override + public boolean sendsNullToastedValuesInOld() { + return false; + } + }, + WAL2JSON("wal2json") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new NonStreamingWal2JsonMessageDecoder(); + } + + @Override + public String getPostgresPluginName() { + return "wal2json"; + } + + @Override + public boolean supportsTruncate() { + return false; + } + + @Override + public boolean hasUnchangedToastColumnMarker() { + return false; + } + + @Override + public boolean sendsNullToastedValuesInOld() { + return false; + } + }, + WAL2JSON_RDS("wal2json_rds") { + @Override + public MessageDecoder messageDecoder(MessageDecoderContext config) { + return new NonStreamingWal2JsonMessageDecoder(); + } + + @Override + public boolean forceRds() { + return true; + } + + @Override + public String getPostgresPluginName() { + return "wal2json"; + } + + @Override + public boolean supportsTruncate() { + return false; + } + + @Override + public boolean hasUnchangedToastColumnMarker() { + return false; + } + + @Override + public boolean sendsNullToastedValuesInOld() { + return false; + } + }; + + private final String decoderName; + + LogicalDecoder(String decoderName) { + this.decoderName = decoderName; + } + + public abstract MessageDecoder messageDecoder(MessageDecoderContext config); + + public boolean forceRds() { + return false; + } + + public boolean hasUnchangedToastColumnMarker() { + return true; + } + + public boolean sendsNullToastedValuesInOld() { + return true; + } + + public static LogicalDecoder parse(String s) { + return valueOf(s.trim().toUpperCase()); + } + + @Override + public String getValue() { + return decoderName; + } + + public abstract String getPostgresPluginName(); + + public abstract boolean supportsTruncate(); + } + + /** The set of predefined TruncateHandlingMode options or aliases */ + public enum TruncateHandlingMode implements EnumeratedValue { + + /** Skip TRUNCATE messages */ + SKIP("skip"), + + /** Handle & Include TRUNCATE messages */ + INCLUDE("include"); + + private final String value; + + TruncateHandlingMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + public static TruncateHandlingMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (TruncateHandlingMode truncateHandlingMode : TruncateHandlingMode.values()) { + if (truncateHandlingMode.getValue().equalsIgnoreCase(value)) { + return truncateHandlingMode; + } + } + return null; + } + } + + /** The set of predefined SchemaRefreshMode options or aliases. */ + public enum SchemaRefreshMode implements EnumeratedValue { + /** + * Refresh the in-memory schema cache whenever there is a discrepancy between it and the + * schema derived from the incoming message. + */ + COLUMNS_DIFF("columns_diff"), + + /** + * Refresh the in-memory schema cache if there is a discrepancy between it and the schema + * derived from the incoming message, unless TOASTable data can account for the discrepancy. + * + *

This setting can improve connector performance significantly if there are + * frequently-updated tables that have TOASTed data that are rarely part of these updates. + * However, it is possible for the in-memory schema to become outdated if TOASTable columns + * are dropped from the table. + */ + COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST("columns_diff_exclude_unchanged_toast"); + + private final String value; + + SchemaRefreshMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static SchemaRefreshMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (SchemaRefreshMode option : SchemaRefreshMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + } + + protected static final String DATABASE_CONFIG_PREFIX = "database."; + protected static final int DEFAULT_PORT = 5_432; + protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = 10_240; + protected static final int DEFAULT_MAX_RETRIES = 6; + + public static final Field PORT = + RelationalDatabaseConnectorConfig.PORT.withDefault(DEFAULT_PORT); + + public static final Field PLUGIN_NAME = + Field.create("plugin.name") + .withDisplayName("Plugin") + .withEnum(LogicalDecoder.class, LogicalDecoder.DECODERBUFS) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDescription( + "The name of the Postgres logical decoding plugin installed on the server. " + + "Supported values are '" + + LogicalDecoder.DECODERBUFS.getValue() + + "', '" + + LogicalDecoder.WAL2JSON.getValue() + + "', '" + + LogicalDecoder.PGOUTPUT.getValue() + + "', '" + + LogicalDecoder.WAL2JSON_STREAMING.getValue() + + "', '" + + LogicalDecoder.WAL2JSON_RDS.getValue() + + "' and '" + + LogicalDecoder.WAL2JSON_RDS_STREAMING.getValue() + + "'. " + + "Defaults to '" + + LogicalDecoder.DECODERBUFS.getValue() + + "'."); + + public static final Field SLOT_NAME = + Field.create("slot.name") + .withDisplayName("Slot") + .withType(Type.STRING) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDefault(ReplicationConnection.Builder.DEFAULT_SLOT_NAME) + .withValidation(PostgresConnectorConfig::validateReplicationSlotName) + .withDescription( + "The name of the Postgres logical decoding slot created for streaming changes from a plugin." + + "Defaults to 'debezium"); + + public static final Field DROP_SLOT_ON_STOP = + Field.create("slot.drop.on.stop") + .withDisplayName("Drop slot on stop") + .withType(Type.BOOLEAN) + .withDefault(false) + .withImportance(Importance.MEDIUM) + .withDescription( + "Whether or not to drop the logical replication slot when the connector finishes orderly" + + "By default the replication is kept so that on restart progress can resume from the last recorded location"); + + public static final Field PUBLICATION_NAME = + Field.create("publication.name") + .withDisplayName("Publication") + .withType(Type.STRING) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDefault(ReplicationConnection.Builder.DEFAULT_PUBLICATION_NAME) + .withDescription( + "The name of the Postgres 10+ publication used for streaming changes from a plugin." + + "Defaults to '" + + ReplicationConnection.Builder.DEFAULT_PUBLICATION_NAME + + "'"); + + public enum AutoCreateMode implements EnumeratedValue { + /** + * No Publication will be created, it's expected the user has already created the + * publication. + */ + DISABLED("disabled"), + /** Enable publication for all tables. */ + ALL_TABLES("all_tables"), + /** Enable publication on a specific set of tables. */ + FILTERED("filtered"); + + private final String value; + + AutoCreateMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static AutoCreateMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (AutoCreateMode option : AutoCreateMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is + * invalid + */ + public static AutoCreateMode parse(String value, String defaultValue) { + AutoCreateMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + public static final Field PUBLICATION_AUTOCREATE_MODE = + Field.create("publication.autocreate.mode") + .withDisplayName("Publication Auto Create Mode") + .withEnum(AutoCreateMode.class, AutoCreateMode.ALL_TABLES) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDescription( + "Applies only when streaming changes using pgoutput." + + "Determine how creation of a publication should work, the default is all_tables." + + "DISABLED - The connector will not attempt to create a publication at all. The expectation is " + + "that the user has created the publication up-front. If the publication isn't found to exist upon " + + "startup, the connector will throw an exception and stop." + + "ALL_TABLES - If no publication exists, the connector will create a new publication for all tables. " + + "Note this requires that the configured user has access. If the publication already exists, it will be used" + + ". i.e CREATE PUBLICATION FOR ALL TABLES;" + + "FILTERED - If no publication exists, the connector will create a new publication for all those tables matching" + + "the current filter configuration (see table/database include/exclude list properties). If the publication already" + + " exists, it will be used. i.e CREATE PUBLICATION FOR TABLE "); + + public static final Field STREAM_PARAMS = + Field.create("slot.stream.params") + .withDisplayName( + "Optional parameters to pass to the logical decoder when the stream is started.") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.LOW) + .withDescription( + "Any optional parameters used by logical decoding plugin. Semi-colon separated. E.g. 'add-tables=public.table,public.table2;include-lsn=true'"); + + public static final Field MAX_RETRIES = + Field.create("slot.max.retries") + .withDisplayName("Retry count") + .withType(Type.INT) + .withImportance(Importance.LOW) + .withDefault(DEFAULT_MAX_RETRIES) + .withValidation(Field::isInteger) + .withDescription( + "How many times to retry connecting to a replication slot when an attempt fails."); + + public static final Field RETRY_DELAY_MS = + Field.create("slot.retry.delay.ms") + .withDisplayName("Retry delay") + .withType(Type.LONG) + .withImportance(Importance.LOW) + .withDefault(Duration.ofSeconds(10).toMillis()) + .withValidation(Field::isInteger) + .withDescription( + "Time to wait between retry attempts when the connector fails to connect to a replication slot, given in milliseconds. Defaults to 10 seconds (10,000 ms)."); + + public static final Field ON_CONNECT_STATEMENTS = + Field.create(DATABASE_CONFIG_PREFIX + JdbcConfiguration.ON_CONNECT_STATEMENTS) + .withDisplayName("Initial statements") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.LOW) + .withDescription( + "A semicolon separated list of SQL statements to be executed when a JDBC connection to the database is established. " + + "Note that the connector may establish JDBC connections at its own discretion, so this should typically be used for configuration" + + "of session parameters only, but not for executing DML statements. Use doubled semicolon (';;') to use a semicolon as a character " + + "and not as a delimiter."); + + public static final Field SSL_MODE = + Field.create(DATABASE_CONFIG_PREFIX + "sslmode") + .withDisplayName("SSL mode") + .withEnum(SecureConnectionMode.class, SecureConnectionMode.DISABLED) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDescription( + "Whether to use an encrypted connection to Postgres. Options include" + + "'disable' (the default) to use an unencrypted connection; " + + "'require' to use a secure (encrypted) connection, and fail if one cannot be established; " + + "'verify-ca' like 'required' but additionally verify the server TLS certificate against the configured Certificate Authority " + + "(CA) certificates, or fail if no valid matching CA certificates are found; or" + + "'verify-full' like 'verify-ca' but additionally verify that the server certificate matches the host to which the connection is attempted."); + + public static final Field SSL_CLIENT_CERT = + Field.create(DATABASE_CONFIG_PREFIX + "sslcert") + .withDisplayName("SSL Client Certificate") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.MEDIUM) + .withDescription( + "File containing the SSL Certificate for the client. See the Postgres SSL docs for further information"); + + public static final Field SSL_CLIENT_KEY = + Field.create(DATABASE_CONFIG_PREFIX + "sslkey") + .withDisplayName("SSL Client Key") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.MEDIUM) + .withDescription( + "File containing the SSL private key for the client. See the Postgres SSL docs for further information"); + + public static final Field SSL_CLIENT_KEY_PASSWORD = + Field.create(DATABASE_CONFIG_PREFIX + "sslpassword") + .withDisplayName("SSL Client Key Password") + .withType(Type.PASSWORD) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withDescription( + "Password to access the client private key from the file specified by 'database.sslkey'. See the Postgres SSL docs for further information"); + + public static final Field SSL_ROOT_CERT = + Field.create(DATABASE_CONFIG_PREFIX + "sslrootcert") + .withDisplayName("SSL Root Certificate") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.MEDIUM) + .withDescription( + "File containing the root certificate(s) against which the server is validated. See the Postgres JDBC SSL docs for further information"); + + public static final Field SSL_SOCKET_FACTORY = + Field.create(DATABASE_CONFIG_PREFIX + "sslfactory") + .withDisplayName("SSL Root Certificate") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withImportance(Importance.MEDIUM) + .withDescription( + "A name of class to that creates SSL Sockets. Use org.postgresql.ssl.NonValidatingFactory to disable SSL validation in development environments"); + + public static final Field SNAPSHOT_MODE = + Field.create("snapshot.mode") + .withDisplayName("Snapshot mode") + .withEnum(SnapshotMode.class, SnapshotMode.INITIAL) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withValidation( + (config, field, output) -> { + if (config.getString(field) + .toLowerCase() + .equals(SnapshotMode.EXPORTED.getValue())) { + LOGGER.warn( + "Value '{}' of 'snapshot.mode' option is deprecated, use '{}' instead", + SnapshotMode.EXPORTED.getValue(), + SnapshotMode.INITIAL.getValue()); + } + return 0; + }) + .withDescription( + "The criteria for running a snapshot upon startup of the connector. " + + "Options include: " + + "'always' to specify that the connector run a snapshot each time it starts up; " + + "'initial' (the default) to specify the connector can run a snapshot only when no offsets are available for the logical server name; " + + "'initial_only' same as 'initial' except the connector should stop after completing the snapshot and before it would normally start emitting changes;" + + "'never' to specify the connector should never run a snapshot and that upon first startup the connector should read from the last position (LSN) recorded by the server; and" + + "'exported' deprecated, use 'initial' instead; " + + "'custom' to specify a custom class with 'snapshot.custom_class' which will be loaded and used to determine the snapshot, see docs for more details."); + + public static final Field SNAPSHOT_MODE_CLASS = + Field.create("snapshot.custom.class") + .withDisplayName("Snapshot Mode Custom Class") + .withType(Type.STRING) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withValidation( + (config, field, output) -> { + if (config.getString(SNAPSHOT_MODE).toLowerCase().equals("custom") + && config.getString(field).isEmpty()) { + output.accept( + field, + "", + "snapshot.custom_class cannot be empty when snapshot.mode 'custom' is defined"); + return 1; + } + return 0; + }) + .withDescription( + "When 'snapshot.mode' is set as custom, this setting must be set to specify a fully qualified class name to load (via the default class loader)." + + "This class must implement the 'Snapshotter' interface and is called on each app boot to determine whether to do a snapshot and how to build queries."); + + public static final Field TRUNCATE_HANDLING_MODE = + Field.create("truncate.handling.mode") + .withDisplayName("Truncate handling mode") + .withEnum(TruncateHandlingMode.class, TruncateHandlingMode.SKIP) + .withWidth(Width.MEDIUM) + .withImportance(Importance.MEDIUM) + .withValidation(PostgresConnectorConfig::validateTruncateHandlingMode) + .withDescription( + "Specify how TRUNCATE operations are handled for change events (supported only on pg11+ pgoutput plugin), including: " + + "'skip' to skip / ignore TRUNCATE events (default), " + + "'include' to handle and include TRUNCATE events"); + + public static final Field HSTORE_HANDLING_MODE = + Field.create("hstore.handling.mode") + .withDisplayName("HStore Handling") + .withEnum(HStoreHandlingMode.class, HStoreHandlingMode.JSON) + .withWidth(Width.MEDIUM) + .withImportance(Importance.LOW) + .withDescription( + "Specify how HSTORE columns should be represented in change events, including:" + + "'json' represents values as string-ified JSON (default)" + + "'map' represents values as a key/value map"); + + public static final Field INTERVAL_HANDLING_MODE = + Field.create("interval.handling.mode") + .withDisplayName("Interval Handling") + .withEnum(IntervalHandlingMode.class, IntervalHandlingMode.NUMERIC) + .withWidth(Width.MEDIUM) + .withImportance(Importance.LOW) + .withDescription( + "Specify how INTERVAL columns should be represented in change events, including:" + + "'string' represents values as an exact ISO formatted string" + + "'numeric' (default) represents values using the inexact conversion into microseconds"); + + public static final Field STATUS_UPDATE_INTERVAL_MS = + Field.create("status.update.interval.ms") + .withDisplayName("Status update interval (ms)") + .withType(Type.INT) // Postgres doesn't accept long for this value + .withDefault(10_000) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withDescription( + "Frequency for sending replication connection status updates to the server, given in milliseconds. Defaults to 10 seconds (10,000 ms).") + .withValidation(Field::isPositiveInteger); + + public static final Field TCP_KEEPALIVE = + Field.create(DATABASE_CONFIG_PREFIX + "tcpKeepAlive") + .withDisplayName("TCP keep-alive probe") + .withType(Type.BOOLEAN) + .withDefault(true) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withDescription( + "Enable or disable TCP keep-alive probe to avoid dropping TCP connection") + .withValidation(Field::isBoolean); + + public static final Field INCLUDE_UNKNOWN_DATATYPES = + Field.create("include.unknown.datatypes") + .withDisplayName("Include unknown datatypes") + .withType(Type.BOOLEAN) + .withDefault(false) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withDescription( + "Specify whether the fields of data type not supported by Debezium should be processed:" + + "'false' (the default) omits the fields; " + + "'true' converts the field into an implementation dependent binary representation."); + + public static final Field SCHEMA_REFRESH_MODE = + Field.create("schema.refresh.mode") + .withDisplayName("Schema refresh mode") + .withEnum(SchemaRefreshMode.class, SchemaRefreshMode.COLUMNS_DIFF) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withDescription( + "Specify the conditions that trigger a refresh of the in-memory schema for a table. " + + "'columns_diff' (the default) is the safest mode, ensuring the in-memory schema stays in-sync with " + + "the database table's schema at all times. " + + "'columns_diff_exclude_unchanged_toast' instructs the connector to refresh the in-memory schema cache if there is a discrepancy between it " + + "and the schema derived from the incoming message, unless unchanged TOASTable data fully accounts for the discrepancy. " + + "This setting can improve connector performance significantly if there are frequently-updated tables that " + + "have TOASTed data that are rarely part of these updates. However, it is possible for the in-memory schema to " + + "become outdated if TOASTable columns are dropped from the table."); + + public static final Field XMIN_FETCH_INTERVAL = + Field.create("xmin.fetch.interval.ms") + .withDisplayName("Xmin fetch interval (ms)") + .withType(Type.LONG) + .withWidth(Width.SHORT) + .withDefault(0L) + .withImportance(Importance.MEDIUM) + .withDescription( + "Specify how often (in ms) the xmin will be fetched from the replication slot. " + + "This xmin value is exposed by the slot which gives a lower bound of where a new replication slot could start from. " + + "The lower the value, the more likely this value is to be the current 'true' value, but the bigger the performance cost. " + + "The bigger the value, the less likely this value is to be the current 'true' value, but the lower the performance penalty. " + + "The default is set to 0 ms, which disables tracking xmin.") + .withValidation(Field::isNonNegativeLong); + + public static final Field TOASTED_VALUE_PLACEHOLDER = + Field.create("toasted.value.placeholder") + .withDisplayName("Toasted value placeholder") + .withType(Type.STRING) + .withWidth(Width.MEDIUM) + .withDefault("__debezium_unavailable_value") + .withImportance(Importance.MEDIUM) + .withDescription( + "Specify the constant that will be provided by Debezium to indicate that " + + "the original value is a toasted value not provided by the database. " + + "If starts with 'hex:' prefix it is expected that the rest of the string repesents hexadecimally encoded octets."); + + private final TruncateHandlingMode truncateHandlingMode; + private final HStoreHandlingMode hStoreHandlingMode; + private final IntervalHandlingMode intervalHandlingMode; + private final SnapshotMode snapshotMode; + private final SchemaRefreshMode schemaRefreshMode; + + public PostgresConnectorConfig(Configuration config) { + super( + config, + config.getString(RelationalDatabaseConnectorConfig.SERVER_NAME), + new SystemTablesPredicate(), + x -> x.schema() + "." + x.table(), + DEFAULT_SNAPSHOT_FETCH_SIZE, + ColumnFilterMode.SCHEMA); + + this.truncateHandlingMode = + TruncateHandlingMode.parse( + config.getString(PostgresConnectorConfig.TRUNCATE_HANDLING_MODE)); + String hstoreHandlingModeStr = + config.getString(PostgresConnectorConfig.HSTORE_HANDLING_MODE); + this.hStoreHandlingMode = HStoreHandlingMode.parse(hstoreHandlingModeStr); + this.intervalHandlingMode = + IntervalHandlingMode.parse( + config.getString(PostgresConnectorConfig.INTERVAL_HANDLING_MODE)); + this.snapshotMode = SnapshotMode.parse(config.getString(SNAPSHOT_MODE)); + this.schemaRefreshMode = SchemaRefreshMode.parse(config.getString(SCHEMA_REFRESH_MODE)); + } + + protected String hostname() { + return getConfig().getString(HOSTNAME); + } + + protected int port() { + return getConfig().getInteger(PORT); + } + + public String databaseName() { + return getConfig().getString(DATABASE_NAME); + } + + public LogicalDecoder plugin() { + return LogicalDecoder.parse(getConfig().getString(PLUGIN_NAME)); + } + + public String slotName() { + return getConfig().getString(SLOT_NAME); + } + + protected boolean dropSlotOnStop() { + if (getConfig().hasKey(DROP_SLOT_ON_STOP.name())) { + return getConfig().getBoolean(DROP_SLOT_ON_STOP); + } + // Return default value + return getConfig().getBoolean(DROP_SLOT_ON_STOP); + } + + public String publicationName() { + return getConfig().getString(PUBLICATION_NAME); + } + + protected AutoCreateMode publicationAutocreateMode() { + return AutoCreateMode.parse(getConfig().getString(PUBLICATION_AUTOCREATE_MODE)); + } + + protected String streamParams() { + return getConfig().getString(STREAM_PARAMS); + } + + public int maxRetries() { + return getConfig().getInteger(MAX_RETRIES); + } + + public Duration retryDelay() { + return Duration.ofMillis(getConfig().getInteger(RETRY_DELAY_MS)); + } + + protected Duration statusUpdateInterval() { + return Duration.ofMillis( + getConfig().getLong(PostgresConnectorConfig.STATUS_UPDATE_INTERVAL_MS)); + } + + public TruncateHandlingMode truncateHandlingMode() { + return truncateHandlingMode; + } + + protected HStoreHandlingMode hStoreHandlingMode() { + return hStoreHandlingMode; + } + + protected IntervalHandlingMode intervalHandlingMode() { + return intervalHandlingMode; + } + + protected boolean includeUnknownDatatypes() { + return getConfig().getBoolean(INCLUDE_UNKNOWN_DATATYPES); + } + + public Map validate() { + return getConfig().validate(ALL_FIELDS); + } + + public Snapshotter getSnapshotter() { + return this.snapshotMode.getSnapshotter(getConfig()); + } + + protected boolean skipRefreshSchemaOnMissingToastableData() { + return SchemaRefreshMode.COLUMNS_DIFF_EXCLUDE_UNCHANGED_TOAST == this.schemaRefreshMode; + } + + protected Duration xminFetchInterval() { + return Duration.ofMillis(getConfig().getLong(PostgresConnectorConfig.XMIN_FETCH_INTERVAL)); + } + + protected byte[] toastedValuePlaceholder() { + final String placeholder = getConfig().getString(TOASTED_VALUE_PLACEHOLDER); + if (placeholder.startsWith("hex:")) { + return Strings.hexStringToByteArray(placeholder.substring(4)); + } + return placeholder.getBytes(); + } + + @Override + protected SourceInfoStructMaker getSourceInfoStructMaker( + CommonConnectorConfig.Version version) { + switch (version) { + case V1: + return new LegacyV1PostgresSourceInfoStructMaker( + Module.name(), Module.version(), this); + default: + return new PostgresSourceInfoStructMaker(Module.name(), Module.version(), this); + } + } + + private static final ConfigDefinition CONFIG_DEFINITION = + RelationalDatabaseConnectorConfig.CONFIG_DEFINITION + .edit() + .name("Postgres") + .type( + HOSTNAME, + PORT, + USER, + PASSWORD, + DATABASE_NAME, + PLUGIN_NAME, + SLOT_NAME, + PUBLICATION_NAME, + PUBLICATION_AUTOCREATE_MODE, + DROP_SLOT_ON_STOP, + STREAM_PARAMS, + ON_CONNECT_STATEMENTS, + SSL_MODE, + SSL_CLIENT_CERT, + SSL_CLIENT_KEY_PASSWORD, + SSL_ROOT_CERT, + SSL_CLIENT_KEY, + MAX_RETRIES, + RETRY_DELAY_MS, + SSL_SOCKET_FACTORY, + STATUS_UPDATE_INTERVAL_MS, + TCP_KEEPALIVE, + XMIN_FETCH_INTERVAL) + .events( + INCLUDE_UNKNOWN_DATATYPES, + DatabaseHeartbeatImpl.HEARTBEAT_ACTION_QUERY, + TOASTED_VALUE_PLACEHOLDER) + .connector( + SNAPSHOT_MODE, + SNAPSHOT_MODE_CLASS, + HSTORE_HANDLING_MODE, + BINARY_HANDLING_MODE, + INTERVAL_HANDLING_MODE, + SCHEMA_REFRESH_MODE, + INCREMENTAL_SNAPSHOT_CHUNK_SIZE) + .excluding(INCLUDE_SCHEMA_CHANGES) + .create(); + + /** The set of {@link Field}s defined as part of this configuration. */ + public static Field.Set ALL_FIELDS = Field.setOf(CONFIG_DEFINITION.all()); + + public static ConfigDef configDef() { + return CONFIG_DEFINITION.configDef(); + } + + // Source of the validation rules - + // https://doxygen.postgresql.org/slot_8c.html#afac399f07320b9adfd2c599cf822aaa3 + private static int validateReplicationSlotName( + Configuration config, Field field, Field.ValidationOutput problems) { + final String name = config.getString(field); + int errors = 0; + if (name != null) { + if (!name.matches("[a-z0-9_]{1,63}")) { + problems.accept( + field, + name, + "Valid replication slot name must contain only digits, lowercase characters and underscores with length <= 63"); + ++errors; + } + } + return errors; + } + + private static int validateTruncateHandlingMode( + Configuration config, Field field, Field.ValidationOutput problems) { + final String value = config.getString(field); + int errors = 0; + if (value != null) { + TruncateHandlingMode truncateHandlingMode = TruncateHandlingMode.parse(value); + if (truncateHandlingMode == null) { + List validModes = + Arrays.stream(TruncateHandlingMode.values()) + .map(TruncateHandlingMode::getValue) + .collect(Collectors.toList()); + String message = + String.format( + "Valid values for %s are %s, but got '%s'", + field.name(), validModes, value); + problems.accept(field, value, message); + errors++; + return errors; + } + if (truncateHandlingMode == TruncateHandlingMode.INCLUDE) { + LogicalDecoder logicalDecoder = + config.getInstance(PLUGIN_NAME, LogicalDecoder.class); + if (!logicalDecoder.supportsTruncate()) { + String message = + String.format( + "%s '%s' is not supported with configuration %s '%s'", + field.name(), + truncateHandlingMode.getValue(), + PLUGIN_NAME.name(), + logicalDecoder.getValue()); + problems.accept(field, value, message); + errors++; + } + } + } + return errors; + } + + @Override + public String getContextName() { + return Module.contextName(); + } + + @Override + public String getConnectorName() { + return Module.name(); + } + + private static class SystemTablesPredicate implements TableFilter { + protected static final List SYSTEM_SCHEMAS = + Arrays.asList("pg_catalog", "information_schema"); + + @Override + public boolean isIncluded(TableId t) { + return !SYSTEM_SCHEMAS.contains(t.schema().toLowerCase()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java new file mode 100644 index 00000000000..3616c4c2d12 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java @@ -0,0 +1,85 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.apache.kafka.connect.data.Struct; + +import io.debezium.data.Envelope; +import io.debezium.pipeline.source.spi.EventMetadataProvider; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.schema.DataCollectionId; +import io.debezium.time.Conversions; +import io.debezium.util.Collect; + +import java.time.Instant; +import java.util.Map; + +public class PostgresEventMetadataProvider implements EventMetadataProvider { + + @Override + public Instant getEventTimestamp( + DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + if (sourceInfo.schema().field(SourceInfo.TIMESTAMP_USEC_KEY) != null) { + final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_USEC_KEY); + return timestamp == null ? null : Conversions.toInstantFromMicros(timestamp); + } + final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_KEY); + return timestamp == null ? null : Instant.ofEpochMilli(timestamp); + } + + @Override + public Map getEventSourcePosition( + DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + Long xmin = sourceInfo.getInt64(SourceInfo.XMIN_KEY); + + Map r = + Collect.hashMapOf( + SourceInfo.LSN_KEY, Long.toString(sourceInfo.getInt64(SourceInfo.LSN_KEY))); + if (xmin != null) { + r.put(SourceInfo.XMIN_KEY, Long.toString(xmin)); + } + return r; + } + + @Override + public String getTransactionId( + DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + return Long.toString(sourceInfo.getInt64(SourceInfo.TXID_KEY)); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java new file mode 100644 index 00000000000..f42021fed37 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java @@ -0,0 +1,382 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.connector.SnapshotRecord; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.spi.OffsetState; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.relational.TableId; +import io.debezium.schema.DataCollectionId; +import io.debezium.time.Conversions; +import io.debezium.util.Clock; + +import java.sql.SQLException; +import java.time.Instant; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class PostgresOffsetContext implements OffsetContext { + private static final Logger LOGGER = + LoggerFactory.getLogger(PostgresSnapshotChangeEventSource.class); + + private static final String SERVER_PARTITION_KEY = "server"; + public static final String LAST_COMPLETELY_PROCESSED_LSN_KEY = "lsn_proc"; + public static final String LAST_COMMIT_LSN_KEY = "lsn_commit"; + + private final Schema sourceInfoSchema; + private final SourceInfo sourceInfo; + private final Map partition; + private boolean lastSnapshotRecord; + private Lsn lastCompletelyProcessedLsn; + private Lsn lastCommitLsn; + private Lsn streamingStoppingLsn = null; + private final TransactionContext transactionContext; + private final IncrementalSnapshotContext incrementalSnapshotContext; + + private PostgresOffsetContext( + PostgresConnectorConfig connectorConfig, + Lsn lsn, + Lsn lastCompletelyProcessedLsn, + Lsn lastCommitLsn, + Long txId, + Instant time, + boolean snapshot, + boolean lastSnapshotRecord, + TransactionContext transactionContext, + IncrementalSnapshotContext incrementalSnapshotContext) { + partition = + Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName()); + sourceInfo = new SourceInfo(connectorConfig); + + this.lastCompletelyProcessedLsn = lastCompletelyProcessedLsn; + this.lastCommitLsn = lastCommitLsn; + sourceInfo.update(lsn, time, txId, null, sourceInfo.xmin(), lastCommitLsn); + sourceInfoSchema = sourceInfo.schema(); + + this.lastSnapshotRecord = lastSnapshotRecord; + if (this.lastSnapshotRecord) { + postSnapshotCompletion(); + } else { + sourceInfo.setSnapshot(snapshot ? SnapshotRecord.TRUE : SnapshotRecord.FALSE); + } + this.transactionContext = transactionContext; + this.incrementalSnapshotContext = incrementalSnapshotContext; + } + + @Override + public Map getPartition() { + return partition; + } + + @Override + public Map getOffset() { + Map result = new HashMap<>(); + if (sourceInfo.timestamp() != null) { + result.put( + SourceInfo.TIMESTAMP_USEC_KEY, + Conversions.toEpochMicros(sourceInfo.timestamp())); + } + if (sourceInfo.txId() != null) { + result.put(SourceInfo.TXID_KEY, sourceInfo.txId()); + } + if (sourceInfo.lsn() != null) { + result.put(SourceInfo.LSN_KEY, sourceInfo.lsn().asLong()); + } + if (sourceInfo.xmin() != null) { + result.put(SourceInfo.XMIN_KEY, sourceInfo.xmin()); + } + if (sourceInfo.isSnapshot()) { + result.put(SourceInfo.SNAPSHOT_KEY, true); + result.put(SourceInfo.LAST_SNAPSHOT_RECORD_KEY, lastSnapshotRecord); + } + if (lastCompletelyProcessedLsn != null) { + result.put(LAST_COMPLETELY_PROCESSED_LSN_KEY, lastCompletelyProcessedLsn.asLong()); + } + if (lastCommitLsn != null) { + result.put(LAST_COMMIT_LSN_KEY, lastCommitLsn.asLong()); + } + return sourceInfo.isSnapshot() + ? result + : incrementalSnapshotContext.store(transactionContext.store(result)); + } + + @Override + public Schema getSourceInfoSchema() { + return sourceInfoSchema; + } + + @Override + public Struct getSourceInfo() { + return sourceInfo.struct(); + } + + @Override + public boolean isSnapshotRunning() { + return sourceInfo.isSnapshot(); + } + + @Override + public void preSnapshotStart() { + sourceInfo.setSnapshot(SnapshotRecord.TRUE); + lastSnapshotRecord = false; + } + + @Override + public void preSnapshotCompletion() { + lastSnapshotRecord = true; + } + + @Override + public void postSnapshotCompletion() { + sourceInfo.setSnapshot(SnapshotRecord.FALSE); + } + + public void updateSnapshotPosition(Instant timestamp, TableId tableId) { + sourceInfo.update(timestamp, tableId); + } + + public void updateWalPosition( + Lsn lsn, + Lsn lastCompletelyProcessedLsn, + Instant commitTime, + Long txId, + TableId tableId, + Long xmin) { + this.lastCompletelyProcessedLsn = lastCompletelyProcessedLsn; + sourceInfo.update(lsn, commitTime, txId, tableId, xmin); + } + + public void updateCommitPosition( + Lsn lsn, + Lsn lastCompletelyProcessedLsn, + Instant commitTime, + Long txId, + TableId tableId, + Long xmin) { + this.lastCompletelyProcessedLsn = lastCompletelyProcessedLsn; + this.lastCommitLsn = lastCompletelyProcessedLsn; + sourceInfo.update(lsn, commitTime, txId, tableId, xmin, lastCompletelyProcessedLsn); + } + + boolean hasLastKnownPosition() { + return sourceInfo.lsn() != null; + } + + boolean hasCompletelyProcessedPosition() { + return this.lastCompletelyProcessedLsn != null; + } + + Lsn lsn() { + return sourceInfo.lsn(); + } + + Lsn lastCompletelyProcessedLsn() { + return lastCompletelyProcessedLsn; + } + + Lsn lastCommitLsn() { + return lastCommitLsn; + } + + /** + * Returns the LSN that the streaming phase should stream events up to or null if a stopping + * point is not set. If set during the streaming phase, any event with an LSN less than the + * stopping LSN will be processed and once the stopping LSN is reached, the streaming phase will + * end. Useful for a pre-snapshot catch up streaming phase. + */ + Lsn getStreamingStoppingLsn() { + return streamingStoppingLsn; + } + + public void setStreamingStoppingLsn(Lsn streamingStoppingLsn) { + this.streamingStoppingLsn = streamingStoppingLsn; + } + + Long xmin() { + return sourceInfo.xmin(); + } + + public static class Loader implements OffsetContext.Loader { + + private final PostgresConnectorConfig connectorConfig; + + public Loader(PostgresConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + } + + @Override + public Map getPartition() { + return Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName()); + } + + private Long longOffsetValue(Map values, String key) { + Object obj = values.get(key); + if (obj == null) { + return null; + } + if (obj instanceof Number) { + return ((Number) obj).longValue(); + } + try { + return Long.parseLong(obj.toString()); + } catch (NumberFormatException ne) { + return Lsn.valueOf((String) obj).asLong(); + } + } + + @SuppressWarnings("unchecked") + @Override + public PostgresOffsetContext load(Map offset) { + final Lsn lsn = Lsn.valueOf(longOffsetValue(offset, SourceInfo.LSN_KEY)); + final Lsn lastCompletelyProcessedLsn = + Lsn.valueOf(longOffsetValue(offset, LAST_COMPLETELY_PROCESSED_LSN_KEY)); + final Lsn lastCommitLsn = + Lsn.valueOf(longOffsetValue(offset, LAST_COMPLETELY_PROCESSED_LSN_KEY)); + final Long txId = longOffsetValue(offset, SourceInfo.TXID_KEY); + + final Instant useconds = + offset.get(SourceInfo.TIMESTAMP_USEC_KEY) != null + ? Conversions.toInstantFromMicros( + (Long) offset.get(SourceInfo.TIMESTAMP_USEC_KEY)) + : Clock.system().currentTimeAsInstant(); + final boolean snapshot = + (boolean) + ((Map) offset) + .getOrDefault(SourceInfo.SNAPSHOT_KEY, Boolean.FALSE); + final boolean lastSnapshotRecord = + (boolean) + ((Map) offset) + .getOrDefault( + SourceInfo.LAST_SNAPSHOT_RECORD_KEY, Boolean.FALSE); + return new PostgresOffsetContext( + connectorConfig, + lsn, + lastCompletelyProcessedLsn, + lastCommitLsn, + txId, + useconds, + snapshot, + lastSnapshotRecord, + TransactionContext.load(offset), + IncrementalSnapshotContext.load(offset, TableId.class)); + } + } + + @Override + public String toString() { + return "PostgresOffsetContext [sourceInfoSchema=" + + sourceInfoSchema + + ", sourceInfo=" + + sourceInfo + + ", partition=" + + partition + + ", lastSnapshotRecord=" + + lastSnapshotRecord + + ", lastCompletelyProcessedLsn=" + + lastCompletelyProcessedLsn + + ", lastCommitLsn=" + + lastCommitLsn + + ", streamingStoppingLsn=" + + streamingStoppingLsn + + ", transactionContext=" + + transactionContext + + ", incrementalSnapshotContext=" + + incrementalSnapshotContext + + "]"; + } + + public static PostgresOffsetContext initialContext( + PostgresConnectorConfig connectorConfig, + PostgresConnection jdbcConnection, + Clock clock) { + return initialContext(connectorConfig, jdbcConnection, clock, null, null); + } + + public static PostgresOffsetContext initialContext( + PostgresConnectorConfig connectorConfig, + PostgresConnection jdbcConnection, + Clock clock, + Lsn lastCommitLsn, + Lsn lastCompletelyProcessedLsn) { + try { + LOGGER.info("Creating initial offset context"); + final Lsn lsn = Lsn.valueOf(jdbcConnection.currentXLogLocation()); + final long txId = jdbcConnection.currentTransactionId().longValue(); + LOGGER.info("Read xlogStart at '{}' from transaction '{}'", lsn, txId); + return new PostgresOffsetContext( + connectorConfig, + lsn, + lastCompletelyProcessedLsn, + lastCommitLsn, + txId, + clock.currentTimeAsInstant(), + false, + false, + new TransactionContext(), + new IncrementalSnapshotContext<>()); + } catch (SQLException e) { + throw new ConnectException("Database processing error", e); + } + } + + public OffsetState asOffsetState() { + return new OffsetState( + sourceInfo.lsn(), + sourceInfo.txId(), + sourceInfo.xmin(), + sourceInfo.timestamp(), + sourceInfo.isSnapshot()); + } + + @Override + public void markLastSnapshotRecord() { + sourceInfo.setSnapshot(SnapshotRecord.LAST); + } + + @Override + public void event(DataCollectionId tableId, Instant instant) { + sourceInfo.update(instant, (TableId) tableId); + } + + @Override + public TransactionContext getTransactionContext() { + return transactionContext; + } + + @Override + public void incrementalSnapshotEvents() { + sourceInfo.setSnapshot(SnapshotRecord.INCREMENTAL); + } + + @Override + public IncrementalSnapshotContext getIncrementalSnapshotContext() { + return incrementalSnapshotContext; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java new file mode 100644 index 00000000000..008bca903d3 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java @@ -0,0 +1,310 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.errors.ConnectException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.annotation.NotThreadSafe; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ServerInfo; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.RelationalDatabaseSchema; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.TableSchemaBuilder; +import io.debezium.relational.Tables; +import io.debezium.schema.TopicSelector; +import io.debezium.util.SchemaNameAdjuster; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * Component that records the schema information for the {@link PostgresConnector}. The schema + * information contains the {@link Tables table definitions} and the Kafka Connect {@link + * #schemaFor(TableId) Schema}s for each table, where the {@link Schema} excludes any columns that + * have been {@link PostgresConnectorConfig#COLUMN_EXCLUDE_LIST specified} in the configuration. + */ +@NotThreadSafe +public class PostgresSchema extends RelationalDatabaseSchema { + + protected static final String PUBLIC_SCHEMA_NAME = "public"; + private static final Logger LOGGER = LoggerFactory.getLogger(PostgresSchema.class); + + private final TypeRegistry typeRegistry; + + private final Map> tableIdToToastableColumns; + private final Map relationIdToTableId; + private final boolean readToastableColumns; + + /** + * Create a schema component given the supplied {@link PostgresConnectorConfig Postgres + * connector configuration}. + * + * @param config the connector configuration, which is presumed to be valid + */ + public PostgresSchema( + PostgresConnectorConfig config, + TypeRegistry typeRegistry, + TopicSelector topicSelector, + PostgresValueConverter valueConverter) { + super( + config, + topicSelector, + new Filters(config).tableFilter(), + config.getColumnFilter(), + getTableSchemaBuilder(config, valueConverter), + false, + config.getKeyMapper()); + + this.typeRegistry = typeRegistry; + this.tableIdToToastableColumns = new HashMap<>(); + this.relationIdToTableId = new HashMap<>(); + this.readToastableColumns = config.skipRefreshSchemaOnMissingToastableData(); + } + + private static TableSchemaBuilder getTableSchemaBuilder( + PostgresConnectorConfig config, PostgresValueConverter valueConverter) { + return new TableSchemaBuilder( + valueConverter, + SchemaNameAdjuster.create(), + config.customConverterRegistry(), + config.getSourceInfoStructMaker().schema(), + config.getSanitizeFieldNames()); + } + + /** + * Initializes the content for this schema by reading all the database information from the + * supplied connection. + * + * @param connection a {@link JdbcConnection} instance, never {@code null} + * @param printReplicaIdentityInfo whether or not to look and print out replica identity + * information about the tables + * @return this object so methods can be chained together; never null + * @throws SQLException if there is a problem obtaining the schema from the database server + */ + protected PostgresSchema refresh( + PostgresConnection connection, boolean printReplicaIdentityInfo) throws SQLException { + // read all the information from the DB + connection.readSchema(tables(), null, null, getTableFilter(), null, true); + if (printReplicaIdentityInfo) { + // print out all the replica identity info + tableIds().forEach(tableId -> printReplicaIdentityInfo(connection, tableId)); + } + // and then refresh the schemas + refreshSchemas(); + if (readToastableColumns) { + tableIds().forEach(tableId -> refreshToastableColumnsMap(connection, tableId)); + } + return this; + } + + private void printReplicaIdentityInfo(PostgresConnection connection, TableId tableId) { + try { + ServerInfo.ReplicaIdentity replicaIdentity = + connection.readReplicaIdentityInfo(tableId); + LOGGER.info( + "REPLICA IDENTITY for '{}' is '{}'; {}", + tableId, + replicaIdentity, + replicaIdentity.description()); + } catch (SQLException e) { + LOGGER.warn("Cannot determine REPLICA IDENTITY info for '{}'", tableId); + } + } + + /** + * Refreshes this schema's content for a particular table + * + * @param connection a {@link JdbcConnection} instance, never {@code null} + * @param tableId the table identifier; may not be null + * @param refreshToastableColumns refreshes the cache of toastable columns for `tableId`, if + * {@code true} + * @throws SQLException if there is a problem refreshing the schema from the database server + */ + protected void refresh( + PostgresConnection connection, TableId tableId, boolean refreshToastableColumns) + throws SQLException { + Tables temp = new Tables(); + connection.readSchema(temp, null, null, tableId::equals, null, true); + + // the table could be deleted before the event was processed + if (temp.size() == 0) { + LOGGER.warn("Refresh of {} was requested but the table no longer exists", tableId); + return; + } + // overwrite (add or update) or views of the tables + tables().overwriteTable(temp.forTable(tableId)); + // refresh the schema + refreshSchema(tableId); + + if (refreshToastableColumns) { + // and refresh toastable columns info + refreshToastableColumnsMap(connection, tableId); + } + } + + protected boolean isFilteredOut(TableId id) { + return !getTableFilter().isIncluded(id); + } + + /** Discard any currently-cached schemas and rebuild them using the filters. */ + protected void refreshSchemas() { + clearSchemas(); + + // Create TableSchema instances for any existing table ... + tableIds().forEach(this::refreshSchema); + } + + private void refreshToastableColumnsMap(PostgresConnection connection, TableId tableId) { + // This method populates the list of 'toastable' columns for `tableId`. + // A toastable column is one that has storage strategy 'x' (inline-compressible + secondary + // storage enabled), + // 'e' (secondary storage enabled), or 'm' (inline-compressible). + // + // Note that, rather confusingly, the 'm' storage strategy does in fact permit secondary + // storage, but only as a + // last resort. + // + // Also, we cannot account for the possibility that future versions of PostgreSQL introduce + // new storage strategies + // that include secondary storage. We should move to native decoding in PG 10 and get rid of + // this hacky code + // before that possibility is realized. + + // Collect the non-system (attnum > 0), present (not attisdropped) column names that are + // toastable. + // + // NOTE (Ian Axelrod): + // I Would prefer to use data provided by PgDatabaseMetaData, but the PG JDBC driver does + // not expose storage type + // information. Thus, we need to make a separate query. If we are refreshing schemas rarely, + // this is not a big + // deal. + List toastableColumns = new ArrayList<>(); + String relName = tableId.table(); + String schema = + tableId.schema() != null && tableId.schema().length() > 0 + ? tableId.schema() + : "public"; + String statement = + "select att.attname" + + " from pg_attribute att " + + " join pg_class tbl on tbl.oid = att.attrelid" + + " join pg_namespace ns on tbl.relnamespace = ns.oid" + + " where tbl.relname = ?" + + " and ns.nspname = ?" + + " and att.attnum > 0" + + " and att.attstorage in ('x', 'e', 'm')" + + " and not att.attisdropped;"; + + try { + connection.prepareQuery( + statement, + stmt -> { + stmt.setString(1, relName); + stmt.setString(2, schema); + }, + rs -> { + while (rs.next()) { + toastableColumns.add(rs.getString(1)); + } + }); + if (!connection.connection().getAutoCommit()) { + connection.connection().commit(); + } + } catch (SQLException e) { + throw new ConnectException("Unable to refresh toastable columns mapping", e); + } + + tableIdToToastableColumns.put(tableId, Collections.unmodifiableList(toastableColumns)); + } + + protected static TableId parse(String table) { + TableId tableId = TableId.parse(table, false); + if (tableId == null) { + return null; + } + return tableId.schema() == null + ? new TableId(tableId.catalog(), PUBLIC_SCHEMA_NAME, tableId.table()) + : tableId; + } + + public TypeRegistry getTypeRegistry() { + return typeRegistry; + } + + public List getToastableColumnsForTableId(TableId tableId) { + return tableIdToToastableColumns.getOrDefault(tableId, Collections.emptyList()); + } + + /** + * Applies schema changes for the specified table. + * + * @param relationId the postgres relation unique identifier for the table + * @param table externally constructed table, typically from the decoder; must not be null + */ + public void applySchemaChangesForTable(int relationId, Table table) { + assert table != null; + + if (isFilteredOut(table.id())) { + LOGGER.trace( + "Skipping schema refresh for table '{}' with relation '{}' as table is filtered", + table.id(), + relationId); + return; + } + + relationIdToTableId.put(relationId, table.id()); + refresh(table); + } + + /** + * Resolve a {@link Table} based on a supplied table relation unique identifier. + * + *

This implementation relies on a prior call to {@link #applySchemaChangesForTable(int, + * Table)} to have applied schema changes from a replication stream with the {@code relationId} + * for the relationship to exist and be capable of lookup. + * + * @param relationId the unique table relation identifier + * @return the resolved table or null + */ + public Table tableFor(int relationId) { + TableId tableId = relationIdToTableId.get(relationId); + if (tableId == null) { + LOGGER.debug("Relation '{}' is unknown, cannot resolve to table", relationId); + return null; + } + LOGGER.debug("Relation '{}' resolved to table '{}'", relationId, tableId); + return tableFor(tableId); + } + + @Override + public boolean tableInformationComplete() { + // PostgreSQL does not support HistorizedDatabaseSchema - so no tables are recovered + return false; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java new file mode 100644 index 00000000000..a1a152dbcf0 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.annotation.ThreadSafe; +import io.debezium.connector.common.CdcSourceTaskContext; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.connector.postgresql.spi.SlotState; +import io.debezium.relational.TableId; +import io.debezium.schema.TopicSelector; +import io.debezium.util.Clock; +import io.debezium.util.ElapsedTimeStrategy; + +import java.sql.SQLException; +import java.util.Collections; + +/** + * The context of a {@link PostgresConnectorTask}. This deals with most of the brunt of reading + * various configuration options and creating other objects with these various options. + */ +@ThreadSafe +public class PostgresTaskContext extends CdcSourceTaskContext { + protected static final Logger LOGGER = LoggerFactory.getLogger(PostgresTaskContext.class); + + private final PostgresConnectorConfig config; + private final TopicSelector topicSelector; + private final PostgresSchema schema; + + private ElapsedTimeStrategy refreshXmin; + private Long lastXmin; + + public PostgresTaskContext( + PostgresConnectorConfig config, + PostgresSchema schema, + TopicSelector topicSelector) { + super(config.getContextName(), config.getLogicalName(), Collections::emptySet); + + this.config = config; + if (config.xminFetchInterval().toMillis() > 0) { + this.refreshXmin = + ElapsedTimeStrategy.constant( + Clock.SYSTEM, config.xminFetchInterval().toMillis()); + } + this.topicSelector = topicSelector; + assert schema != null; + this.schema = schema; + } + + protected TopicSelector topicSelector() { + return topicSelector; + } + + protected PostgresSchema schema() { + return schema; + } + + protected PostgresConnectorConfig config() { + return config; + } + + public void refreshSchema(PostgresConnection connection, boolean printReplicaIdentityInfo) + throws SQLException { + schema.refresh(connection, printReplicaIdentityInfo); + } + + Long getSlotXmin(PostgresConnection connection) throws SQLException { + // when xmin fetch is set to 0, we don't track it to ignore any performance of querying the + // slot periodically + if (config.xminFetchInterval().toMillis() <= 0) { + return null; + } + assert this.refreshXmin != null; + + if (this.refreshXmin.hasElapsed()) { + lastXmin = getCurrentSlotState(connection).slotCatalogXmin(); + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("Fetched new xmin from slot of {}", lastXmin); + } + } else { + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("reusing xmin value of {}", lastXmin); + } + } + + return lastXmin; + } + + private SlotState getCurrentSlotState(PostgresConnection connection) throws SQLException { + return connection.getReplicationSlotState( + config.slotName(), config.plugin().getPostgresPluginName()); + } + + public ReplicationConnection createReplicationConnection(boolean doSnapshot) + throws SQLException { + final boolean dropSlotOnStop = config.dropSlotOnStop(); + if (dropSlotOnStop) { + LOGGER.warn( + "Connector has enabled automated replication slot removal upon restart ({} = true). " + + "This setting is not recommended for production environments, as a new replication slot " + + "will be created after a connector restart, resulting in missed data change events.", + PostgresConnectorConfig.DROP_SLOT_ON_STOP.name()); + } + return ReplicationConnection.builder(config) + .withSlot(config.slotName()) + .withPublication(config.publicationName()) + .withTableFilter(config.getTableFilters()) + .withPublicationAutocreateMode(config.publicationAutocreateMode()) + .withPlugin(config.plugin()) + .withTruncateHandlingMode(config.truncateHandlingMode()) + .dropSlotOnClose(dropSlotOnStop) + .streamParams(config.streamParams()) + .statusUpdateInterval(config.statusUpdateInterval()) + .withTypeRegistry(schema.getTypeRegistry()) + .doSnapshot(doSnapshot) + .withSchema(schema) + .build(); + } + + PostgresConnectorConfig getConfig() { + return config; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java new file mode 100644 index 00000000000..76eaa25f122 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java @@ -0,0 +1,487 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql; + +import org.apache.kafka.connect.errors.ConnectException; + +import org.postgresql.core.BaseConnection; +import org.postgresql.core.TypeInfo; +import org.postgresql.jdbc.PgDatabaseMetaData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.DebeziumException; +import io.debezium.annotation.Immutable; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.util.Collect; + +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.sql.Types; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A registry of types supported by a PostgreSQL instance. Allows lookup of the types according to + * type name or OID. + * + * @author Jiri Pechanec + */ +public class TypeRegistry { + + private static final Logger LOGGER = LoggerFactory.getLogger(TypeRegistry.class); + + public static final String TYPE_NAME_GEOGRAPHY = "geography"; + public static final String TYPE_NAME_GEOMETRY = "geometry"; + public static final String TYPE_NAME_CITEXT = "citext"; + public static final String TYPE_NAME_HSTORE = "hstore"; + public static final String TYPE_NAME_LTREE = "ltree"; + + public static final String TYPE_NAME_HSTORE_ARRAY = "_hstore"; + public static final String TYPE_NAME_GEOGRAPHY_ARRAY = "_geography"; + public static final String TYPE_NAME_GEOMETRY_ARRAY = "_geometry"; + public static final String TYPE_NAME_CITEXT_ARRAY = "_citext"; + public static final String TYPE_NAME_LTREE_ARRAY = "_ltree"; + + public static final int NO_TYPE_MODIFIER = -1; + public static final int UNKNOWN_LENGTH = -1; + + // PostgreSQL driver reports user-defined Domain types as Types.DISTINCT + public static final int DOMAIN_TYPE = Types.DISTINCT; + + private static final String CATEGORY_ARRAY = "A"; + private static final String CATEGORY_ENUM = "E"; + + private static final String SQL_ENUM_VALUES = + "SELECT t.enumtypid as id, array_agg(t.enumlabel) as values " + + "FROM pg_catalog.pg_enum t GROUP BY id"; + + private static final String SQL_TYPES = + "SELECT t.oid AS oid, t.typname AS name, t.typelem AS element, t.typbasetype AS parentoid, t.typtypmod as modifiers, t.typcategory as category, e.values as enum_values " + + "FROM pg_catalog.pg_type t " + + "JOIN pg_catalog.pg_namespace n ON (t.typnamespace = n.oid) " + + "LEFT JOIN (" + + SQL_ENUM_VALUES + + ") e ON (t.oid = e.id) " + + "WHERE n.nspname != 'pg_toast' and t.typtypmod != 0"; + + private static final String SQL_NAME_LOOKUP = SQL_TYPES + " AND t.typname = ?"; + + private static final String SQL_OID_LOOKUP = SQL_TYPES + " AND t.oid = ?"; + + private static final Map LONG_TYPE_NAMES = + Collections.unmodifiableMap(getLongTypeNames()); + + private static Map getLongTypeNames() { + Map longTypeNames = new HashMap<>(); + + longTypeNames.put("bigint", "int8"); + longTypeNames.put("bit varying", "varbit"); + longTypeNames.put("boolean", "bool"); + longTypeNames.put("character", "bpchar"); + longTypeNames.put("character varying", "varchar"); + longTypeNames.put("double precision", "float8"); + longTypeNames.put("integer", "int4"); + longTypeNames.put("real", "float4"); + longTypeNames.put("smallint", "int2"); + longTypeNames.put("timestamp without time zone", "timestamp"); + longTypeNames.put("timestamp with time zone", "timestamptz"); + longTypeNames.put("time without time zone", "time"); + longTypeNames.put("time with time zone", "timetz"); + + return longTypeNames; + } + + private final Map nameToType = new HashMap<>(); + private final Map oidToType = new HashMap<>(); + + private final Connection connection; + private final TypeInfo typeInfo; + private final SqlTypeMapper sqlTypeMapper; + + private int geometryOid = Integer.MIN_VALUE; + private int geographyOid = Integer.MIN_VALUE; + private int citextOid = Integer.MIN_VALUE; + private int hstoreOid = Integer.MIN_VALUE; + private int ltreeOid = Integer.MIN_VALUE; + + private int hstoreArrayOid = Integer.MIN_VALUE; + private int geometryArrayOid = Integer.MIN_VALUE; + private int geographyArrayOid = Integer.MIN_VALUE; + private int citextArrayOid = Integer.MIN_VALUE; + private int ltreeArrayOid = Integer.MIN_VALUE; + + public TypeRegistry(PostgresConnection connection) { + try { + this.connection = connection.connection(); + typeInfo = ((BaseConnection) this.connection).getTypeInfo(); + sqlTypeMapper = new SqlTypeMapper(this.connection, typeInfo); + + prime(); + } catch (SQLException e) { + throw new DebeziumException("Couldn't initialize type registry", e); + } + } + + private void addType(PostgresType type) { + oidToType.put(type.getOid(), type); + nameToType.put(type.getName(), type); + + if (TYPE_NAME_GEOMETRY.equals(type.getName())) { + geometryOid = type.getOid(); + } else if (TYPE_NAME_GEOGRAPHY.equals(type.getName())) { + geographyOid = type.getOid(); + } else if (TYPE_NAME_CITEXT.equals(type.getName())) { + citextOid = type.getOid(); + } else if (TYPE_NAME_HSTORE.equals(type.getName())) { + hstoreOid = type.getOid(); + } else if (TYPE_NAME_LTREE.equals(type.getName())) { + ltreeOid = type.getOid(); + } else if (TYPE_NAME_HSTORE_ARRAY.equals(type.getName())) { + hstoreArrayOid = type.getOid(); + } else if (TYPE_NAME_GEOMETRY_ARRAY.equals(type.getName())) { + geometryArrayOid = type.getOid(); + } else if (TYPE_NAME_GEOGRAPHY_ARRAY.equals(type.getName())) { + geographyArrayOid = type.getOid(); + } else if (TYPE_NAME_CITEXT_ARRAY.equals(type.getName())) { + citextArrayOid = type.getOid(); + } else if (TYPE_NAME_LTREE_ARRAY.equals(type.getName())) { + ltreeArrayOid = type.getOid(); + } + } + + /** + * @param oid - PostgreSQL OID + * @return type associated with the given OID + */ + public PostgresType get(int oid) { + PostgresType r = oidToType.get(oid); + if (r == null) { + r = resolveUnknownType(oid); + if (r == null) { + LOGGER.warn("Unknown OID {} requested", oid); + r = PostgresType.UNKNOWN; + } + } + return r; + } + + /** + * @param name - PostgreSQL type name + * @return type associated with the given type name + */ + public PostgresType get(String name) { + switch (name) { + case "serial": + name = "int4"; + break; + case "smallserial": + name = "int2"; + break; + case "bigserial": + name = "int8"; + break; + } + String[] parts = name.split("\\."); + if (parts.length > 1) { + name = parts[1]; + } + if (name.charAt(0) == '"') { + name = name.substring(1, name.length() - 1); + } + PostgresType r = nameToType.get(name); + if (r == null) { + r = resolveUnknownType(name); + if (r == null) { + LOGGER.warn("Unknown type named {} requested", name); + r = PostgresType.UNKNOWN; + } + } + return r; + } + + /** @return OID for {@code GEOMETRY} type of this PostgreSQL instance */ + public int geometryOid() { + return geometryOid; + } + + /** @return OID for {@code GEOGRAPHY} type of this PostgreSQL instance */ + public int geographyOid() { + return geographyOid; + } + + /** @return OID for {@code CITEXT} type of this PostgreSQL instance */ + public int citextOid() { + return citextOid; + } + + /** @return OID for {@code HSTORE} type of this PostgreSQL instance */ + public int hstoreOid() { + return hstoreOid; + } + + /** @return OID for {@code LTREE} type of this PostgreSQL instance */ + public int ltreeOid() { + return ltreeOid; + } + + /** @return OID for array of {@code HSTORE} type of this PostgreSQL instance */ + public int hstoreArrayOid() { + return hstoreArrayOid; + } + + /** @return OID for array of {@code GEOMETRY} type of this PostgreSQL instance */ + public int geometryArrayOid() { + return geometryArrayOid; + } + + /** @return OID for array of {@code GEOGRAPHY} type of this PostgreSQL instance */ + public int geographyArrayOid() { + return geographyArrayOid; + } + + /** @return OID for array of {@code CITEXT} type of this PostgreSQL instance */ + public int citextArrayOid() { + return citextArrayOid; + } + + /** @return OID for array of {@code LTREE} type of this PostgreSQL instance */ + public int ltreeArrayOid() { + return ltreeArrayOid; + } + + /** + * Converts a type name in long (readable) format like boolean to s standard data + * type name like bool. + * + * @param typeName - a type name in long format + * @return - the type name in standardized format + */ + public static String normalizeTypeName(String typeName) { + return LONG_TYPE_NAMES.getOrDefault(typeName, typeName); + } + + /** Prime the {@link TypeRegistry} with all existing database types */ + private void prime() throws SQLException { + try (final Statement statement = connection.createStatement(); + final ResultSet rs = statement.executeQuery(SQL_TYPES)) { + final List delayResolvedBuilders = new ArrayList<>(); + while (rs.next()) { + PostgresType.Builder builder = createTypeBuilderFromResultSet(rs); + + // If the type does have have a base type, we can build/add immediately. + if (!builder.hasParentType()) { + addType(builder.build()); + continue; + } + + // For types with base type mappings, they need to be delayed. + delayResolvedBuilders.add(builder); + } + + // Resolve delayed builders + for (PostgresType.Builder builder : delayResolvedBuilders) { + addType(builder.build()); + } + } + } + + private PostgresType.Builder createTypeBuilderFromResultSet(ResultSet rs) throws SQLException { + // Coerce long to int so large unsigned values are represented as signed + // Same technique is used in TypeInfoCache + final int oid = (int) rs.getLong("oid"); + final int parentTypeOid = (int) rs.getLong("parentoid"); + final int modifiers = (int) rs.getLong("modifiers"); + String typeName = rs.getString("name"); + String category = rs.getString("category"); + + PostgresType.Builder builder = + new PostgresType.Builder( + this, + typeName, + oid, + sqlTypeMapper.getSqlType(typeName), + modifiers, + typeInfo); + + if (CATEGORY_ENUM.equals(category)) { + String[] enumValues = (String[]) rs.getArray("enum_values").getArray(); + builder = builder.enumValues(Arrays.asList(enumValues)); + } else if (CATEGORY_ARRAY.equals(category)) { + builder = builder.elementType((int) rs.getLong("element")); + } + return builder.parentType(parentTypeOid); + } + + private PostgresType resolveUnknownType(String name) { + try { + LOGGER.trace("Type '{}' not cached, attempting to lookup from database.", name); + + try (final PreparedStatement statement = connection.prepareStatement(SQL_NAME_LOOKUP)) { + statement.setString(1, name); + return loadType(statement); + } + } catch (SQLException e) { + throw new ConnectException( + "Database connection failed during resolving unknown type", e); + } + } + + private PostgresType resolveUnknownType(int lookupOid) { + try { + LOGGER.trace( + "Type OID '{}' not cached, attempting to lookup from database.", lookupOid); + + try (final PreparedStatement statement = connection.prepareStatement(SQL_OID_LOOKUP)) { + statement.setInt(1, lookupOid); + return loadType(statement); + } + } catch (SQLException e) { + throw new ConnectException( + "Database connection failed during resolving unknown type", e); + } + } + + private PostgresType loadType(PreparedStatement statement) throws SQLException { + try (final ResultSet rs = statement.executeQuery()) { + while (rs.next()) { + PostgresType result = createTypeBuilderFromResultSet(rs).build(); + addType(result); + return result; + } + } + return null; + } + + /** + * Allows to obtain the SQL type corresponding to PG types. This uses a custom statement instead + * of going through {@link PgDatabaseMetaData#getTypeInfo()} as the latter causes N+1 SELECTs, + * making it very slow on installations with many custom types. + * + * @author Gunnar Morling + * @see DBZ-899 + */ + private static class SqlTypeMapper { + + /** + * Based on org.postgresql.jdbc.TypeInfoCache.getSQLType(String). To emulate the original + * statement's behavior (which works for single types only), PG's DISTINCT ON extension is + * used to just return the first entry should a type exist in multiple schemas. + */ + private static final String SQL_TYPE_DETAILS = + "SELECT DISTINCT ON (typname) typname, typinput='array_in'::regproc, typtype, sp.r, pg_type.oid " + + " FROM pg_catalog.pg_type " + + " LEFT " + + " JOIN (select ns.oid as nspoid, ns.nspname, r.r " + + " from pg_namespace as ns " + // -- go with older way of unnesting array to be compatible with 8.0 + + " join ( select s.r, (current_schemas(false))[s.r] as nspname " + + " from generate_series(1, array_upper(current_schemas(false), 1)) as s(r) ) as r " + + " using ( nspname ) " + + " ) as sp " + + " ON sp.nspoid = typnamespace " + + " ORDER BY typname, sp.r, pg_type.oid;"; + + private final TypeInfo typeInfo; + + @Immutable private final Set preloadedSqlTypes; + + @Immutable private final Map sqlTypesByPgTypeNames; + + private SqlTypeMapper(Connection db, TypeInfo typeInfo) throws SQLException { + this.typeInfo = typeInfo; + this.preloadedSqlTypes = Collect.unmodifiableSet(typeInfo.getPGTypeNamesWithSQLTypes()); + this.sqlTypesByPgTypeNames = Collections.unmodifiableMap(getSqlTypes(db, typeInfo)); + } + + public int getSqlType(String typeName) throws SQLException { + boolean isCoreType = preloadedSqlTypes.contains(typeName); + + // obtain core types such as bool, int2 etc. from the driver, as it correctly maps these + // types to the JDBC + // type codes. Also those values are cached in TypeInfoCache. + if (isCoreType) { + return typeInfo.getSQLType(typeName); + } + if (typeName.endsWith("[]")) { + return Types.ARRAY; + } + // get custom type mappings from the map which was built up with a single query + else { + try { + final Integer pgType = sqlTypesByPgTypeNames.get(typeName); + if (pgType != null) { + return pgType; + } + LOGGER.info( + "Failed to obtain SQL type information for type {} via custom statement, falling back to TypeInfo#getSQLType()", + typeName); + return typeInfo.getSQLType(typeName); + } catch (Exception e) { + LOGGER.warn( + "Failed to obtain SQL type information for type {} via custom statement, falling back to TypeInfo#getSQLType()", + typeName, + e); + return typeInfo.getSQLType(typeName); + } + } + } + + /** + * Builds up a map of SQL (JDBC) types by PG type name; contains only values for non-core + * types. + */ + private static Map getSqlTypes(Connection db, TypeInfo typeInfo) + throws SQLException { + Map sqlTypesByPgTypeNames = new HashMap<>(); + + try (final Statement statement = db.createStatement()) { + try (final ResultSet rs = statement.executeQuery(SQL_TYPE_DETAILS)) { + while (rs.next()) { + int type; + boolean isArray = rs.getBoolean(2); + String typtype = rs.getString(3); + if (isArray) { + type = Types.ARRAY; + } else if ("c".equals(typtype)) { + type = Types.STRUCT; + } else if ("d".equals(typtype)) { + type = Types.DISTINCT; + } else if ("e".equals(typtype)) { + type = Types.VARCHAR; + } else { + type = Types.OTHER; + } + + sqlTypesByPgTypeNames.put(rs.getString(1), type); + } + } + } + + return sqlTypesByPgTypeNames; + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java new file mode 100644 index 00000000000..8f374bca5bf --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java @@ -0,0 +1,773 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package io.debezium.connector.postgresql.connection; + +import org.apache.kafka.connect.errors.ConnectException; + +import org.postgresql.core.BaseConnection; +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 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 java.nio.charset.Charset; +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Statement; +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 + */ +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()), + JdbcConfiguration.HOSTNAME.withDefault( + PostgresConnectorConfig.HOSTNAME.defaultValueAsString()), + JdbcConfiguration.DATABASE.withDefault( + PostgresConnectorConfig.DATABASE_NAME.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) { + 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()); + } + } + + /** + * 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/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java new file mode 100644 index 00000000000..401173b0874 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config; + +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; +import org.apache.seatunnel.connectors.cdc.base.config.StopConfig; + +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.relational.RelationalTableFilters; + +import java.util.List; +import java.util.Properties; + +public class PostgresSourceConfig extends JdbcSourceConfig { + private static final long serialVersionUID = 1L; + + public PostgresSourceConfig( + StartupConfig startupConfig, + StopConfig stopConfig, + List databaseList, + List tableList, + int splitSize, + double distributionFactorUpper, + double distributionFactorLower, + int sampleShardingThreshold, + int inverseSamplingRate, + Properties dbzProperties, + String driverClassName, + String hostname, + int port, + String username, + String password, + String originUrl, + int fetchSize, + String serverTimeZone, + long connectTimeoutMillis, + int connectMaxRetries, + int connectionPoolSize, + boolean exactlyOnc) { + super( + startupConfig, + stopConfig, + databaseList, + tableList, + splitSize, + distributionFactorUpper, + distributionFactorLower, + sampleShardingThreshold, + inverseSamplingRate, + dbzProperties, + driverClassName, + hostname, + port, + username, + password, + originUrl, + fetchSize, + serverTimeZone, + connectTimeoutMillis, + connectMaxRetries, + connectionPoolSize, + exactlyOnc); + } + + @Override + public PostgresConnectorConfig getDbzConnectorConfig() { + return new PostgresConnectorConfig(getDbzConfiguration()); + } + + public RelationalTableFilters getTableFilters() { + return getDbzConnectorConfig().getTableFilters(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java new file mode 100644 index 00000000000..fcf613c8b6a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config; + +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfigFactory; +import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; +import org.apache.seatunnel.connectors.cdc.debezium.EmbeddedDatabaseHistory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; + +import io.debezium.connector.postgresql.PostgresConnector; + +import java.util.Properties; +import java.util.UUID; +import java.util.stream.Collectors; + +import static com.google.common.base.Preconditions.checkNotNull; + +public class PostgresSourceConfigFactory extends JdbcSourceConfigFactory { + + private static final String DATABASE_SERVER_NAME = "postgres_cdc_source"; + + private static final String DRIVER_CLASS_NAME = "org.postgresql.Driver"; + + private String decodingPluginName = PostgresOptions.DECODING_PLUGIN_NAME.defaultValue(); + + private String slotName = PostgresOptions.SLOT_NAME.defaultValue(); + + private String publicationName = PostgresOptions.PUBLICATION_NAME.defaultValue(); + + protected boolean exactlyOnce = JdbcSourceOptions.EXACTLY_ONCE.defaultValue(); + + @Override + public JdbcSourceConfigFactory fromReadonlyConfig(ReadonlyConfig config) { + super.fromReadonlyConfig(config); + this.decodingPluginName = config.get(PostgresOptions.DECODING_PLUGIN_NAME); + this.slotName = config.get(PostgresOptions.SLOT_NAME); + return this; + } + + @Override + public PostgresSourceConfig create(int subtask) { + Properties props = new Properties(); + props.setProperty("connector.class", PostgresConnector.class.getCanonicalName()); + // 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", DATABASE_SERVER_NAME); + props.setProperty("database.hostname", checkNotNull(hostname)); + props.setProperty("database.user", checkNotNull(username)); + props.setProperty("database.password", checkNotNull(password)); + props.setProperty("database.port", String.valueOf(port)); + props.setProperty("database.dbname", checkNotNull(databaseList.get(0))); + props.setProperty("plugin.name", decodingPluginName); + props.setProperty("slot.name", slotName); + props.setProperty("publication.name", publicationName); + + // database history + props.setProperty("database.history", EmbeddedDatabaseHistory.class.getCanonicalName()); + props.setProperty("database.history.instance.name", UUID.randomUUID() + "_" + subtask); + props.setProperty("database.history.skip.unparseable.ddl", String.valueOf(true)); + props.setProperty("database.history.refer.ddl", String.valueOf(true)); + + if (tableList != null) { + // PostgreSQL identifier is of the form schemaName.tableName + String tableIncludeList = + tableList.stream() + .map(table -> table.substring(table.indexOf(".") + 1)) + .collect(Collectors.joining(",")); + props.setProperty("table.include.list", tableIncludeList); + } + + if (dbzProperties != null) { + props.putAll(dbzProperties); + } + + return new PostgresSourceConfig( + startupConfig, + stopConfig, + databaseList, + tableList, + splitSize, + distributionFactorUpper, + distributionFactorLower, + sampleShardingThreshold, + inverseSamplingRate, + props, + DRIVER_CLASS_NAME, + hostname, + port, + username, + password, + originUrl, + fetchSize, + serverTimeZone, + connectTimeoutMillis, + connectMaxRetries, + connectionPoolSize, + exactlyOnce); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java new file mode 100644 index 00000000000..bd5cdd2291f --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.Options; + +public interface PostgresOptions { + + Option DECODING_PLUGIN_NAME = + Options.key("decoding.plugin.name") + .stringType() + .defaultValue("pgoutput") + .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."); + + Option PUBLICATION_NAME = + Options.key("publication.name") + .stringType() + .defaultValue("seatunnel_pub") + .withDescription( + "The name of the Postgres 10+ publication used for streaming changes from a plugin."); + + Option SLOT_NAME = + Options.key("slot.name") + .stringType() + .defaultValue("seatunnel") + .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 \"seatunnel\"."); +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java new file mode 100644 index 00000000000..2538f6d1cc6 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; + +import org.apache.seatunnel.common.utils.SeaTunnelException; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.enumerator.PostgresChunkSplitter; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot.PostgresSnapshotFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.wal.PostgresWalFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.TableDiscoveryUtils; + +import io.debezium.config.Configuration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.history.TableChanges; + +import java.sql.SQLException; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresConnectionUtils.createPostgresConnection; + +public class PostgresDialect implements JdbcDataSourceDialect { + + private static final long serialVersionUID = 1L; + private final SourceConfig sourceConfig; + + private transient PostgresSchema postgresSchema; + + public PostgresDialect(PostgresSourceConfigFactory configFactory) { + this.sourceConfig = configFactory.create(0); + } + + @Override + public String getName() { + return "Postgres"; + } + + @Override + public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { + // todo: need to check the case sensitive of the database + return true; + } + + @Override + public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { + RelationalDatabaseConnectorConfig dbzConnectorConfig = sourceConfig.getDbzConnectorConfig(); + Configuration jdbcConfig = dbzConnectorConfig.getJdbcConfig(); + return createPostgresConnection(jdbcConfig); + } + + @Override + public ChunkSplitter createChunkSplitter(JdbcSourceConfig sourceConfig) { + return new PostgresChunkSplitter(sourceConfig, this); + } + + @Override + public JdbcConnectionPoolFactory getPooledDataSourceFactory() { + return new PostgresPooledDataSourceFactory(); + } + + @Override + public List discoverDataCollections(JdbcSourceConfig sourceConfig) { + PostgresSourceConfig postgresSourceConfig = (PostgresSourceConfig) sourceConfig; + try (JdbcConnection jdbcConnection = openJdbcConnection(sourceConfig)) { + return TableDiscoveryUtils.listTables( + jdbcConnection, postgresSourceConfig.getTableFilters()); + } catch (SQLException e) { + throw new SeaTunnelException("Error to discover tables: " + e.getMessage(), e); + } + } + + @Override + public TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { + if (postgresSchema == null) { + postgresSchema = new PostgresSchema(); + } + return postgresSchema.getTableSchema(jdbc, tableId); + } + + @Override + public PostgresSourceFetchTaskContext createFetchTaskContext( + SourceSplitBase sourceSplitBase, JdbcSourceConfig taskSourceConfig) { + + return new PostgresSourceFetchTaskContext(taskSourceConfig, this); + } + + @Override + public FetchTask createFetchTask(SourceSplitBase sourceSplitBase) { + if (sourceSplitBase.isSnapshotSplit()) { + return new PostgresSnapshotFetchTask(sourceSplitBase.asSnapshotSplit()); + } else { + return new PostgresWalFetchTask(sourceSplitBase.asIncrementalSplit()); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java new file mode 100644 index 00000000000..100fed9c0b6 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; + +import org.apache.seatunnel.api.configuration.Option; +import org.apache.seatunnel.api.configuration.ReadonlyConfig; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.table.catalog.Catalog; +import org.apache.seatunnel.api.table.catalog.CatalogOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.TablePath; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.common.utils.JdbcUrlUtil; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.DataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.cdc.base.source.IncrementalSource; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.cdc.debezium.DebeziumDeserializationSchema; +import org.apache.seatunnel.connectors.cdc.debezium.DeserializeFormat; +import org.apache.seatunnel.connectors.cdc.debezium.row.DebeziumJsonDeserializeSchema; +import org.apache.seatunnel.connectors.cdc.debezium.row.SeaTunnelRowDebeziumDeserializeSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffsetFactory; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalogFactory; + +import com.google.auto.service.AutoService; +import lombok.NoArgsConstructor; + +import java.time.ZoneId; + +@NoArgsConstructor +@AutoService(SeaTunnelSource.class) +public class PostgresIncrementalSource extends IncrementalSource { + + static final String IDENTIFIER = "Postgres-CDC"; + + public PostgresIncrementalSource( + ReadonlyConfig options, SeaTunnelDataType dataType) { + super(options, dataType); + } + + @Override + public String getPluginName() { + return IDENTIFIER; + } + + @Override + public Option getStartupModeOption() { + return PostgresSourceOptions.STARTUP_MODE; + } + + @Override + public Option getStopModeOption() { + return PostgresSourceOptions.STOP_MODE; + } + + @Override + public SourceConfig.Factory createSourceConfigFactory(ReadonlyConfig config) { + PostgresSourceConfigFactory configFactory = new PostgresSourceConfigFactory(); + configFactory.fromReadonlyConfig(readonlyConfig); + JdbcUrlUtil.UrlInfo urlInfo = + JdbcUrlUtil.getUrlInfo(config.get(JdbcCatalogOptions.BASE_URL)); + configFactory.originUrl(urlInfo.getOrigin()); + configFactory.hostname(urlInfo.getHost()); + configFactory.port(urlInfo.getPort()); + configFactory.startupOptions(startupConfig); + configFactory.stopOptions(stopConfig); + return configFactory; + } + + @SuppressWarnings("unchecked") + @Override + public DebeziumDeserializationSchema createDebeziumDeserializationSchema( + ReadonlyConfig config) { + if (DeserializeFormat.COMPATIBLE_DEBEZIUM_JSON.equals( + config.get(JdbcSourceOptions.FORMAT))) { + return (DebeziumDeserializationSchema) + new DebeziumJsonDeserializeSchema( + config.get(JdbcSourceOptions.DEBEZIUM_PROPERTIES)); + } + + SeaTunnelDataType physicalRowType; + if (dataType == null) { + try (Catalog catalog = new PostgresCatalogFactory().createCatalog("postgres", config)) { + catalog.open(); + CatalogTable table = + catalog.getTable( + TablePath.of(config.get(CatalogOptions.TABLE_NAMES).get(0))); + physicalRowType = table.getTableSchema().toPhysicalRowDataType(); + } + } else { + physicalRowType = dataType; + } + String zoneId = config.get(JdbcSourceOptions.SERVER_TIME_ZONE); + return (DebeziumDeserializationSchema) + SeaTunnelRowDebeziumDeserializeSchema.builder() + .setPhysicalRowType(physicalRowType) + .setResultTypeInfo(physicalRowType) + .setServerTimeZone(ZoneId.of(zoneId)) + .build(); + } + + @Override + public DataSourceDialect createDataSourceDialect(ReadonlyConfig config) { + return new PostgresDialect((PostgresSourceConfigFactory) configFactory); + } + + @Override + public OffsetFactory createOffsetFactory(ReadonlyConfig config) { + return new LsnOffsetFactory( + (PostgresSourceConfigFactory) configFactory, (PostgresDialect) dataSourceDialect); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java new file mode 100644 index 00000000000..9f600534192 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; + +import org.apache.seatunnel.api.configuration.util.OptionRule; +import org.apache.seatunnel.api.source.SeaTunnelSource; +import org.apache.seatunnel.api.source.SourceSplit; +import org.apache.seatunnel.api.table.catalog.CatalogOptions; +import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.connector.TableSource; +import org.apache.seatunnel.api.table.factory.Factory; +import org.apache.seatunnel.api.table.factory.SupportMultipleTable; +import org.apache.seatunnel.api.table.factory.TableFactoryContext; +import org.apache.seatunnel.api.table.factory.TableSourceFactory; +import org.apache.seatunnel.api.table.type.MultipleRowType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRow; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; +import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; + +import com.google.auto.service.AutoService; + +import java.io.Serializable; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +@AutoService(Factory.class) +public class PostgresIncrementalSourceFactory implements TableSourceFactory, SupportMultipleTable { + @Override + public String factoryIdentifier() { + return PostgresIncrementalSource.IDENTIFIER; + } + + @Override + public OptionRule optionRule() { + return JdbcSourceOptions.getBaseRule() + .required( + JdbcSourceOptions.USERNAME, + JdbcSourceOptions.PASSWORD, + CatalogOptions.TABLE_NAMES, + JdbcCatalogOptions.BASE_URL) + .optional( + JdbcSourceOptions.DATABASE_NAMES, + JdbcSourceOptions.SERVER_TIME_ZONE, + JdbcSourceOptions.CONNECT_TIMEOUT_MS, + JdbcSourceOptions.CONNECT_MAX_RETRIES, + JdbcSourceOptions.CONNECTION_POOL_SIZE, + PostgresOptions.DECODING_PLUGIN_NAME, + PostgresOptions.SLOT_NAME, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND, + JdbcSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_UPPER_BOUND, + JdbcSourceOptions.SAMPLE_SHARDING_THRESHOLD) + .optional(PostgresSourceOptions.STARTUP_MODE, PostgresSourceOptions.STOP_MODE) + .conditional( + PostgresSourceOptions.STARTUP_MODE, + StartupMode.SPECIFIC, + SourceOptions.STARTUP_SPECIFIC_OFFSET_POS) + .conditional( + PostgresSourceOptions.STOP_MODE, + StopMode.SPECIFIC, + SourceOptions.STOP_SPECIFIC_OFFSET_POS) + .conditional( + PostgresSourceOptions.STARTUP_MODE, + StartupMode.INITIAL, + SourceOptions.EXACTLY_ONCE) + .build(); + } + + @Override + public Class getSourceClass() { + return PostgresIncrementalSource.class; + } + + @Override + public + TableSource createSource(TableFactoryContext context) { + return () -> { + SeaTunnelDataType dataType; + if (context.getCatalogTables().size() == 1) { + dataType = + context.getCatalogTables().get(0).getTableSchema().toPhysicalRowDataType(); + } else { + Map rowTypeMap = new HashMap<>(); + for (CatalogTable catalogTable : context.getCatalogTables()) { + rowTypeMap.put( + catalogTable.getTableId().toTablePath().toString(), + catalogTable.getTableSchema().toPhysicalRowDataType()); + } + dataType = new MultipleRowType(rowTypeMap); + } + return (SeaTunnelSource) + new PostgresIncrementalSource<>(context.getOptions(), dataType); + }; + } + + @Override + public Result applyTables(TableFactoryContext context) { + return Result.of(context.getCatalogTables(), Collections.emptyList()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java new file mode 100644 index 00000000000..e1cfa4e912e --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; + +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; + +/** Factory to create {@link JdbcConnectionPoolFactory} for Postgre SQL. */ +public class PostgresPooledDataSourceFactory extends JdbcConnectionPoolFactory { + + private static final String URL_PATTERN = "jdbc:postgresql://%s:%s/%s"; + + @Override + public String getJdbcUrl(JdbcSourceConfig sourceConfig) { + String hostName = sourceConfig.getHostname(); + int port = sourceConfig.getPort(); + String database = sourceConfig.getDatabaseList().get(0); + return String.format(URL_PATTERN, hostName, port, database); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java new file mode 100644 index 00000000000..014467638f5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; + +import org.apache.seatunnel.api.configuration.Options; +import org.apache.seatunnel.api.configuration.SingleChoiceOption; +import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; +import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; +import org.apache.seatunnel.connectors.cdc.base.option.StopMode; + +import java.util.Arrays; + +public class PostgresSourceOptions { + public static final SingleChoiceOption STARTUP_MODE = + (SingleChoiceOption) + Options.key(SourceOptions.STARTUP_MODE_KEY) + .singleChoice( + StartupMode.class, + Arrays.asList( + StartupMode.INITIAL, + StartupMode.EARLIEST, + StartupMode.LATEST)) + .defaultValue(StartupMode.INITIAL) + .withDescription( + "Optional startup mode for CDC source, valid enumerations are " + + "\"initial\", \"earliest\", \"latest\", \"timestamp\"\n or \"specific\""); + + public static final SingleChoiceOption STOP_MODE = + (SingleChoiceOption) + Options.key("stop.mode") + .singleChoice(StopMode.class, Arrays.asList(StopMode.NEVER)) + .defaultValue(StopMode.NEVER) + .withDescription( + "Optional stop mode for CDC source, valid enumerations are " + + "\"never\", \"latest\", \"timestamp\"\n or \"specific\""); +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java new file mode 100644 index 00000000000..3e00c101c57 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.enumerator; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresTypeUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; + +import java.sql.SQLException; + +/** The {@code ChunkSplitter} used to split table into a set of chunks for JDBC data source. */ +@Slf4j +public class PostgresChunkSplitter extends AbstractJdbcSourceChunkSplitter { + + public PostgresChunkSplitter(JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dialect) { + super(sourceConfig, dialect); + } + + @Override + public Object[] queryMinMax(JdbcConnection jdbc, TableId tableId, String columnName) + throws SQLException { + return PostgresUtils.queryMinMax(jdbc, tableId, columnName); + } + + @Override + public Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException { + return PostgresUtils.queryMin(jdbc, tableId, columnName, excludedLowerBound); + } + + @Override + public Object[] sampleDataFromColumn( + JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) + throws SQLException { + return PostgresUtils.sampleDataFromColumn(jdbc, tableId, columnName, inverseSamplingRate); + } + + @Override + public Object queryNextChunkMax( + JdbcConnection jdbc, + TableId tableId, + String columnName, + int chunkSize, + Object includedLowerBound) + throws SQLException { + return PostgresUtils.queryNextChunkMax( + jdbc, tableId, columnName, chunkSize, includedLowerBound); + } + + @Override + public Long queryApproximateRowCnt(JdbcConnection jdbc, TableId tableId) throws SQLException { + return PostgresUtils.queryApproximateRowCnt(jdbc, tableId); + } + + @Override + public String buildSplitScanQuery( + TableId tableId, + SeaTunnelRowType splitKeyType, + boolean isFirstSplit, + boolean isLastSplit) { + return PostgresUtils.buildSplitScanQuery(tableId, splitKeyType, isFirstSplit, isLastSplit); + } + + @Override + public SeaTunnelDataType fromDbzColumn(Column splitColumn) { + return PostgresTypeUtils.convertFromColumn(splitColumn); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java new file mode 100644 index 00000000000..f9df4b8d878 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; + +import io.debezium.connector.postgresql.SourceInfo; +import io.debezium.connector.postgresql.connection.Lsn; + +import java.util.HashMap; +import java.util.Map; + +public class LsnOffset extends Offset { + + private static final long serialVersionUID = 1L; + + public static final LsnOffset INITIAL_OFFSET = new LsnOffset(Lsn.valueOf(Long.MIN_VALUE)); + public static final LsnOffset NO_STOPPING_OFFSET = new LsnOffset(Lsn.valueOf(Long.MAX_VALUE)); + + /** + * the position in the server WAL for a particular event; may be null indicating that this + * information is not available. + */ + private Lsn lsn; + + /** + * the ID of the transaction that generated the transaction; may be null if this information is + * not available. + */ + private Long txId; + + /** the xmin of the slot, may be null. */ + private Long xmin; + + public LsnOffset(Map offset) { + this.offset = offset; + } + + public LsnOffset(Lsn lsn) { + this(lsn, null, null); + } + + public LsnOffset(Lsn lsn, Long txId, Long xmin) { + Map offsetMap = new HashMap<>(); + + if (lsn != null && lsn.isValid()) { + offsetMap.put(SourceInfo.LSN_KEY, String.valueOf(lsn.asLong())); + } + if (txId != null) { + offsetMap.put(SourceInfo.TXID_KEY, String.valueOf(txId)); + } + if (xmin != null) { + offsetMap.put(SourceInfo.XMIN_KEY, String.valueOf(xmin)); + } + + this.offset = offsetMap; + } + + public Lsn getLsn() { + return Lsn.valueOf(Long.valueOf(offset.get(SourceInfo.LSN_KEY))); + } + + public Long getTxId() { + return Long.parseLong(offset.get(SourceInfo.TXID_KEY)); + } + + public Long getXmin() { + return Long.parseLong(offset.get(SourceInfo.XMIN_KEY)); + } + + @Override + public int compareTo(Offset o) { + LsnOffset that = (LsnOffset) o; + if (NO_STOPPING_OFFSET.equals(that) && NO_STOPPING_OFFSET.equals(this)) { + return 0; + } + if (NO_STOPPING_OFFSET.equals(this)) { + return 1; + } + if (NO_STOPPING_OFFSET.equals(that)) { + return -1; + } + + Lsn thisLsn = this.getLsn(); + Lsn thatLsn = that.getLsn(); + if (thatLsn.isValid()) { + if (thisLsn.isValid()) { + return thisLsn.compareTo(thatLsn); + } + return -1; + } else if (thisLsn.isValid()) { + return 1; + } + return 0; + } + + @SuppressWarnings("checkstyle:EqualsHashCode") + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof LsnOffset)) { + return false; + } + LsnOffset that = (LsnOffset) o; + return offset.equals(that.offset); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java new file mode 100644 index 00000000000..3633f41130a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset; + +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; + +import io.debezium.connector.postgresql.SourceInfo; +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConnection; + +import java.util.Map; + +public class LsnOffsetFactory extends OffsetFactory { + + private final PostgresSourceConfig sourceConfig; + + private final PostgresDialect dialect; + + public LsnOffsetFactory(PostgresSourceConfigFactory configFactory, PostgresDialect dialect) { + this.sourceConfig = configFactory.create(0); + this.dialect = dialect; + } + + @Override + public Offset earliest() { + return LsnOffset.INITIAL_OFFSET; + } + + @Override + public Offset neverStop() { + return LsnOffset.NO_STOPPING_OFFSET; + } + + @Override + public Offset latest() { + try (JdbcConnection jdbcConnection = dialect.openJdbcConnection(sourceConfig)) { + return PostgresUtils.currentLsn((PostgresConnection) jdbcConnection); + } catch (Exception e) { + throw new RuntimeException("Read the binlog offset error", e); + } + } + + @Override + public Offset specific(Map offset) { + return new LsnOffset(Lsn.valueOf(Long.valueOf(offset.get(SourceInfo.LSN_KEY)))); + } + + @Override + public Offset specific(String filename, Long position) { + throw new UnsupportedOperationException( + "not supported create new Offset by filename and position."); + } + + @Override + public Offset timestamp(long timestamp) { + throw new UnsupportedOperationException("not supported create new Offset by timestamp."); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java new file mode 100644 index 00000000000..b297db40f76 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java @@ -0,0 +1,364 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader; + +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; +import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; +import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.JdbcSourceFetchTaskContext; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.debezium.EmbeddedDatabaseHistory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; + +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.SourceRecord; + +import io.debezium.DebeziumException; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresErrorHandler; +import io.debezium.connector.postgresql.PostgresEventMetadataProvider; +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.PostgresValueConverter; +import io.debezium.connector.postgresql.TypeRegistry; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.connector.postgresql.connection.ReplicationConnection; +import io.debezium.connector.postgresql.spi.SlotState; +import io.debezium.connector.postgresql.spi.Snapshotter; +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.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.relational.history.TableChanges; +import io.debezium.schema.TopicSelector; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import lombok.Getter; +import lombok.extern.slf4j.Slf4j; + +import java.sql.SQLException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresConnectionUtils.createPostgresConnection; + +@Slf4j +public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { + + private static final String CONTEXT_NAME = "postgres-cdc-connector-task"; + + private final PostgresConnection dataConnection; + + @Getter private ReplicationConnection replicationConnection; + + private final PostgresEventMetadataProvider metadataProvider; + + @Getter private Snapshotter snapshotter; + private PostgresSchema databaseSchema; + private PostgresOffsetContext offsetContext; + private TopicSelector topicSelector; + private JdbcSourceEventDispatcher dispatcher; + private ChangeEventQueue queue; + private PostgresErrorHandler errorHandler; + + @Getter private PostgresTaskContext taskContext; + + private SnapshotChangeEventSourceMetrics snapshotChangeEventSourceMetrics; + + public PostgresSourceFetchTaskContext( + JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dataSourceDialect) { + super(sourceConfig, dataSourceDialect); + this.dataConnection = + createPostgresConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); + this.metadataProvider = new PostgresEventMetadataProvider(); + } + + @Override + public void configure(SourceSplitBase sourceSplitBase) { + registerDatabaseHistory(sourceSplitBase); + + // initial stateful objects + final PostgresConnectorConfig connectorConfig = getDbzConnectorConfig(); + this.snapshotter = connectorConfig.getSnapshotter(); + + this.topicSelector = PostgresTopicSelector.create(connectorConfig); + + final PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = + typeRegistry -> + PostgresValueConverter.of( + connectorConfig, dataConnection.getDatabaseCharset(), typeRegistry); + final TypeRegistry typeRegistry = dataConnection.getTypeRegistry(); + + this.databaseSchema = + new PostgresSchema( + connectorConfig, + typeRegistry, + topicSelector, + valueConverterBuilder.build(typeRegistry)); + this.taskContext = new PostgresTaskContext(connectorConfig, databaseSchema, topicSelector); + try { + taskContext.refreshSchema(dataConnection, false); + } catch (SQLException e) { + throw new DebeziumException("load schema failed", e); + } + this.offsetContext = + loadStartingOffsetState( + new PostgresOffsetContext.Loader(connectorConfig), sourceSplitBase); + + final int queueSize = + sourceSplitBase.isSnapshotSplit() + ? Integer.MAX_VALUE + : getSourceConfig().getDbzConnectorConfig().getMaxQueueSize(); + + // Print out the server information + SlotState slotInfo = null; + try { + if (log.isInfoEnabled()) { + log.info(dataConnection.serverInfo().toString()); + } + slotInfo = + dataConnection.getReplicationSlotState( + connectorConfig.slotName(), + connectorConfig.plugin().getPostgresPluginName()); + } catch (SQLException e) { + log.warn( + "unable to load info of replication slot, Debezium will try to create the slot"); + } + + if (offsetContext == null) { + log.info("No previous offset found"); + // if we have no initial offset, indicate that to Snapshotter by passing null + snapshotter.init(connectorConfig, null, slotInfo); + } else { + log.info("Found previous offset {}", offsetContext); + snapshotter.init(connectorConfig, offsetContext.asOffsetState(), slotInfo); + } + + if (snapshotter.shouldStream()) { + final boolean doSnapshot = snapshotter.shouldSnapshot(); + this.replicationConnection = + createReplicationConnection( + this.taskContext, + doSnapshot, + connectorConfig.maxRetries(), + connectorConfig.retryDelay()); + + // we need to create the slot before we start streaming if it doesn't exist + // otherwise we can't stream back changes happening while the snapshot is taking + // place + if (slotInfo == null) { + try { + replicationConnection.createReplicationSlot().orElse(null); + } catch (SQLException ex) { + String message = "Creation of replication slot failed"; + if (ex.getMessage().contains("already exists")) { + message += + "; when setting up multiple connectors for the same database host, please make sure to use a distinct replication slot name for each."; + } + throw new DebeziumException(message, ex); + } + } + } + + try { + dataConnection.commit(); + } catch (SQLException e) { + throw new DebeziumException(e); + } + + this.queue = + new ChangeEventQueue.Builder() + .pollInterval(connectorConfig.getPollInterval()) + .maxBatchSize(connectorConfig.getMaxBatchSize()) + .maxQueueSize(queueSize) + .maxQueueSizeInBytes(connectorConfig.getMaxQueueSizeInBytes()) + .loggingContextSupplier( + () -> taskContext.configureLoggingContext(CONTEXT_NAME)) + // do not buffer any element, we use signal event + // .buffering() + .build(); + + this.dispatcher = + new JdbcSourceEventDispatcher( + connectorConfig, + topicSelector, + databaseSchema, + queue, + connectorConfig.getTableFilters().dataCollectionFilter(), + DataChangeEvent::new, + metadataProvider, + schemaNameAdjuster); + + this.snapshotChangeEventSourceMetrics = + new DefaultChangeEventSourceMetricsFactory() + .getSnapshotMetrics(taskContext, queue, metadataProvider); + + this.errorHandler = new PostgresErrorHandler(connectorConfig.getLogicalName(), queue); + } + + private void registerDatabaseHistory(SourceSplitBase sourceSplitBase) { + List engineHistory = new ArrayList<>(); + // TODO: support save table schema + if (sourceSplitBase instanceof SnapshotSplit) { + SnapshotSplit snapshotSplit = (SnapshotSplit) sourceSplitBase; + engineHistory.add( + dataSourceDialect.queryTableSchema(dataConnection, snapshotSplit.getTableId())); + } else { + IncrementalSplit incrementalSplit = (IncrementalSplit) sourceSplitBase; + for (TableId tableId : incrementalSplit.getTableIds()) { + engineHistory.add(dataSourceDialect.queryTableSchema(dataConnection, tableId)); + } + } + + EmbeddedDatabaseHistory.registerHistory( + sourceConfig + .getDbzConfiguration() + .getString(EmbeddedDatabaseHistory.DATABASE_HISTORY_INSTANCE_NAME), + engineHistory); + } + + @Override + public PostgresSourceConfig getSourceConfig() { + return (PostgresSourceConfig) sourceConfig; + } + + public PostgresConnection getDataConnection() { + return dataConnection; + } + + public SnapshotChangeEventSourceMetrics getSnapshotChangeEventSourceMetrics() { + return snapshotChangeEventSourceMetrics; + } + + @Override + public PostgresConnectorConfig getDbzConnectorConfig() { + return (PostgresConnectorConfig) super.getDbzConnectorConfig(); + } + + @Override + public PostgresOffsetContext getOffsetContext() { + return offsetContext; + } + + @Override + public ErrorHandler getErrorHandler() { + return errorHandler; + } + + @Override + public PostgresSchema getDatabaseSchema() { + return databaseSchema; + } + + @Override + public SeaTunnelRowType getSplitType(Table table) { + return PostgresUtils.getSplitType(table); + } + + @Override + public JdbcSourceEventDispatcher getDispatcher() { + return dispatcher; + } + + @Override + public ChangeEventQueue getQueue() { + return queue; + } + + @Override + public Tables.TableFilter getTableFilter() { + return getDbzConnectorConfig().getTableFilters().dataCollectionFilter(); + } + + @Override + public Offset getStreamOffset(SourceRecord sourceRecord) { + return PostgresUtils.getLsnPosition(sourceRecord); + } + + @Override + public void close() { + try { + this.dataConnection.close(); + this.replicationConnection.close(); + } catch (Exception e) { + log.warn("Failed to close connection", e); + } + } + + /** Loads the connector's persistent offset (if present) via the given loader. */ + private PostgresOffsetContext loadStartingOffsetState( + PostgresOffsetContext.Loader loader, SourceSplitBase split) { + Offset offset = + split.isSnapshotSplit() + ? LsnOffset.INITIAL_OFFSET + : split.asIncrementalSplit().getStartupOffset(); + return loader.load(offset.getOffset()); + } + + public ReplicationConnection createReplicationConnection( + PostgresTaskContext taskContext, + boolean doSnapshot, + int maxRetries, + Duration retryDelay) + throws ConnectException { + final Metronome metronome = Metronome.parker(retryDelay, Clock.SYSTEM); + short retryCount = 0; + ReplicationConnection replicationConnection = null; + while (retryCount <= maxRetries) { + try { + return taskContext.createReplicationConnection(doSnapshot); + } catch (SQLException ex) { + retryCount++; + if (retryCount > maxRetries) { + log.error( + "Too many errors connecting to server. All {} retries failed.", + maxRetries); + throw new ConnectException(ex); + } + + log.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) { + log.warn("Connection retry sleep interrupted by exception: " + e); + Thread.currentThread().interrupt(); + } + } + } + return replicationConnection; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java new file mode 100644 index 00000000000..0a1d8fa851b --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; + +import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; + +import io.debezium.pipeline.spi.SnapshotResult; +import lombok.extern.slf4j.Slf4j; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; + +@Slf4j +public class PostgresSnapshotFetchTask implements FetchTask { + + private final SnapshotSplit split; + + private volatile boolean taskRunning = false; + + private PostgresSnapshotSplitReadTask snapshotSplitReadTask; + + public PostgresSnapshotFetchTask(SnapshotSplit split) { + this.split = split; + } + + @Override + public void execute(FetchTask.Context context) throws Exception { + PostgresSourceFetchTaskContext sourceFetchContext = + (PostgresSourceFetchTaskContext) context; + taskRunning = true; + snapshotSplitReadTask = + new PostgresSnapshotSplitReadTask( + sourceFetchContext.getDbzConnectorConfig(), + sourceFetchContext.getOffsetContext(), + sourceFetchContext.getSnapshotChangeEventSourceMetrics(), + sourceFetchContext.getDatabaseSchema(), + sourceFetchContext.getDataConnection(), + sourceFetchContext.getDispatcher(), + split); + SnapshotSplitChangeEventSourceContext changeEventSourceContext = + new SnapshotSplitChangeEventSourceContext(); + SnapshotResult snapshotResult = + snapshotSplitReadTask.execute( + changeEventSourceContext, sourceFetchContext.getOffsetContext()); + if (!snapshotResult.isCompletedOrSkipped()) { + taskRunning = false; + throw new IllegalStateException( + String.format("Read snapshot for split %s fail", split)); + } + + boolean changed = + changeEventSourceContext + .getHighWatermark() + .isAfter(changeEventSourceContext.getLowWatermark()); + if (!context.isExactlyOnce()) { + taskRunning = false; + if (changed) { + log.debug("Skip merge changelog(exactly-once) for snapshot split {}", split); + } + return; + } + final IncrementalSplit backfillBinlogSplit = + createBackFillWalSplit(changeEventSourceContext); + + final IncrementalSplit backfillSplit = createBackFillWalSplit(changeEventSourceContext); + // optimization that skip the binlog read when the low watermark equals high + // watermark + if (!changed) { + dispatchBinlogEndEvent( + backfillSplit, + ((PostgresSourceFetchTaskContext) context).getOffsetContext().getPartition(), + ((PostgresSourceFetchTaskContext) context).getDispatcher()); + taskRunning = false; + return; + } + taskRunning = false; + } + + private IncrementalSplit createBackFillWalSplit( + SnapshotSplitChangeEventSourceContext sourceContext) { + return new IncrementalSplit( + split.splitId(), + Collections.singletonList(split.getTableId()), + sourceContext.getLowWatermark(), + sourceContext.getHighWatermark(), + new ArrayList<>()); + } + + private void dispatchBinlogEndEvent( + IncrementalSplit backFillBinlogSplit, + Map sourcePartition, + JdbcSourceEventDispatcher eventDispatcher) + throws InterruptedException { + eventDispatcher.dispatchWatermarkEvent( + sourcePartition, + backFillBinlogSplit, + backFillBinlogSplit.getStopOffset(), + WatermarkKind.END); + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() { + taskRunning = false; + } + + @Override + public SourceSplitBase getSplit() { + return split; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java new file mode 100644 index 00000000000..ac40981d5a7 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; + +import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; + +import org.apache.kafka.connect.errors.ConnectException; + +import io.debezium.DebeziumException; +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.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.ChangeRecordEmitter; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.spi.SnapshotResult; +import io.debezium.relational.Column; +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 lombok.extern.slf4j.Slf4j; + +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.sql.Types; +import java.time.Duration; + +@Slf4j +public class PostgresSnapshotSplitReadTask extends AbstractSnapshotChangeEventSource { + + /** Interval for showing a log statement with the progress while scanning a single table. */ + private static final Duration LOG_INTERVAL = Duration.ofMillis(10_000); + + private final PostgresConnectorConfig connectorConfig; + private final PostgresSchema databaseSchema; + private final PostgresConnection jdbcConnection; + private final JdbcSourceEventDispatcher dispatcher; + private final Clock clock; + private final SnapshotSplit snapshotSplit; + private final PostgresOffsetContext offsetContext; + private final SnapshotProgressListener snapshotProgressListener; + + public PostgresSnapshotSplitReadTask( + PostgresConnectorConfig connectorConfig, + PostgresOffsetContext previousOffset, + SnapshotProgressListener snapshotProgressListener, + PostgresSchema databaseSchema, + PostgresConnection jdbcConnection, + JdbcSourceEventDispatcher dispatcher, + SnapshotSplit snapshotSplit) { + super(connectorConfig, snapshotProgressListener); + this.offsetContext = previousOffset; + this.connectorConfig = connectorConfig; + this.databaseSchema = databaseSchema; + this.jdbcConnection = jdbcConnection; + this.dispatcher = dispatcher; + this.clock = Clock.SYSTEM; + this.snapshotSplit = snapshotSplit; + this.snapshotProgressListener = snapshotProgressListener; + } + + @Override + public SnapshotResult execute( + ChangeEventSource.ChangeEventSourceContext context, OffsetContext previousOffset) + throws InterruptedException { + SnapshottingTask snapshottingTask = getSnapshottingTask(previousOffset); + final SnapshotContext ctx; + try { + ctx = prepare(context); + } catch (Exception e) { + log.error("Failed to initialize snapshot context.", e); + throw new RuntimeException(e); + } + try { + return doExecute(context, previousOffset, ctx, snapshottingTask); + } catch (InterruptedException e) { + log.warn("Snapshot was interrupted before completion"); + throw e; + } catch (Exception t) { + throw new DebeziumException(t); + } + } + + @Override + protected SnapshotResult doExecute( + ChangeEventSource.ChangeEventSourceContext context, + OffsetContext previousOffset, + AbstractSnapshotChangeEventSource.SnapshotContext snapshotContext, + AbstractSnapshotChangeEventSource.SnapshottingTask snapshottingTask) + throws Exception { + final PostgreSqlSnapshotContext ctx = (PostgreSqlSnapshotContext) snapshotContext; + ctx.offset = offsetContext; + + final LsnOffset lowWatermark = PostgresUtils.currentLsn(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 LsnOffset highWatermark = PostgresUtils.currentLsn(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); + } + + @Override + protected AbstractSnapshotChangeEventSource.SnapshottingTask getSnapshottingTask( + OffsetContext previousOffset) { + return new SnapshottingTask(false, true); + } + + @Override + protected AbstractSnapshotChangeEventSource.SnapshotContext prepare( + ChangeEventSource.ChangeEventSourceContext changeEventSourceContext) throws Exception { + return new PostgreSqlSnapshotContext(); + } + + private void createDataEvents(PostgreSqlSnapshotContext snapshotContext, TableId tableId) + throws Exception { + EventDispatcher.SnapshotReceiver snapshotReceiver = + dispatcher.getSnapshotChangeEventReceiver(); + log.debug("Snapshotting table {}", tableId); + // todo pg 的 schema 不包含database + TableId newTableId = new TableId(null, tableId.schema(), tableId.table()); + createDataEventsForTable( + snapshotContext, snapshotReceiver, databaseSchema.tableFor(newTableId)); + snapshotReceiver.completeSnapshot(); + } + + /** Dispatches the data change events for the records of a single table. */ + private void createDataEventsForTable( + PostgreSqlSnapshotContext 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 = + PostgresUtils.buildSplitScanQuery( + snapshotSplit.getTableId(), + snapshotSplit.getSplitKeyType(), + snapshotSplit.getSplitStart() == null, + snapshotSplit.getSplitEnd() == null); + log.info( + "For split '{}' of table {} using select statement: '{}'", + snapshotSplit.splitId(), + table.id(), + selectSql); + + try (PreparedStatement selectStatement = + PostgresUtils.readTableSplitDataStatement( + jdbcConnection, + selectSql, + snapshotSplit.getSplitStart() == null, + snapshotSplit.getSplitEnd() == null, + snapshotSplit.getSplitStart(), + snapshotSplit.getSplitEnd(), + snapshotSplit.getSplitKeyType().getTotalFields(), + 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++) { + Column actualColumn = table.columns().get(i); + row[columnArray.getColumns()[i].position() - 1] = readField(rs, 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(); + } + dispatcher.dispatchSnapshotEvent( + table.id(), + getChangeRecordEmitter(snapshotContext, table.id(), row), + snapshotReceiver); + } + log.info( + "Finished exporting {} records for split '{}', total duration '{}'", + rows, + snapshotSplit.splitId(), + Strings.duration(clock.currentTimeInMillis() - exportStart)); + } catch (SQLException e) { + throw new ConnectException("Snapshotting of table " + table.id() + " failed", e); + } + } + + protected ChangeRecordEmitter getChangeRecordEmitter( + PostgreSqlSnapshotContext snapshotContext, TableId tableId, Object[] row) { + snapshotContext.offset.event(tableId, clock.currentTime()); + return new SnapshotChangeRecordEmitter(snapshotContext.offset, row, clock); + } + + private Threads.Timer getTableScanLogTimer() { + return Threads.timer(clock, LOG_INTERVAL); + } + + private Object readField(ResultSet rs, int columnIndex) throws SQLException { + final ResultSetMetaData metaData = rs.getMetaData(); + final int columnType = metaData.getColumnType(columnIndex); + + if (columnType == Types.TIME) { + return rs.getTimestamp(columnIndex); + } else { + return rs.getObject(columnIndex); + } + } + + private static class PostgreSqlSnapshotContext + extends RelationalSnapshotChangeEventSource.RelationalSnapshotContext { + + public PostgreSqlSnapshotContext() throws SQLException { + super(""); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java new file mode 100644 index 00000000000..9aaa3f2ded5 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; + +import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; + +import io.debezium.pipeline.source.spi.ChangeEventSource; + +/** + * {@link ChangeEventSource.ChangeEventSourceContext} implementation that keeps low/high watermark + * for each {@link SnapshotSplit}. + */ +public class SnapshotSplitChangeEventSourceContext + implements ChangeEventSource.ChangeEventSourceContext { + + private LsnOffset lowWatermark; + private LsnOffset highWatermark; + + public LsnOffset getLowWatermark() { + return lowWatermark; + } + + public void setLowWatermark(LsnOffset lowWatermark) { + this.lowWatermark = lowWatermark; + } + + public LsnOffset getHighWatermark() { + return highWatermark; + } + + public void setHighWatermark(LsnOffset highWatermark) { + this.highWatermark = highWatermark; + } + + @Override + public boolean isRunning() { + return lowWatermark != null && highWatermark != null; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java new file mode 100644 index 00000000000..340eaf5311a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.wal; + +import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; +import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; +import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; + +import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource; +import io.debezium.pipeline.source.spi.ChangeEventSource; +import io.debezium.util.Clock; + +public class PostgresWalFetchTask implements FetchTask { + private final IncrementalSplit split; + private volatile boolean taskRunning = false; + + public PostgresWalFetchTask(IncrementalSplit split) { + this.split = split; + } + + @Override + public void execute(FetchTask.Context context) throws Exception { + PostgresSourceFetchTaskContext sourceFetchContext = + (PostgresSourceFetchTaskContext) context; + taskRunning = true; + + PostgresStreamingChangeEventSource streamingChangeEventSource = + new PostgresStreamingChangeEventSource( + sourceFetchContext.getDbzConnectorConfig(), + sourceFetchContext.getSnapshotter(), + sourceFetchContext.getDataConnection(), + sourceFetchContext.getDispatcher(), + sourceFetchContext.getErrorHandler(), + Clock.SYSTEM, + sourceFetchContext.getDatabaseSchema(), + sourceFetchContext.getTaskContext(), + sourceFetchContext.getReplicationConnection()); + + WalSplitChangeEventSourceContext changeEventSourceContext = + new WalSplitChangeEventSourceContext(); + + streamingChangeEventSource.execute( + changeEventSourceContext, sourceFetchContext.getOffsetContext()); + } + + @Override + public boolean isRunning() { + return taskRunning; + } + + @Override + public void shutdown() { + taskRunning = false; + } + + @Override + public SourceSplitBase getSplit() { + return split; + } + + private class WalSplitChangeEventSourceContext + implements ChangeEventSource.ChangeEventSourceContext { + @Override + public boolean isRunning() { + return taskRunning; + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java new file mode 100644 index 00000000000..304e8835c76 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; + +import io.debezium.config.Configuration; +import io.debezium.connector.postgresql.PostgresConnectorConfig; +import io.debezium.connector.postgresql.PostgresValueConverter; +import io.debezium.connector.postgresql.connection.PostgresConnection; + +import java.nio.charset.Charset; + +/** Utils for Postgres connection. */ +public class PostgresConnectionUtils { + + public static PostgresConnection createPostgresConnection(Configuration dbzConfiguration) { + + PostgresConnection heartbeatConnection = new PostgresConnection(dbzConfiguration); + final Charset databaseCharset = heartbeatConnection.getDatabaseCharset(); + + final PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = + (typeRegistry) -> + PostgresValueConverter.of( + new PostgresConnectorConfig(dbzConfiguration), + databaseCharset, + typeRegistry); + + return new PostgresConnection(dbzConfiguration, valueConverterBuilder); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java new file mode 100644 index 00000000000..cb0910a4782 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java @@ -0,0 +1,86 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; + +import org.apache.seatunnel.common.utils.SeaTunnelException; + +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables; +import io.debezium.relational.history.TableChanges; + +import java.sql.SQLException; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +public class PostgresSchema { + + private final Map schemasByTableId; + + public PostgresSchema() { + this.schemasByTableId = new ConcurrentHashMap<>(); + } + + public TableChanges.TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { + // read schema from cache first + TableChanges.TableChange schema = schemasByTableId.get(tableId); + if (schema == null) { + schema = readTableSchema(jdbc, tableId); + schemasByTableId.put(tableId, schema); + } + return schema; + } + + private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) { + + // Because the catalog is null in the postgresConnection.readSchema method + tableId = new TableId(null, tableId.schema(), tableId.table()); + + PostgresConnection postgresConnection = (PostgresConnection) jdbc; + Set tableIdSet = new HashSet<>(); + tableIdSet.add(tableId); + + final Map tableChangeMap = new HashMap<>(); + Tables tables = new Tables(); + tables.overwriteTable(tables.editOrCreateTable(tableId).create()); + + try { + postgresConnection.readSchema( + tables, tableId.catalog(), tableId.schema(), null, null, false); + Table table = tables.forTable(tableId); + TableChanges.TableChange tableChange = + new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table); + tableChangeMap.put(tableId, tableChange); + } catch (SQLException e) { + throw new SeaTunnelException( + String.format("Failed to read schema for table %s ", tableId), e); + } + + if (!tableChangeMap.containsKey(tableId)) { + throw new SeaTunnelException( + String.format("Can't obtain schema for table %s ", tableId)); + } + + return tableChangeMap.get(tableId); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java new file mode 100644 index 00000000000..905423470b1 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; + +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; + +import io.debezium.relational.Column; +import io.debezium.relational.Table; + +import java.sql.Types; +import java.util.List; + +public class PostgresTypeUtils { + private PostgresTypeUtils() {} + + public static SeaTunnelDataType convertFromColumn(Column column) { + switch (column.jdbcType()) { + case Types.CHAR: + case Types.VARCHAR: + case Types.NCHAR: + case Types.NVARCHAR: + case Types.STRUCT: + case Types.CLOB: + case Types.LONGVARCHAR: + case Types.LONGNVARCHAR: + return BasicType.STRING_TYPE; + case Types.BLOB: + return PrimitiveByteArrayType.INSTANCE; + case Types.INTEGER: + return BasicType.INT_TYPE; + case Types.SMALLINT: + case Types.TINYINT: + return BasicType.SHORT_TYPE; + case Types.BIGINT: + return BasicType.LONG_TYPE; + case Types.FLOAT: + case Types.REAL: + return BasicType.FLOAT_TYPE; + case Types.DOUBLE: + return BasicType.DOUBLE_TYPE; + case Types.NUMERIC: + case Types.DECIMAL: + return new DecimalType(column.length(), column.scale().orElse(0)); + case Types.TIMESTAMP: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case Types.DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + case Types.TIME: + return LocalTimeType.LOCAL_TIME_TYPE; + case Types.BOOLEAN: + case Types.BIT: + return BasicType.BOOLEAN_TYPE; + default: + throw new UnsupportedOperationException( + String.format( + "Don't support Postgres type '%s' yet, jdbcType:'%s'.", + column.typeName(), column.jdbcType())); + } + } + + public static SeaTunnelRowType convertFromTable(Table table) { + + List columns = table.columns(); + String[] fieldNames = columns.stream().map(Column::name).toArray(String[]::new); + + SeaTunnelDataType[] fieldTypes = + columns.stream() + .map(PostgresTypeUtils::convertFromColumn) + .toArray(SeaTunnelDataType[]::new); + + return new SeaTunnelRowType(fieldNames, fieldTypes); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java new file mode 100644 index 00000000000..4f57d717679 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java @@ -0,0 +1,419 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; + +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; +import org.apache.seatunnel.api.table.type.SeaTunnelRowType; +import org.apache.seatunnel.common.utils.SeaTunnelException; +import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; +import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; + +import org.apache.kafka.connect.source.SourceRecord; + +import io.debezium.connector.postgresql.connection.Lsn; +import io.debezium.connector.postgresql.connection.PostgresConnection; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; + +import java.sql.Connection; +import java.sql.PreparedStatement; +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.Optional; + +/** The utils for Postgres data source. */ +public class PostgresUtils { + private PostgresUtils() {} + + 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 SourceRecordUtils.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. + final String rowCountQuery = + String.format( + "SELECT reltuples FROM pg_class r WHERE relkind = 'r' AND relname = '%s';", + tableId.table()); + return jdbc.queryAndMap( + rowCountQuery, + rs -> { + if (!rs.next()) { + throw new SQLException( + String.format( + "No result returned after running query [%s]", + rowCountQuery)); + } + return rs.getLong(1); + }); + } + + public static Object queryMin( + JdbcConnection jdbc, TableId tableId, String columnName, Object excludedLowerBound) + throws SQLException { + final String minQuery = + String.format( + "SELECT MIN(%s) FROM %s WHERE %s > ?", + quote(columnName), quote(tableId), quote(columnName)); + return jdbc.prepareQueryAndMap( + minQuery, + 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]", minQuery)); + } + return rs.getObject(1); + }); + } + + public static Object[] sampleDataFromColumn( + JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) + throws SQLException { + final String minQuery = + String.format( + "SELECT %s FROM %s WHERE MOD((%s - (SELECT MIN(%s) FROM %s)), %s) = 0 ORDER BY %s", + quote(columnName), + quote(tableId), + quote(columnName), + quote(columnName), + quote(tableId), + inverseSamplingRate, + quote(columnName)); + return jdbc.queryAndMap( + minQuery, + resultSet -> { + List results = new ArrayList<>(); + while (resultSet.next()) { + results.add(resultSet.getObject(1)); + } + return results.toArray(); + }); + } + + /** + * Returns the next LSN to be read from the database. This is the LSN of the last record that + * was read from the database. + */ + 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 SeaTunnelRowType getSplitType(Table table) { + List primaryKeys = table.primaryKeyColumns(); + if (primaryKeys.isEmpty()) { + throw new SeaTunnelException( + String.format( + "Incremental snapshot for tables requires primary key," + + " but table %s doesn't have primary key.", + table.id())); + } + + // use first field in primary key as the split key + return getSplitType(primaryKeys.get(0)); + } + + public static SeaTunnelRowType getSplitType(Column splitColumn) { + return new SeaTunnelRowType( + new String[] {splitColumn.name()}, + new SeaTunnelDataType[] {PostgresTypeUtils.convertFromColumn(splitColumn)}); + } + + public static Offset getLsnPosition(SourceRecord record) { + return getLsnPosition(record.sourceOffset()); + } + + public static LsnOffset getLsnPosition(Map offset) { + Map offsetStrMap = new HashMap<>(); + for (Map.Entry entry : offset.entrySet()) { + offsetStrMap.put( + entry.getKey(), entry.getValue() == null ? null : entry.getValue().toString()); + } + return new LsnOffset(offsetStrMap); + } + + /** Fetch current largest log sequence number (LSN) of the database. */ + public static LsnOffset currentLsn(PostgresConnection connection) { + try { + final Lsn lsn = Lsn.valueOf(connection.currentXLogLocation()); + final long txId = connection.currentTransactionId().longValue(); + return new LsnOffset(lsn, txId, null); + } catch (SQLException e) { + throw new SeaTunnelException(e.getMessage(), e); + } + } + + /** Get split scan query for the given table. */ + public static String buildSplitScanQuery( + TableId tableId, SeaTunnelRowType rowType, boolean isFirstSplit, boolean isLastSplit) { + return buildSplitQuery(tableId, rowType, isFirstSplit, isLastSplit, -1, true); + } + + /** Get table split data PreparedStatement. */ + 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); + } + } + + private static String getPrimaryKeyColumnsProjection(SeaTunnelRowType rowType) { + StringBuilder sql = new StringBuilder(); + for (Iterator fieldNamesIt = Arrays.stream(rowType.getFieldNames()).iterator(); + fieldNamesIt.hasNext(); ) { + sql.append(fieldNamesIt.next()); + if (fieldNamesIt.hasNext()) { + sql.append(" , "); + } + } + return sql.toString(); + } + + private static String buildSplitQuery( + TableId tableId, + SeaTunnelRowType rowType, + 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(rowType, sql, " <= ?"); + if (isScanningData) { + sql.append(" AND NOT ("); + addPrimaryKeyColumnsToCondition(rowType, sql, " = ?"); + sql.append(")"); + } + condition = sql.toString(); + } else if (isLastSplit) { + final StringBuilder sql = new StringBuilder(); + addPrimaryKeyColumnsToCondition(rowType, sql, " >= ?"); + condition = sql.toString(); + } else { + final StringBuilder sql = new StringBuilder(); + addPrimaryKeyColumnsToCondition(rowType, sql, " >= ?"); + if (isScanningData) { + sql.append(" AND NOT ("); + addPrimaryKeyColumnsToCondition(rowType, sql, " = ?"); + sql.append(")"); + } + sql.append(" AND "); + addPrimaryKeyColumnsToCondition(rowType, sql, " <= ?"); + condition = sql.toString(); + } + + if (isScanningData) { + return buildSelectWithRowLimits( + tableId, limitSize, "*", Optional.ofNullable(condition), Optional.empty()); + } else { + final String orderBy = String.join(", ", rowType.getFieldNames()); + return buildSelectWithBoundaryRowLimits( + tableId, + limitSize, + getPrimaryKeyColumnsProjection(rowType), + getMaxPrimaryKeyColumnsProjection(rowType), + Optional.ofNullable(condition), + orderBy); + } + } + + 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 String getMaxPrimaryKeyColumnsProjection(SeaTunnelRowType rowType) { + StringBuilder sql = new StringBuilder(); + for (Iterator fieldNamesIt = Arrays.stream(rowType.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 "); + if (limit > 0) { + sql.append(" TOP( ").append(limit).append(") "); + } + sql.append(projection).append(" FROM "); + sql.append(quoteSchemaAndTable(tableId)); + if (condition.isPresent()) { + sql.append(" WHERE ").append(condition.get()); + } + if (orderBy.isPresent()) { + sql.append(" ORDER BY ").append(orderBy.get()); + } + return sql.toString(); + } + + private static String quoteSchemaAndTable(TableId tableId) { + StringBuilder quoted = new StringBuilder(); + + if (tableId.schema() != null && !tableId.schema().isEmpty()) { + quoted.append(quote(tableId.schema())).append("."); + } + + quoted.append(quote(tableId.table())); + return quoted.toString(); + } + + public static String quote(String dbOrTableName) { + return "\"" + dbOrTableName + "\""; + } + + public static String quote(TableId tableId) { + return "\"" + tableId.schema() + "\".\"" + tableId.table() + "\""; + } + + private static void addPrimaryKeyColumnsToCondition( + SeaTunnelRowType rowType, StringBuilder sql, String predicate) { + for (Iterator fieldNamesIt = Arrays.stream(rowType.getFieldNames()).iterator(); + fieldNamesIt.hasNext(); ) { + sql.append(fieldNamesIt.next()).append(predicate); + if (fieldNamesIt.hasNext()) { + sql.append(" AND "); + } + } + } + + 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(" TOP( ").append(limit).append(") "); + sql.append(projection); + sql.append(" FROM "); + sql.append(quoteSchemaAndTable(tableId)); + if (condition.isPresent()) { + sql.append(" WHERE ").append(condition.get()); + } + sql.append(" ORDER BY ").append(orderBy); + sql.append(") T"); + return sql.toString(); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java new file mode 100644 index 00000000000..5528c09bf8b --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.RelationalTableFilters; +import io.debezium.relational.TableId; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +public class TableDiscoveryUtils { + private static final Logger LOG = LoggerFactory.getLogger(TableDiscoveryUtils.class); + + @SuppressWarnings("MagicNumber") + public static List listTables(JdbcConnection jdbc, RelationalTableFilters tableFilters) + throws SQLException { + final List capturedTableIds = new ArrayList<>(); + // ------------------- + // READ DATABASE NAMES + // ------------------- + // Get the list of databases ... + LOG.info("Read list of available databases"); + final List databaseNames = new ArrayList<>(); + + jdbc.query( + "select datname from pg_database", + rs -> { + while (rs.next()) { + databaseNames.add(rs.getString(1)); + } + }); + LOG.info("\t list of available databases is: {}", databaseNames); + + // ---------------- + // READ TABLE NAMES + // ---------------- + // Get the list of table IDs for each database. We can't use a prepared statement with + // SqlServer, so we have to build the SQL statement each time. Although in other cases this + // might lead to SQL injection, in our case we are reading the database names from the + // database and not taking them from the user ... + LOG.info("Read list of available tables in each database"); + for (String dbName : databaseNames) { + try { + jdbc.query( + "SELECT * FROM \"" + + dbName + + "\".INFORMATION_SCHEMA.TABLES WHERE TABLE_TYPE = 'BASE TABLE';", + rs -> { + while (rs.next()) { + TableId tableId = + new TableId( + rs.getString(1), rs.getString(2), rs.getString(3)); + if (tableFilters.dataCollectionFilter().isIncluded(tableId)) { + capturedTableIds.add(tableId); + LOG.info("\t including '{}' for further processing", tableId); + } else { + LOG.info("\t '{}' is filtered out of capturing", tableId); + } + } + }); + } catch (SQLException e) { + // We were unable to execute the query or process the results, so skip this ... + LOG.warn( + "\t skipping database '{}' due to error reading tables: {}", + dbName, + e.getMessage()); + } + } + return capturedTableIds; + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java new file mode 100644 index 00000000000..931dd10a06a --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source; + +import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresIncrementalSourceFactory; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class PostgresIncrementalSourceFactoryTest { + @Test + public void testOptionRule() { + Assertions.assertNotNull(new PostgresIncrementalSourceFactory().optionRule()); + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index 3519b1c51c0..c54d1bdc066 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -34,6 +34,7 @@ connector-cdc-mysql connector-cdc-sqlserver connector-cdc-mongodb + connector-cdc-postgres diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml new file mode 100644 index 00000000000..77497df8577 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml @@ -0,0 +1,60 @@ + + + + 4.0.0 + + org.apache.seatunnel + seatunnel-connector-v2-e2e + ${revision} + + + connector-cdc-postgres-e2e + SeaTunnel : E2E : Connector V2 : CDC Postgres + + + + + org.apache.seatunnel + connector-cdc-postgres + ${project.version} + pom + import + + + + + + + + org.apache.seatunnel + connector-cdc-postgres + ${project.version} + test + + + org.testcontainers + postgresql + ${testcontainer.version} + + + org.apache.seatunnel + connector-jdbc + ${project.version} + test + + + diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java new file mode 100644 index 00000000000..eb172569d1c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java @@ -0,0 +1,315 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.e2e.connector.cdc.postgres; + +import org.apache.seatunnel.e2e.common.TestResource; +import org.apache.seatunnel.e2e.common.TestSuiteBase; +import org.apache.seatunnel.e2e.common.container.EngineType; +import org.apache.seatunnel.e2e.common.container.TestContainer; +import org.apache.seatunnel.e2e.common.junit.DisabledOnContainer; + +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.lifecycle.Startables; +import org.testcontainers.utility.MountableFile; + +import lombok.extern.slf4j.Slf4j; + +import java.io.IOException; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; +import java.util.stream.Stream; + +import static org.awaitility.Awaitility.await; + +@Slf4j +@DisabledOnContainer( + value = {}, + type = {EngineType.SPARK, EngineType.FLINK}, + disabledReason = "Currently SPARK and FLINK do not support cdc") +public class PostgresCDCIT extends TestSuiteBase implements TestResource { + + // postgres + private static final String POSTGRES_HOST = "postgres_cdc_e2e"; + private static final String POSTGRES_USER_NAME = "postgres"; + private static final String POSTGRES_USER_PASSWORD = "123456"; + private static final String POSTGRES_DATABASE = "cdc_e2e"; + private static final PostgreSQLContainer POSTGRES_CONTAINER = createPostgresContainer(); + + // postgres source table query sql + private static final String SOURCE_SQL = + "select \n" + + "gid,\n" + + "text_col,\n" + + "varchar_col,\n" + + "char_col,\n" + + "boolean_col,\n" + + "smallint_col,\n" + + "integer_col,\n" + + "bigint_col,\n" + + "decimal_col,\n" + + "numeric_col,\n" + + "real_col,\n" + + "double_precision_col,\n" + + "smallserial_col,\n" + + "serial_col,\n" + + "bigserial_col,\n" + + "date_col,\n" + + "timestamp_col,\n" + + "bpchar_col,\n" + + "age,\n" + + "name\n" + + " from public.postgres_cdc_e2e_source_table"; + private static final String SINK_SQL = + "select\n" + + " gid,\n" + + " text_col,\n" + + " varchar_col,\n" + + " char_col,\n" + + " boolean_col,\n" + + " smallint_col,\n" + + " integer_col,\n" + + " bigint_col,\n" + + " decimal_col,\n" + + " numeric_col,\n" + + " real_col,\n" + + " double_precision_col,\n" + + " smallserial_col,\n" + + " serial_col,\n" + + " bigserial_col,\n" + + " date_col,\n" + + " timestamp_col,\n" + + " bpchar_col," + + " age,\n" + + " name \n" + + "from\n" + + " public.postgres_cdc_e2e_sink_table"; + + private static PostgreSQLContainer createPostgresContainer() { + return new PostgreSQLContainer<>("postgres:11.1") + .withNetwork(NETWORK) + .withNetworkAliases(POSTGRES_HOST) + .withCopyFileToContainer( + MountableFile.forClasspathResource("docker/postgres.cnf"), + "/usr/share/postgresql/postgresql.conf.sample") + .withUsername(POSTGRES_USER_NAME) + .withPassword(POSTGRES_USER_PASSWORD) + .withInitScript("ddl/postgres_cdc.sql") + .withDatabaseName(POSTGRES_DATABASE); + } + + @BeforeAll + @Override + public void startUp() throws ClassNotFoundException, InterruptedException { + log.info("The second stage: Starting postgres containers..."); + Startables.deepStart(Stream.of(POSTGRES_CONTAINER)).join(); + log.info("postgres Containers are started"); + log.info("postgres ddl execution is complete"); + } + + @TestTemplate + public void testpostgresCdcCheckDataE2e(TestContainer container) + throws IOException, InterruptedException { + + CompletableFuture executeJobFuture = + CompletableFuture.supplyAsync( + () -> { + try { + container.executeJob("/postgres_cdc_to_postgres.conf"); + } catch (Exception e) { + log.error("Commit task exception :" + e.getMessage()); + throw new RuntimeException(e); + } + return null; + }); + await().atMost(6000000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + log.info(querySql(SINK_SQL).toString()); + Assertions.assertIterableEquals( + querySql(SOURCE_SQL), querySql(SINK_SQL)); + }); + + // insert update delete + upsertDeleteSourceTable(); + + // stream stage + await().atMost(6000000, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + Assertions.assertIterableEquals( + querySql(SOURCE_SQL), querySql(SINK_SQL)); + }); + } + + private Connection getJdbcConnection() throws SQLException { + return DriverManager.getConnection( + POSTGRES_CONTAINER.getJdbcUrl(), + POSTGRES_CONTAINER.getUsername(), + POSTGRES_CONTAINER.getPassword()); + } + + private List> querySql(String sql) { + try (Connection connection = getJdbcConnection()) { + ResultSet resultSet = connection.createStatement().executeQuery(sql); + List> result = new ArrayList<>(); + int columnCount = resultSet.getMetaData().getColumnCount(); + while (resultSet.next()) { + ArrayList objects = new ArrayList<>(); + for (int i = 1; i <= columnCount; i++) { + + objects.add(resultSet.getObject(i)); + } + if (sql.equals(SINK_SQL)) { + log.info("Print postgres Cdc Sink data:" + objects); + } + if (sql.equals(SOURCE_SQL)) { + log.info("Print postgres Cdc Source data:" + objects); + } + result.add(objects); + } + return result; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + // Execute SQL + private void executeSql(String sql) { + try (Connection connection = getJdbcConnection()) { + connection.createStatement().execute(sql); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private void upsertDeleteSourceTable() { + + executeSql( + "INSERT INTO public.postgres_cdc_e2e_source_table (gid,\n" + + " text_col,\n" + + " varchar_col,\n" + + " char_col,\n" + + " boolean_col,\n" + + " smallint_col,\n" + + " integer_col,\n" + + " bigint_col,\n" + + " decimal_col,\n" + + " numeric_col,\n" + + " real_col,\n" + + " double_precision_col,\n" + + " smallserial_col,\n" + + " serial_col,\n" + + " bigserial_col,\n" + + " date_col,\n" + + " timestamp_col,\n" + + " bpchar_col,\n" + + " age,\n" + + " name\n" + + " )\n" + + " VALUES\n" + + " (\n" + + " '4',\n" + + " 'Hello World',\n" + + " 'Test',\n" + + " 'Testing',\n" + + " true,\n" + + " 10,\n" + + " 100,\n" + + " 1000,\n" + + " 10.55,\n" + + " 8.8888,\n" + + " 3.14,\n" + + " 3.14159265,\n" + + " 1,\n" + + " 100,\n" + + " 10000,\n" + + " '2023-05-07',\n" + + " '2023-05-07 14:30:00',\n" + + " 'Testing',\n" + + " 21,\n" + + " 'Leblanc');\n"); + + executeSql( + "INSERT INTO public.postgres_cdc_e2e_source_table (gid,\n" + + " text_col,\n" + + " varchar_col,\n" + + " char_col,\n" + + " boolean_col,\n" + + " smallint_col,\n" + + " integer_col,\n" + + " bigint_col,\n" + + " decimal_col,\n" + + " numeric_col,\n" + + " real_col,\n" + + " double_precision_col,\n" + + " smallserial_col,\n" + + " serial_col,\n" + + " bigserial_col,\n" + + " date_col,\n" + + " timestamp_col,\n" + + " bpchar_col,\n" + + " age,\n" + + " name\n" + + " )\n" + + " VALUES\n" + + " (\n" + + " '5',\n" + + " 'Hello World',\n" + + " 'Test',\n" + + " 'Testing',\n" + + " true,\n" + + " 10,\n" + + " 100,\n" + + " 1000,\n" + + " 10.55,\n" + + " 8.8888,\n" + + " 3.14,\n" + + " 3.14159265,\n" + + " 1,\n" + + " 100,\n" + + " 10000,\n" + + " '2023-05-07',\n" + + " '2023-05-07 14:30:00',\n" + + " 'Testing',\n" + + " 21,\n" + + " 'Leblanc');\n"); + + executeSql("DELETE FROM public.postgres_cdc_e2e_source_table where gid = 1"); + + executeSql("UPDATE public.postgres_cdc_e2e_source_table SET name = 'test' where gid = 3"); + } + + @Override + @AfterAll + public void tearDown() { + // close Container + if (POSTGRES_CONTAINER != null) { + POSTGRES_CONTAINER.close(); + } + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql new file mode 100644 index 00000000000..092a3665718 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql @@ -0,0 +1,199 @@ +-- +-- Licensed to the Apache Software Foundation (ASF) under one or more +-- contributor license agreements. See the NOTICE file distributed with +-- this work for additional information regarding copyright ownership. +-- The ASF licenses this file to You under the Apache License, Version 2.0 +-- (the "License"); you may not use this file except in compliance with +-- the License. You may obtain a copy of the License at +-- +-- http://www.apache.org/licenses/LICENSE-2.0 +-- +-- Unless required by applicable law or agreed to in writing, software +-- distributed under the License is distributed on an "AS IS" BASIS, +-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +-- See the License for the specific language governing permissions and +-- limitations under the License. + + +CREATE TABLE IF NOT EXISTS public.postgres_cdc_e2e_source_table ( + gid SERIAL PRIMARY KEY, + text_col TEXT, + varchar_col VARCHAR(255), + char_col CHAR(10), + boolean_col bool, + smallint_col int2, + integer_col int4, + bigint_col BIGINT, + decimal_col DECIMAL(10, 2), + numeric_col NUMERIC(8, 4), + real_col float4, + double_precision_col float8, + smallserial_col SMALLSERIAL, + serial_col SERIAL, + bigserial_col BIGSERIAL, + date_col DATE, + timestamp_col TIMESTAMP, + bpchar_col BPCHAR(10), + age INT NOT null, + name VARCHAR(255) NOT null + ); + +alter table public.postgres_cdc_e2e_source_table REPLICA IDENTITY FULL; + +CREATE TABLE IF NOT EXISTS public.postgres_cdc_e2e_sink_table ( + gid SERIAL PRIMARY KEY, + text_col TEXT, + varchar_col VARCHAR(255), + char_col CHAR(10), + boolean_col bool, + smallint_col int2, + integer_col int4, + bigint_col BIGINT, + decimal_col DECIMAL(10, 2), + numeric_col NUMERIC(8, 4), + real_col float4, + double_precision_col float8, + smallserial_col SMALLSERIAL, + serial_col SERIAL, + bigserial_col BIGSERIAL, + date_col DATE, + timestamp_col TIMESTAMP, + bpchar_col BPCHAR(10), + age INT NOT null, + name VARCHAR(255) NOT null + ); + +truncate public.postgres_cdc_e2e_source_table; +truncate public.postgres_cdc_e2e_sink_table; + +INSERT INTO public.postgres_cdc_e2e_source_table (gid, + text_col, + varchar_col, + char_col, + boolean_col, + smallint_col, + integer_col, + bigint_col, + decimal_col, + numeric_col, + real_col, + double_precision_col, + smallserial_col, + serial_col, + bigserial_col, + date_col, + timestamp_col, + bpchar_col, + age, + name + ) + VALUES + ( + '1', + 'Hello World', + 'Test', + 'Testing', + true, + 10, + 100, + 1000, + 10.55, + 8.8888, + 3.14, + 3.14159265, + 1, + 100, + 10000, + '2023-05-07', + '2023-05-07 14:30:00', + 'Testing', + 21, + 'Leblanc'); + +INSERT INTO public.postgres_cdc_e2e_source_table (gid, + text_col, + varchar_col, + char_col, + boolean_col, + smallint_col, + integer_col, + bigint_col, + decimal_col, + numeric_col, + real_col, + double_precision_col, + smallserial_col, + serial_col, + bigserial_col, + date_col, + timestamp_col, + bpchar_col, + age, + name + ) + VALUES + ( + '2', + 'Hello World', + 'Test', + 'Testing', + true, + 10, + 100, + 1000, + 10.55, + 8.8888, + 3.14, + 3.14159265, + 1, + 100, + 10000, + '2023-05-07', + '2023-05-07 14:30:00', + 'Testing', + 21, + 'Leblanc'); + +INSERT INTO public.postgres_cdc_e2e_source_table (gid, + text_col, + varchar_col, + char_col, + boolean_col, + smallint_col, + integer_col, + bigint_col, + decimal_col, + numeric_col, + real_col, + double_precision_col, + smallserial_col, + serial_col, + bigserial_col, + date_col, + timestamp_col, + bpchar_col, + age, + name + ) + VALUES + ( + '3', + 'Hello World', + 'Test', + 'Testing', + true, + 10, + 100, + 1000, + 10.55, + 8.8888, + 3.14, + 3.14159265, + 1, + 100, + 10000, + '2023-05-07', + '2023-05-07 14:30:00', + 'Testing', + 21, + 'Leblanc'); diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf new file mode 100644 index 00000000000..655db90633c --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf @@ -0,0 +1,704 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# ----------------------------- +# PostgreSQL configuration file +# ----------------------------- +# +# This file consists of lines of the form: +# +# name = value +# +# (The "=" is optional.) Whitespace may be used. Comments are introduced with +# "#" anywhere on a line. The complete list of parameter names and allowed +# values can be found in the PostgreSQL documentation. +# +# The commented-out settings shown in this file represent the default values. +# Re-commenting a setting is NOT sufficient to revert it to the default value; +# you need to reload the server. +# +# This file is read on server startup and when the server receives a SIGHUP +# signal. If you edit the file on a running system, you have to SIGHUP the +# server for the changes to take effect, run "pg_ctl reload", or execute +# "SELECT pg_reload_conf()". Some parameters, which are marked below, +# require a server shutdown and restart to take effect. +# +# Any parameter can also be given as a command-line option to the server, e.g., +# "postgres -c log_connections=on". Some parameters can be changed at run time +# with the "SET" SQL command. +# +# Memory units: kB = kilobytes Time units: ms = milliseconds +# MB = megabytes s = seconds +# GB = gigabytes min = minutes +# TB = terabytes h = hours +# d = days + + +#------------------------------------------------------------------------------ +# FILE LOCATIONS +#------------------------------------------------------------------------------ + +# The default values of these variables are driven from the -D command-line +# option or PGDATA environment variable, represented here as ConfigDir. + +#data_directory = 'ConfigDir' # use data in another directory + # (change requires restart) +#hba_file = 'ConfigDir/pg_hba.conf' # host-based authentication file + # (change requires restart) +#ident_file = 'ConfigDir/pg_ident.conf' # ident configuration file + # (change requires restart) + +# If external_pid_file is not explicitly set, no extra PID file is written. +#external_pid_file = '' # write an extra PID file + # (change requires restart) + + +#------------------------------------------------------------------------------ +# CONNECTIONS AND AUTHENTICATION +#------------------------------------------------------------------------------ + +# - Connection Settings - + +listen_addresses = '*' + # comma-separated list of addresses; + # defaults to 'localhost'; use '*' for all + # (change requires restart) +#port = 5432 # (change requires restart) +max_connections = 100 # (change requires restart) +#superuser_reserved_connections = 3 # (change requires restart) +#unix_socket_directories = '/var/run/postgresql' # comma-separated list of directories + # (change requires restart) +#unix_socket_group = '' # (change requires restart) +#unix_socket_permissions = 0777 # begin with 0 to use octal notation + # (change requires restart) +#bonjour = off # advertise server via Bonjour + # (change requires restart) +#bonjour_name = '' # defaults to the computer name + # (change requires restart) + +# - TCP Keepalives - +# see "man 7 tcp" for details + +#tcp_keepalives_idle = 0 # TCP_KEEPIDLE, in seconds; + # 0 selects the system default +#tcp_keepalives_interval = 0 # TCP_KEEPINTVL, in seconds; + # 0 selects the system default +#tcp_keepalives_count = 0 # TCP_KEEPCNT; + # 0 selects the system default + +# - Authentication - + +#authentication_timeout = 1min # 1s-600s +#password_encryption = md5 # md5 or scram-sha-256 +#db_user_namespace = off + +# GSSAPI using Kerberos +#krb_server_keyfile = '' +#krb_caseins_users = off + +# - SSL - + +#ssl = off +#ssl_ca_file = '' +#ssl_cert_file = 'server.crt' +#ssl_crl_file = '' +#ssl_key_file = 'server.key' +#ssl_ciphers = 'HIGH:MEDIUM:+3DES:!aNULL' # allowed SSL ciphers +#ssl_prefer_server_ciphers = on +#ssl_ecdh_curve = 'prime256v1' +#ssl_dh_params_file = '' +#ssl_passphrase_command = '' +#ssl_passphrase_command_supports_reload = off + + +#------------------------------------------------------------------------------ +# RESOURCE USAGE (except WAL) +#------------------------------------------------------------------------------ + +# - Memory - + +shared_buffers = 128MB # min 128kB + # (change requires restart) +#huge_pages = try # on, off, or try + # (change requires restart) +#temp_buffers = 8MB # min 800kB +#max_prepared_transactions = 0 # zero disables the feature + # (change requires restart) +# Caution: it is not advisable to set max_prepared_transactions nonzero unless +# you actively intend to use prepared transactions. +#work_mem = 4MB # min 64kB +#maintenance_work_mem = 64MB # min 1MB +#autovacuum_work_mem = -1 # min 1MB, or -1 to use maintenance_work_mem +#max_stack_depth = 2MB # min 100kB +dynamic_shared_memory_type = posix # the default is the first option + # supported by the operating system: + # posix + # sysv + # windows + # mmap + # use none to disable dynamic shared memory + # (change requires restart) + +# - Disk - + +#temp_file_limit = -1 # limits per-process temp file space + # in kB, or -1 for no limit + +# - Kernel Resources - + +#max_files_per_process = 1000 # min 25 + # (change requires restart) + +# - Cost-Based Vacuum Delay - + +#vacuum_cost_delay = 0 # 0-100 milliseconds +#vacuum_cost_page_hit = 1 # 0-10000 credits +#vacuum_cost_page_miss = 10 # 0-10000 credits +#vacuum_cost_page_dirty = 20 # 0-10000 credits +#vacuum_cost_limit = 200 # 1-10000 credits + +# - Background Writer - + +#bgwriter_delay = 200ms # 10-10000ms between rounds +#bgwriter_lru_maxpages = 100 # max buffers written/round, 0 disables +#bgwriter_lru_multiplier = 2.0 # 0-10.0 multiplier on buffers scanned/round +#bgwriter_flush_after = 512kB # measured in pages, 0 disables + +# - Asynchronous Behavior - + +#effective_io_concurrency = 1 # 1-1000; 0 disables prefetching +#max_worker_processes = 8 # (change requires restart) +#max_parallel_maintenance_workers = 2 # taken from max_parallel_workers +#max_parallel_workers_per_gather = 2 # taken from max_parallel_workers +#parallel_leader_participation = on +#max_parallel_workers = 8 # maximum number of max_worker_processes that + # can be used in parallel operations +#old_snapshot_threshold = -1 # 1min-60d; -1 disables; 0 is immediate + # (change requires restart) +#backend_flush_after = 0 # measured in pages, 0 disables + + +#------------------------------------------------------------------------------ +# WRITE-AHEAD LOG +#------------------------------------------------------------------------------ + +# - Settings - + +wal_level = logical # minimal, replica, or logical + # (change requires restart) +#fsync = on # flush data to disk for crash safety + # (turning this off can cause + # unrecoverable data corruption) +#synchronous_commit = on # synchronization level; + # off, local, remote_write, remote_apply, or on +#wal_sync_method = fsync # the default is the first option + # supported by the operating system: + # open_datasync + # fdatasync (default on Linux) + # fsync + # fsync_writethrough + # open_sync +#full_page_writes = on # recover from partial page writes +#wal_compression = off # enable compression of full-page writes +#wal_log_hints = off # also do full page writes of non-critical updates + # (change requires restart) +#wal_buffers = -1 # min 32kB, -1 sets based on shared_buffers + # (change requires restart) +#wal_writer_delay = 200ms # 1-10000 milliseconds +#wal_writer_flush_after = 1MB # measured in pages, 0 disables + +#commit_delay = 0 # range 0-100000, in microseconds +#commit_siblings = 5 # range 1-1000 + +# - Checkpoints - + +#checkpoint_timeout = 5min # range 30s-1d +max_wal_size = 1GB +min_wal_size = 80MB +#checkpoint_completion_target = 0.5 # checkpoint target duration, 0.0 - 1.0 +#checkpoint_flush_after = 256kB # measured in pages, 0 disables +#checkpoint_warning = 30s # 0 disables + +# - Archiving - + +#archive_mode = off # enables archiving; off, on, or always + # (change requires restart) +#archive_command = '' # command to use to archive a logfile segment + # placeholders: %p = path of file to archive + # %f = file name only + # e.g. 'test ! -f /mnt/server/archivedir/%f && cp %p /mnt/server/archivedir/%f' +#archive_timeout = 0 # force a logfile segment switch after this + # number of seconds; 0 disables + + +#------------------------------------------------------------------------------ +# REPLICATION +#------------------------------------------------------------------------------ + +# - Sending Servers - + +# Set these on the master and on any standby that will send replication data. + +#max_wal_senders = 10 # max number of walsender processes + # (change requires restart) +#wal_keep_segments = 0 # in logfile segments; 0 disables +#wal_sender_timeout = 60s # in milliseconds; 0 disables + +#max_replication_slots = 10 # max number of replication slots + # (change requires restart) +#track_commit_timestamp = off # collect timestamp of transaction commit + # (change requires restart) + +# - Master Server - + +# These settings are ignored on a standby server. + +#synchronous_standby_names = '' # standby servers that provide sync rep + # method to choose sync standbys, number of sync standbys, + # and comma-separated list of application_name + # from standby(s); '*' = all +#vacuum_defer_cleanup_age = 0 # number of xacts by which cleanup is delayed + +# - Standby Servers - + +# These settings are ignored on a master server. + +#hot_standby = on # "off" disallows queries during recovery + # (change requires restart) +#max_standby_archive_delay = 30s # max delay before canceling queries + # when reading WAL from archive; + # -1 allows indefinite delay +#max_standby_streaming_delay = 30s # max delay before canceling queries + # when reading streaming WAL; + # -1 allows indefinite delay +#wal_receiver_status_interval = 10s # send replies at least this often + # 0 disables +#hot_standby_feedback = off # send info from standby to prevent + # query conflicts +#wal_receiver_timeout = 60s # time that receiver waits for + # communication from master + # in milliseconds; 0 disables +#wal_retrieve_retry_interval = 5s # time to wait before retrying to + # retrieve WAL after a failed attempt + +# - Subscribers - + +# These settings are ignored on a publisher. + +#max_logical_replication_workers = 4 # taken from max_worker_processes + # (change requires restart) +#max_sync_workers_per_subscription = 2 # taken from max_logical_replication_workers + + +#------------------------------------------------------------------------------ +# QUERY TUNING +#------------------------------------------------------------------------------ + +# - Planner Method Configuration - + +#enable_bitmapscan = on +#enable_hashagg = on +#enable_hashjoin = on +#enable_indexscan = on +#enable_indexonlyscan = on +#enable_material = on +#enable_mergejoin = on +#enable_nestloop = on +#enable_parallel_append = on +#enable_seqscan = on +#enable_sort = on +#enable_tidscan = on +#enable_partitionwise_join = off +#enable_partitionwise_aggregate = off +#enable_parallel_hash = on +#enable_partition_pruning = on + +# - Planner Cost Constants - + +#seq_page_cost = 1.0 # measured on an arbitrary scale +#random_page_cost = 4.0 # same scale as above +#cpu_tuple_cost = 0.01 # same scale as above +#cpu_index_tuple_cost = 0.005 # same scale as above +#cpu_operator_cost = 0.0025 # same scale as above +#parallel_tuple_cost = 0.1 # same scale as above +#parallel_setup_cost = 1000.0 # same scale as above + +#jit_above_cost = 100000 # perform JIT compilation if available + # and query more expensive than this; + # -1 disables +#jit_inline_above_cost = 500000 # inline small functions if query is + # more expensive than this; -1 disables +#jit_optimize_above_cost = 500000 # use expensive JIT optimizations if + # query is more expensive than this; + # -1 disables + +#min_parallel_table_scan_size = 8MB +#min_parallel_index_scan_size = 512kB +#effective_cache_size = 4GB + +# - Genetic Query Optimizer - + +#geqo = on +#geqo_threshold = 12 +#geqo_effort = 5 # range 1-10 +#geqo_pool_size = 0 # selects default based on effort +#geqo_generations = 0 # selects default based on effort +#geqo_selection_bias = 2.0 # range 1.5-2.0 +#geqo_seed = 0.0 # range 0.0-1.0 + +# - Other Planner Options - + +#default_statistics_target = 100 # range 1-10000 +#constraint_exclusion = partition # on, off, or partition +#cursor_tuple_fraction = 0.1 # range 0.0-1.0 +#from_collapse_limit = 8 +#join_collapse_limit = 8 # 1 disables collapsing of explicit + # JOIN clauses +#force_parallel_mode = off +#jit = off # allow JIT compilation + + +#------------------------------------------------------------------------------ +# REPORTING AND LOGGING +#------------------------------------------------------------------------------ + +# - Where to Log - + +#log_destination = 'stderr' # Valid values are combinations of + # stderr, csvlog, syslog, and eventlog, + # depending on platform. csvlog + # requires logging_collector to be on. + +# This is used when logging to stderr: +#logging_collector = off # Enable capturing of stderr and csvlog + # into log files. Required to be on for + # csvlogs. + # (change requires restart) + +# These are only used if logging_collector is on: +#log_directory = 'log' # directory where log files are written, + # can be absolute or relative to PGDATA +#log_filename = 'postgresql-%Y-%m-%d_%H%M%S.log' # log file name pattern, + # can include strftime() escapes +#log_file_mode = 0600 # creation mode for log files, + # begin with 0 to use octal notation +#log_truncate_on_rotation = off # If on, an existing log file with the + # same name as the new log file will be + # truncated rather than appended to. + # But such truncation only occurs on + # time-driven rotation, not on restarts + # or size-driven rotation. Default is + # off, meaning append to existing files + # in all cases. +#log_rotation_age = 1d # Automatic rotation of logfiles will + # happen after that time. 0 disables. +#log_rotation_size = 10MB # Automatic rotation of logfiles will + # happen after that much log output. + # 0 disables. + +# These are relevant when logging to syslog: +#syslog_facility = 'LOCAL0' +#syslog_ident = 'postgres' +#syslog_sequence_numbers = on +#syslog_split_messages = on + +# This is only relevant when logging to eventlog (win32): +# (change requires restart) +#event_source = 'PostgreSQL' + +# - When to Log - + +#client_min_messages = notice # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # log + # notice + # warning + # error + +#log_min_messages = warning # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # info + # notice + # warning + # error + # log + # fatal + # panic + +#log_min_error_statement = error # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # info + # notice + # warning + # error + # log + # fatal + # panic (effectively off) + +#log_min_duration_statement = -1 # -1 is disabled, 0 logs all statements + # and their durations, > 0 logs only + # statements running at least this number + # of milliseconds + + +# - What to Log - + +#debug_print_parse = off +#debug_print_rewritten = off +#debug_print_plan = off +#debug_pretty_print = on +#log_checkpoints = off +#log_connections = off +#log_disconnections = off +#log_duration = off +#log_error_verbosity = default # terse, default, or verbose messages +#log_hostname = off +#log_line_prefix = '%m [%p] ' # special values: + # %a = application name + # %u = user name + # %d = database name + # %r = remote host and port + # %h = remote host + # %p = process ID + # %t = timestamp without milliseconds + # %m = timestamp with milliseconds + # %n = timestamp with milliseconds (as a Unix epoch) + # %i = command tag + # %e = SQL state + # %c = session ID + # %l = session line number + # %s = session start timestamp + # %v = virtual transaction ID + # %x = transaction ID (0 if none) + # %q = stop here in non-session + # processes + # %% = '%' + # e.g. '<%u%%%d> ' +#log_lock_waits = off # log lock waits >= deadlock_timeout +#log_statement = 'none' # none, ddl, mod, all +#log_replication_commands = off +#log_temp_files = -1 # log temporary files equal or larger + # than the specified size in kilobytes; + # -1 disables, 0 logs all temp files +log_timezone = 'UTC' + +#------------------------------------------------------------------------------ +# PROCESS TITLE +#------------------------------------------------------------------------------ + +#cluster_name = '' # added to process titles if nonempty + # (change requires restart) +#update_process_title = on + + +#------------------------------------------------------------------------------ +# STATISTICS +#------------------------------------------------------------------------------ + +# - Query and Index Statistics Collector - + +#track_activities = on +#track_counts = on +#track_io_timing = off +#track_functions = none # none, pl, all +#track_activity_query_size = 1024 # (change requires restart) +#stats_temp_directory = 'pg_stat_tmp' + + +# - Monitoring - + +#log_parser_stats = off +#log_planner_stats = off +#log_executor_stats = off +#log_statement_stats = off + + +#------------------------------------------------------------------------------ +# AUTOVACUUM +#------------------------------------------------------------------------------ + +#autovacuum = on # Enable autovacuum subprocess? 'on' + # requires track_counts to also be on. +#log_autovacuum_min_duration = -1 # -1 disables, 0 logs all actions and + # their durations, > 0 logs only + # actions running at least this number + # of milliseconds. +#autovacuum_max_workers = 3 # max number of autovacuum subprocesses + # (change requires restart) +#autovacuum_naptime = 1min # time between autovacuum runs +#autovacuum_vacuum_threshold = 50 # min number of row updates before + # vacuum +#autovacuum_analyze_threshold = 50 # min number of row updates before + # analyze +#autovacuum_vacuum_scale_factor = 0.2 # fraction of table size before vacuum +#autovacuum_analyze_scale_factor = 0.1 # fraction of table size before analyze +#autovacuum_freeze_max_age = 200000000 # maximum XID age before forced vacuum + # (change requires restart) +#autovacuum_multixact_freeze_max_age = 400000000 # maximum multixact age + # before forced vacuum + # (change requires restart) +#autovacuum_vacuum_cost_delay = 20ms # default vacuum cost delay for + # autovacuum, in milliseconds; + # -1 means use vacuum_cost_delay +#autovacuum_vacuum_cost_limit = -1 # default vacuum cost limit for + # autovacuum, -1 means use + # vacuum_cost_limit + + +#------------------------------------------------------------------------------ +# CLIENT CONNECTION DEFAULTS +#------------------------------------------------------------------------------ + +# - Statement Behavior - + +#search_path = '"$user", public' # schema names +#row_security = on +#default_tablespace = '' # a tablespace name, '' uses the default +#temp_tablespaces = '' # a list of tablespace names, '' uses + # only default tablespace +#check_function_bodies = on +#default_transaction_isolation = 'read committed' +#default_transaction_read_only = off +#default_transaction_deferrable = off +#session_replication_role = 'origin' +#statement_timeout = 0 # in milliseconds, 0 is disabled +#lock_timeout = 0 # in milliseconds, 0 is disabled +#idle_in_transaction_session_timeout = 0 # in milliseconds, 0 is disabled +#vacuum_freeze_min_age = 50000000 +#vacuum_freeze_table_age = 150000000 +#vacuum_multixact_freeze_min_age = 5000000 +#vacuum_multixact_freeze_table_age = 150000000 +#vacuum_cleanup_index_scale_factor = 0.1 # fraction of total number of tuples + # before index cleanup, 0 always performs + # index cleanup +#bytea_output = 'hex' # hex, escape +#xmlbinary = 'base64' +#xmloption = 'content' +#gin_fuzzy_search_limit = 0 +#gin_pending_list_limit = 4MB + +# - Locale and Formatting - + +datestyle = 'iso, mdy' +#intervalstyle = 'postgres' +timezone = 'UTC' +#timezone_abbreviations = 'Default' # Select the set of available time zone + # abbreviations. Currently, there are + # Default + # Australia (historical usage) + # India + # You can create your own file in + # share/timezonesets/. +#extra_float_digits = 0 # min -15, max 3 +#client_encoding = sql_ascii # actually, defaults to database + # encoding + +# These settings are initialized by initdb, but they can be changed. +lc_messages = 'en_US.utf8' # locale for system error message + # strings +lc_monetary = 'en_US.utf8' # locale for monetary formatting +lc_numeric = 'en_US.utf8' # locale for number formatting +lc_time = 'en_US.utf8' # locale for time formatting + +# default configuration for text search +default_text_search_config = 'pg_catalog.english' + +# - Shared Library Preloading - + +#shared_preload_libraries = '' # (change requires restart) +#local_preload_libraries = '' +#session_preload_libraries = '' +#jit_provider = 'llvmjit' # JIT library to use + +# - Other Defaults - + +#dynamic_library_path = '$libdir' + + +#------------------------------------------------------------------------------ +# LOCK MANAGEMENT +#------------------------------------------------------------------------------ + +#deadlock_timeout = 1s +#max_locks_per_transaction = 64 # min 10 + # (change requires restart) +#max_pred_locks_per_transaction = 64 # min 10 + # (change requires restart) +#max_pred_locks_per_relation = -2 # negative values mean + # (max_pred_locks_per_transaction + # / -max_pred_locks_per_relation) - 1 +#max_pred_locks_per_page = 2 # min 0 + + +#------------------------------------------------------------------------------ +# VERSION AND PLATFORM COMPATIBILITY +#------------------------------------------------------------------------------ + +# - Previous PostgreSQL Versions - + +#array_nulls = on +#backslash_quote = safe_encoding # on, off, or safe_encoding +#default_with_oids = off +#escape_string_warning = on +#lo_compat_privileges = off +#operator_precedence_warning = off +#quote_all_identifiers = off +#standard_conforming_strings = on +#synchronize_seqscans = on + +# - Other Platforms and Clients - + +#transform_null_equals = off + + +#------------------------------------------------------------------------------ +# ERROR HANDLING +#------------------------------------------------------------------------------ + +#exit_on_error = off # terminate session on any error? +#restart_after_crash = on # reinitialize after backend crash? + + +#------------------------------------------------------------------------------ +# CONFIG FILE INCLUDES +#------------------------------------------------------------------------------ + +# These options allow settings to be loaded from files other than the +# default postgresql.conf. + +#include_dir = 'conf.d' # include files ending in '.conf' from + # directory 'conf.d' +#include_if_exists = 'exists.conf' # include file only if it exists +#include = 'special.conf' # include file + + +#------------------------------------------------------------------------------ +# CUSTOMIZED OPTIONS +#------------------------------------------------------------------------------ + +# Add settings for extensions here diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf new file mode 100644 index 00000000000..a319a172c00 --- /dev/null +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf @@ -0,0 +1,55 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +###### +###### This config file is a demonstration of streaming processing in seatunnel config +###### + +env { + # You can set engine configuration here + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 + #execution.checkpoint.data-uri = "hdfs://localhost:9000/checkpoint" +} + +source { + + # This is a example source plugin **only for test and demonstrate the feature source plugin** + Postgres-CDC { + base-url = "jdbc:postgresql://postgres_cdc_e2e:5432/cdc_e2e" + username = "postgres" + password = "123456" + database-names = ["cdc_e2e"] + table-names = ["cdc_e2e.public.postgres_cdc_e2e_source_table"] + } +} + +transform { +} + +sink { + JDBC { + url = "jdbc:postgresql://postgres_cdc_e2e:5432/cdc_e2e" + driver = "org.postgresql.Driver" + user = "postgres" + database = "cdc_e2e" + password = "123456" + generate_sink_sql = true + table = public.postgres_cdc_e2e_sink_table + primary_keys = ["gid"] + } +} diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 8644b551b2f..a8baf1c68da 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -61,6 +61,7 @@ connector-file-ftp-e2e connector-pulsar-e2e connector-paimon-e2e + connector-cdc-postgres-e2e From b55926335d1c5df0dd54d43f365671098063d508 Mon Sep 17 00:00:00 2001 From: sunkang Date: Wed, 6 Sep 2023 16:45:32 +0800 Subject: [PATCH 2/3] [Feature][connector-v2][PostgreSQL CDC]Support source PostgreSQL CDC --- docs/en/connector-v2/source/Postgres-CDC.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/connector-v2/source/Postgres-CDC.md b/docs/en/connector-v2/source/Postgres-CDC.md index fd0ddad176d..b1b1b8fc181 100644 --- a/docs/en/connector-v2/source/Postgres-CDC.md +++ b/docs/en/connector-v2/source/Postgres-CDC.md @@ -18,8 +18,8 @@ describes how to set up the Postgres CDC connector to run SQL queries against Po ## Options -| name | type | required | default value | -| ---------------------------------------------- | -------- | -------- | ------------- | +| name | type | required | default value | +|------------------------------------------------|----------|----------|---------------| | base-url | String | Yes | - | | username | String | Yes | - | | password | String | Yes | - | From 0d60de407ea473fb6a90a6a01177caae0dc6fb7e Mon Sep 17 00:00:00 2001 From: sunkang Date: Wed, 6 Dec 2023 12:51:21 +0800 Subject: [PATCH 3/3] [Feature][connector-v2][PostgreSQL CDC]Support source PostgreSQL CDC --- docs/en/connector-v2/source/PostgreSQL-CDC.md | 287 ++++++++++++++++++ docs/en/connector-v2/source/Postgres-CDC.md | 190 ------------ plugin-mapping.properties | 2 +- .../cdc/postgres/utils/PostgresTypeUtils.java | 93 ------ .../pom.xml | 6 +- .../postgresql/PostgresConnectorConfig.java | 0 .../PostgresEventMetadataProvider.java | 0 .../postgresql/PostgresOffsetContext.java | 8 +- .../connector/postgresql/PostgresSchema.java | 0 .../postgresql/PostgresTaskContext.java | 0 .../connector/postgresql/TypeRegistry.java | 0 .../connection/PostgresConnection.java | 0 .../config/PostgresSourceConfig.java | 2 +- .../config/PostgresSourceConfigFactory.java | 4 +- .../postgresql}/option/PostgresOptions.java | 2 +- .../postgresql}/source/PostgresDialect.java | 33 +- .../source/PostgresIncrementalSource.java | 13 +- .../PostgresIncrementalSourceFactory.java | 43 +-- .../PostgresPooledDataSourceFactory.java | 2 +- .../source/PostgresSourceOptions.java | 2 +- .../enumerator/PostgresChunkSplitter.java | 6 +- .../postgresql}/source/offset/LsnOffset.java | 2 +- .../source/offset/LsnOffsetFactory.java | 10 +- .../PostgresSourceFetchTaskContext.java | 13 +- .../snapshot/PostgresSnapshotFetchTask.java | 6 +- .../PostgresSnapshotSplitReadTask.java | 7 +- ...SnapshotSplitChangeEventSourceContext.java | 4 +- .../reader/wal/PostgresWalFetchTask.java | 4 +- .../utils/PostgresConnectionUtils.java | 16 +- .../cdc/postgresql}/utils/PostgresSchema.java | 14 +- .../postgresql/utils/PostgresTypeUtils.java | 148 +++++++++ .../cdc/postgresql}/utils/PostgresUtils.java | 58 +++- .../utils/TableDiscoveryUtils.java | 2 +- .../PostgresIncrementalSourceFactoryTest.java | 2 +- seatunnel-connectors-v2/connector-cdc/pom.xml | 2 +- .../pom.xml | 8 +- .../cdc/postgresql/PostgreSQLCDCIT.java} | 18 +- .../src/test/resources/ddl/postgres_cdc.sql | 3 - .../src/test/resources/docker/postgres.cnf | 0 .../resources/postgres_cdc_to_postgres.conf | 0 .../seatunnel-connector-v2-e2e/pom.xml | 2 +- 41 files changed, 600 insertions(+), 412 deletions(-) create mode 100644 docs/en/connector-v2/source/PostgreSQL-CDC.md delete mode 100644 docs/en/connector-v2/source/Postgres-CDC.md delete mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/pom.xml (95%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java (99%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java (100%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/config/PostgresSourceConfig.java (97%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/config/PostgresSourceConfigFactory.java (96%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/option/PostgresOptions.java (96%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/PostgresDialect.java (73%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/PostgresIncrementalSource.java (92%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/PostgresIncrementalSourceFactory.java (73%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/PostgresPooledDataSourceFactory.java (95%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/PostgresSourceOptions.java (97%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/enumerator/PostgresChunkSplitter.java (93%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/offset/LsnOffset.java (97%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/offset/LsnOffsetFactory.java (85%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/reader/PostgresSourceFetchTaskContext.java (96%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/reader/snapshot/PostgresSnapshotFetchTask.java (94%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/reader/snapshot/PostgresSnapshotSplitReadTask.java (97%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java (90%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/source/reader/wal/PostgresWalFetchTask.java (94%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/utils/PostgresConnectionUtils.java (74%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/utils/PostgresSchema.java (85%) create mode 100644 seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresTypeUtils.java rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/utils/PostgresUtils.java (89%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres => connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql}/utils/TableDiscoveryUtils.java (98%) rename seatunnel-connectors-v2/connector-cdc/{connector-cdc-postgres => connector-cdc-postgresql}/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java (91%) rename seatunnel-e2e/seatunnel-connector-v2-e2e/{connector-cdc-postgres-e2e => connector-cdc-postgresql-e2e}/pom.xml (90%) rename seatunnel-e2e/seatunnel-connector-v2-e2e/{connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java => connector-cdc-postgresql-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgresql/PostgreSQLCDCIT.java} (96%) rename seatunnel-e2e/seatunnel-connector-v2-e2e/{connector-cdc-postgres-e2e => connector-cdc-postgresql-e2e}/src/test/resources/ddl/postgres_cdc.sql (98%) rename seatunnel-e2e/seatunnel-connector-v2-e2e/{connector-cdc-postgres-e2e => connector-cdc-postgresql-e2e}/src/test/resources/docker/postgres.cnf (100%) rename seatunnel-e2e/seatunnel-connector-v2-e2e/{connector-cdc-postgres-e2e => connector-cdc-postgresql-e2e}/src/test/resources/postgres_cdc_to_postgres.conf (100%) diff --git a/docs/en/connector-v2/source/PostgreSQL-CDC.md b/docs/en/connector-v2/source/PostgreSQL-CDC.md new file mode 100644 index 00000000000..abeada5d3cd --- /dev/null +++ b/docs/en/connector-v2/source/PostgreSQL-CDC.md @@ -0,0 +1,287 @@ +# PostgreSQL CDC + +> PostgreSQL CDC source connector + +## Support Those Engines + +> SeaTunnel Zeta
+> Flink
+ +## Key features + +- [ ] [batch](../../concept/connector-v2-features.md) +- [x] [stream](../../concept/connector-v2-features.md) +- [x] [exactly-once](../../concept/connector-v2-features.md) +- [ ] [column projection](../../concept/connector-v2-features.md) +- [x] [parallelism](../../concept/connector-v2-features.md) +- [x] [support user-defined split](../../concept/connector-v2-features.md) + +## Description + +The PostgreSQL CDC connector allows for reading snapshot data and incremental data from PostgreSQL database. This document +describes how to set up the PostgreSQL CDC connector to run SQL queries against PostgreSQL databases. + +## Supported DataSource Info + +| Datasource | Supported versions | Driver | Url | Maven | +|------------|------------------------|-----------------------|---------------------------------------|---------------------------------------------------------------------| +| PostgreSQL | 9.6, 10, 11, 12 and 13 | org.postgresql.Driver | jdbc:postgresql://localhost:5432/test | https://mvnrepository.com/artifact/org.postgresql/postgresql/42.6.0 | + +## Database Dependency + +### Install Jdbc Driver + +Please download and put PostgreSQL driver in `${SEATUNNEL_HOME}/lib/` dir. For example: cp postgresql-xxx.jar `$SEATNUNNEL_HOME/lib/` + +### Configuring the PostgreSQL server + +configure the replication slot regardless of the decoder being used, specify the following in the `postgresql.conf` file: + +```properties +# REPLICATION +wal_level = logical +``` + +### Setting up permissions + +Setting up a PostgreSQL server to run a Debezium connector requires a database user that can perform replications. Replication can be performed only by a database user that has appropriate permissions and only for a configured number of hosts. + +Although, by default, superusers have the necessary REPLICATION and LOGIN roles, as mentioned in Security, it is best not to provide the Debezium replication user with elevated privileges. Instead, create a Debezium user that has the minimum required privileges. + +Prerequisites +1. PostgreSQL administrative permissions. + +Procedure +1. provide a user with replication permissions, define a PostgreSQL role that has at least the REPLICATION and LOGIN permissions, and then grant that role to the user. For example: + +```properties +CREATE ROLE REPLICATION LOGIN; +``` + +### Configuring PostgreSQL to allow replication with the Debezium connector host + +To enable Debezium to replicate PostgreSQL data, you must configure the database to permit replication with the host that runs the PostgreSQL connector. To specify the clients that are permitted to replicate with the database, add entries to the PostgreSQL host-based authentication file, pg_hba.conf. For more information about the pg_hba.conf file, see the PostgreSQL documentation. + +- Procedure + Add entries to the pg_hba.conf file to specify the Debezium connector hosts that can replicate with the database host. For example, + +pg_hba.conf file example: + +```properties +local replication trust +host replication 127.0.0.1/32 trust +host replication ::1/128 trust +``` + +### Replica identity + +[REPLICA IDENTITY](https://www.postgresql.org/docs/current/static/sql-altertable.html#SQL-CREATETABLE-REPLICA-IDENTITY) is a PostgreSQL-specific table-level setting that determines the amount of information that is available to the logical decoding plug-in for `UPDATE` and `DELETE` events. More specifically, the setting of `REPLICA IDENTITY` controls what (if any) information is available for the previous values of the table columns involved, whenever an `UPDATE` or `DELETE` event occurs. + +ALL table need set REPLICA IDENTITY + +```sql +ALTER TABLE customers REPLICA IDENTITY FULL; +``` + +## Data Type Mapping + +| PostgreSQL Data type | SeaTunnel Data type | +|-----------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------| +| BOOL
| BOOLEAN | +| _BOOL
| ARRAY<BOOLEAN> | +| BYTEA
| BYTES | +| _BYTEA
| ARRAY<TINYINT> | +| INT2
SMALLSERIAL
INT4
SERIAL
| INT | +| _INT2
_INT4
| ARRAY<INT> | +| INT8
BIGSERIAL
| BIGINT | +| _INT8
| ARRAY<BIGINT> | +| FLOAT4
| FLOAT | +| _FLOAT4
| ARRAY<FLOAT> | +| FLOAT8
| DOUBLE | +| _FLOAT8
| ARRAY<DOUBLE> | +| NUMERIC(Get the designated column's specified column size>0) | DECIMAL(Get the designated column's specified column size,Gets the number of digits in the specified column to the right of the decimal point) | +| NUMERIC(Get the designated column's specified column size<0) | DECIMAL(38, 18) | +| BPCHAR
CHARACTER
VARCHAR
TEXT
GEOMETRY
GEOGRAPHY
JSON
JSONB | STRING | +| _BPCHAR
_CHARACTER
_VARCHAR
_TEXT | ARRAY<STRING> | +| TIMESTAMP
| TIMESTAMP | +| TIME
| TIME | +| DATE
| DATE | +| OTHER DATA TYPES | NOT SUPPORTED YET | + +## Source Options + +| name | type | required | default value | +|------------------------------------------------|----------|----------|---------------| +| base-url | String | Yes | - | +| username | String | Yes | - | +| password | String | Yes | - | +| database-names | List | Yes | - | +| table-names | List | Yes | - | +| startup.mode | Enum | No | INITIAL | +| startup.timestamp | Long | No | - | +| startup.specific-offset.file | String | No | - | +| startup.specific-offset.pos | Long | No | - | +| stop.mode | Enum | No | NEVER | +| stop.timestamp | Long | No | - | +| stop.specific-offset.file | String | No | - | +| stop.specific-offset.pos | Long | No | - | +| incremental.parallelism | Integer | No | 1 | +| snapshot.split.size | Integer | No | 8096 | +| snapshot.fetch.size | Integer | No | 1024 | +| server-time-zone | String | No | UTC | +| connect.timeout.ms | Duration | No | 30000 | +| connect.max-retries | Integer | No | 3 | +| connection.pool.size | Integer | No | 20 | +| chunk-key.even-distribution.factor.upper-bound | Double | No | 1000 | +| chunk-key.even-distribution.factor.lower-bound | Double | No | 0.05 | +| debezium.* | config | No | - | +| format | Enum | No | DEFAULT | +| common-options | | no | - | + +### username [String] + +Name of the database to use when connecting to the database server. + +### password [String] + +Password to use when connecting to the database server. + +### database-name [String] + +Database name of the database to monitor. + +### schmea-list [List] + +Schmea name of the database to monitor. + +### table-names [List] + +Table name of the database to monitor. The table name needs to include the schema name, for example: schema_name.table_name + +### startup.mode [Enum] + +Optional startup mode for postgres CDC consumer, valid enumerations are "initial", "earliest", "latest" and "specific". + +### startup.timestamp [Long] + +Start from the specified epoch timestamp (in milliseconds). + +**Note, This option is required when the "startup.mode" option used `'timestamp'`.** + +### startup.specific-offset.file [String] + +Start from the specified binlog file name. + +**Note, This option is required when the "startup.mode" option used `'specific'`.** + +### startup.specific-offset.pos [Long] + +Start from the specified binlog file position. + +**Note, This option is required when the "startup.mode" option used `'specific'`.** + +### stop.mode [Enum] + +Optional stop mode for postgres CDC consumer, valid enumerations are "never". + +### stop.timestamp [Long] + +Stop from the specified epoch timestamp (in milliseconds). + +**Note, This option is required when the "stop.mode" option used `'timestamp'`.** + +### stop.specific-offset.file [String] + +Stop from the specified binlog file name. + +**Note, This option is required when the "stop.mode" option used `'specific'`.** + +### stop.specific-offset.pos [Long] + +Stop from the specified binlog file position. + +**Note, This option is required when the "stop.mode" option used `'specific'`.** + +### incremental.parallelism [Integer] + +The number of parallel readers in the incremental phase. + +### snapshot.split.size [Integer] + +The split size (number of rows) of table snapshot, captured tables are split into multiple splits when read the snapshot +of table. + +### snapshot.fetch.size [Integer] + +The maximum fetch size for per poll when read table snapshot. + +### server-time-zone [String] + +The session time zone in database server. + +### connect.timeout.ms [long] + +The maximum time that the connector should wait after trying to connect to the database server before timing out. + +### connect.max-retries [Integer] + +The max retry times that the connector should retry to build database server connection. + +### connection.pool.size [Integer] + +The connection pool size. + +### debezium [Config] + +Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from postgres server. + +See more about +the [Debezium's postgres Connector properties](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#postgresql-connector-properties) + +### format [Enum] + +Optional output format for Postgres CDC, valid enumerations are "DEFAULT"、"COMPATIBLE_DEBEZIUM_JSON". + +## Task Example + +### Simple + +> Support multi-table reading + +``` +env { + execution.parallelism = 1 + job.mode = "STREAMING" + checkpoint.interval = 5000 +} + +source { + Postgres-CDC { + base-url = "jdbc:postgresql://localhost:5432/postgres" + username = "postgres" + password = "postgres" + database-names = ["postgres"] + table-names = ["postgres.public.mysql_cdc_source_table"] + } +} + +transform { +} + +sink { + Console { + } +} + +``` + +### Support debezium-compatible format send to kafka + +> Must be used with kafka connector sink, see [compatible debezium format](../formats/cdc-compatible-debezium-json.md) for details + +## Changelog + +- Add PostgreSQL CDC Source Connector + +### next version + diff --git a/docs/en/connector-v2/source/Postgres-CDC.md b/docs/en/connector-v2/source/Postgres-CDC.md deleted file mode 100644 index b1b1b8fc181..00000000000 --- a/docs/en/connector-v2/source/Postgres-CDC.md +++ /dev/null @@ -1,190 +0,0 @@ -# Postgres CDC - -> Postgres CDC source connector - -## Description - -The Postgres CDC connector allows for reading snapshot data and incremental data from Postgres database. This document -describes how to set up the Postgres CDC connector to run SQL queries against Postgres databases. - -## Key features - -- [ ] [batch](../../concept/connector-v2-features.md) -- [x] [stream](../../concept/connector-v2-features.md) -- [x] [exactly-once](../../concept/connector-v2-features.md) -- [ ] [column projection](../../concept/connector-v2-features.md) -- [x] [parallelism](../../concept/connector-v2-features.md) -- [x] [support user-defined split](../../concept/connector-v2-features.md) - -## Options - -| name | type | required | default value | -|------------------------------------------------|----------|----------|---------------| -| base-url | String | Yes | - | -| username | String | Yes | - | -| password | String | Yes | - | -| database-names | List | Yes | - | -| table-names | List | Yes | - | -| startup.mode | Enum | No | INITIAL | -| startup.timestamp | Long | No | - | -| startup.specific-offset.file | String | No | - | -| startup.specific-offset.pos | Long | No | - | -| stop.mode | Enum | No | NEVER | -| stop.timestamp | Long | No | - | -| stop.specific-offset.file | String | No | - | -| stop.specific-offset.pos | Long | No | - | -| incremental.parallelism | Integer | No | 1 | -| snapshot.split.size | Integer | No | 8096 | -| snapshot.fetch.size | Integer | No | 1024 | -| server-time-zone | String | No | UTC | -| connect.timeout.ms | Duration | No | 30000 | -| connect.max-retries | Integer | No | 3 | -| connection.pool.size | Integer | No | 20 | -| chunk-key.even-distribution.factor.upper-bound | Double | No | 1000 | -| chunk-key.even-distribution.factor.lower-bound | Double | No | 0.05 | -| debezium.* | config | No | - | -| format | Enum | No | DEFAULT | -| common-options | | no | - | - -### username [String] - -Name of the database to use when connecting to the database server. - -### password [String] - -Password to use when connecting to the database server. - -### database-name [String] - -Database name of the database to monitor. - -### schmea-list [List] - -Schmea name of the database to monitor. - -### table-names [List] - -Table name of the database to monitor. The table name needs to include the schema name, for example: schema_name.table_name - -### startup.mode [Enum] - -Optional startup mode for postgres CDC consumer, valid enumerations are "initial", "earliest", "latest" and "specific". - -### startup.timestamp [Long] - -Start from the specified epoch timestamp (in milliseconds). - -**Note, This option is required when the "startup.mode" option used `'timestamp'`.** - -### startup.specific-offset.file [String] - -Start from the specified binlog file name. - -**Note, This option is required when the "startup.mode" option used `'specific'`.** - -### startup.specific-offset.pos [Long] - -Start from the specified binlog file position. - -**Note, This option is required when the "startup.mode" option used `'specific'`.** - -### stop.mode [Enum] - -Optional stop mode for postgres CDC consumer, valid enumerations are "never". - -### stop.timestamp [Long] - -Stop from the specified epoch timestamp (in milliseconds). - -**Note, This option is required when the "stop.mode" option used `'timestamp'`.** - -### stop.specific-offset.file [String] - -Stop from the specified binlog file name. - -**Note, This option is required when the "stop.mode" option used `'specific'`.** - -### stop.specific-offset.pos [Long] - -Stop from the specified binlog file position. - -**Note, This option is required when the "stop.mode" option used `'specific'`.** - -### incremental.parallelism [Integer] - -The number of parallel readers in the incremental phase. - -### snapshot.split.size [Integer] - -The split size (number of rows) of table snapshot, captured tables are split into multiple splits when read the snapshot -of table. - -### snapshot.fetch.size [Integer] - -The maximum fetch size for per poll when read table snapshot. - -### server-time-zone [String] - -The session time zone in database server. - -### connect.timeout.ms [long] - -The maximum time that the connector should wait after trying to connect to the database server before timing out. - -### connect.max-retries [Integer] - -The max retry times that the connector should retry to build database server connection. - -### connection.pool.size [Integer] - -The connection pool size. - -### debezium [Config] - -Pass-through Debezium's properties to Debezium Embedded Engine which is used to capture data changes from postgres server. - -See more about -the [Debezium's postgres Connector properties](https://debezium.io/documentation/reference/1.6/connectors/postgresql.html#postgresql-connector-properties) - -### format [Enum] - -Optional output format for Postgres CDC, valid enumerations are "DEFAULT"、"COMPATIBLE_DEBEZIUM_JSON". - -#### example - -```conf -source { - Postgres-CDC { - debezium { - snapshot.mode = "never" - decimal.handling.mode = "double" - } - } -} -``` - -### common options - -Source plugin common parameters, please refer to [Source Common Options](common-options.md) for details. - -## Example - -```Jdbc { -source { - Postgres-CDC { - base-url = "jdbc:postgresql://127.0.0.1:5432/postgres" - username = "postgres" - password = "postgres" - database-names = ["postgres"] - table-names = ["postgres.public.cdc_source_table"] - } - -} -``` - -## Changelog - -- Add Postgres CDC Source Connector - -### next version - diff --git a/plugin-mapping.properties b/plugin-mapping.properties index c3a97a51a5f..14dad95dfeb 100644 --- a/plugin-mapping.properties +++ b/plugin-mapping.properties @@ -115,4 +115,4 @@ seatunnel.source.AmazonSqs = connector-amazonsqs seatunnel.sink.AmazonSqs = connector-amazonsqs seatunnel.source.Paimon = connector-paimon seatunnel.sink.Paimon = connector-paimon -seatunnel.source.Postgres-CDC = connector-cdc-postgres +seatunnel.source.Postgres-CDC = connector-cdc-postgresql diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java deleted file mode 100644 index 905423470b1..00000000000 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresTypeUtils.java +++ /dev/null @@ -1,93 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; - -import org.apache.seatunnel.api.table.type.BasicType; -import org.apache.seatunnel.api.table.type.DecimalType; -import org.apache.seatunnel.api.table.type.LocalTimeType; -import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; -import org.apache.seatunnel.api.table.type.SeaTunnelDataType; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; - -import io.debezium.relational.Column; -import io.debezium.relational.Table; - -import java.sql.Types; -import java.util.List; - -public class PostgresTypeUtils { - private PostgresTypeUtils() {} - - public static SeaTunnelDataType convertFromColumn(Column column) { - switch (column.jdbcType()) { - case Types.CHAR: - case Types.VARCHAR: - case Types.NCHAR: - case Types.NVARCHAR: - case Types.STRUCT: - case Types.CLOB: - case Types.LONGVARCHAR: - case Types.LONGNVARCHAR: - return BasicType.STRING_TYPE; - case Types.BLOB: - return PrimitiveByteArrayType.INSTANCE; - case Types.INTEGER: - return BasicType.INT_TYPE; - case Types.SMALLINT: - case Types.TINYINT: - return BasicType.SHORT_TYPE; - case Types.BIGINT: - return BasicType.LONG_TYPE; - case Types.FLOAT: - case Types.REAL: - return BasicType.FLOAT_TYPE; - case Types.DOUBLE: - return BasicType.DOUBLE_TYPE; - case Types.NUMERIC: - case Types.DECIMAL: - return new DecimalType(column.length(), column.scale().orElse(0)); - case Types.TIMESTAMP: - return LocalTimeType.LOCAL_DATE_TIME_TYPE; - case Types.DATE: - return LocalTimeType.LOCAL_DATE_TYPE; - case Types.TIME: - return LocalTimeType.LOCAL_TIME_TYPE; - case Types.BOOLEAN: - case Types.BIT: - return BasicType.BOOLEAN_TYPE; - default: - throw new UnsupportedOperationException( - String.format( - "Don't support Postgres type '%s' yet, jdbcType:'%s'.", - column.typeName(), column.jdbcType())); - } - } - - public static SeaTunnelRowType convertFromTable(Table table) { - - List columns = table.columns(); - String[] fieldNames = columns.stream().map(Column::name).toArray(String[]::new); - - SeaTunnelDataType[] fieldTypes = - columns.stream() - .map(PostgresTypeUtils::convertFromColumn) - .toArray(SeaTunnelDataType[]::new); - - return new SeaTunnelRowType(fieldNames, fieldTypes); - } -} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/pom.xml similarity index 95% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/pom.xml index 540e29d3d61..19becb4ec3a 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/pom.xml @@ -25,11 +25,12 @@ connector-cdc ${revision} - connector-cdc-postgres - SeaTunnel : Connectors V2 : CDC : Postgres + connector-cdc-postgresql + SeaTunnel : Connectors V2 : CDC : PostgreSQL + org.apache.seatunnel connector-cdc-base @@ -64,6 +65,7 @@ connector-jdbc ${project.version} + diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresEventMetadataProvider.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java similarity index 99% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java index f42021fed37..2042ab8b32e 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresOffsetContext.java @@ -142,6 +142,9 @@ public boolean isSnapshotRunning() { return sourceInfo.isSnapshot(); } + @Override + public void markLastSnapshotRecord() {} + @Override public void preSnapshotStart() { sourceInfo.setSnapshot(SnapshotRecord.TRUE); @@ -355,11 +358,6 @@ public OffsetState asOffsetState() { sourceInfo.isSnapshot()); } - @Override - public void markLastSnapshotRecord() { - sourceInfo.setSnapshot(SnapshotRecord.LAST); - } - @Override public void event(DataCollectionId tableId, Instant instant) { sourceInfo.update(instant, (TableId) tableId); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresSchema.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/PostgresTaskContext.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/TypeRegistry.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java similarity index 100% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/io/debezium/connector/postgresql/connection/PostgresConnection.java diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfig.java similarity index 97% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfig.java index 401173b0874..fd5cbddcc99 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfig.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfig.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.config.StartupConfig; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfigFactory.java similarity index 96% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfigFactory.java index fcf613c8b6a..a5023052c27 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/config/PostgresSourceConfigFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/config/PostgresSourceConfigFactory.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config; import org.apache.seatunnel.api.configuration.ReadonlyConfig; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfigFactory; import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; import org.apache.seatunnel.connectors.cdc.debezium.EmbeddedDatabaseHistory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.option.PostgresOptions; import io.debezium.connector.postgresql.PostgresConnector; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/option/PostgresOptions.java similarity index 96% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/option/PostgresOptions.java index bd5cdd2291f..493fd8150b9 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/option/PostgresOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/option/PostgresOptions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.option; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.Options; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresDialect.java similarity index 73% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresDialect.java index 2538f6d1cc6..4e7e43f67ed 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresDialect.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresDialect.java @@ -15,40 +15,37 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; -import org.apache.seatunnel.connectors.cdc.base.config.SourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.ChunkSplitter; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.enumerator.PostgresChunkSplitter; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot.PostgresSnapshotFetchTask; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.wal.PostgresWalFetchTask; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresSchema; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.TableDiscoveryUtils; - -import io.debezium.config.Configuration; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.enumerator.PostgresChunkSplitter; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.PostgresSourceFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.snapshot.PostgresSnapshotFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.wal.PostgresWalFetchTask; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresSchema; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.TableDiscoveryUtils; + import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.RelationalDatabaseConnectorConfig; import io.debezium.relational.TableId; import io.debezium.relational.history.TableChanges; import java.sql.SQLException; import java.util.List; -import static org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresConnectionUtils.createPostgresConnection; +import static org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresConnectionUtils.createPostgresConnection; public class PostgresDialect implements JdbcDataSourceDialect { private static final long serialVersionUID = 1L; - private final SourceConfig sourceConfig; + private final PostgresSourceConfig sourceConfig; private transient PostgresSchema postgresSchema; @@ -69,9 +66,7 @@ public boolean isDataCollectionIdCaseSensitive(JdbcSourceConfig sourceConfig) { @Override public JdbcConnection openJdbcConnection(JdbcSourceConfig sourceConfig) { - RelationalDatabaseConnectorConfig dbzConnectorConfig = sourceConfig.getDbzConnectorConfig(); - Configuration jdbcConfig = dbzConnectorConfig.getJdbcConfig(); - return createPostgresConnection(jdbcConfig); + return createPostgresConnection(sourceConfig.getDbzConfiguration()); } @Override @@ -98,7 +93,7 @@ public List discoverDataCollections(JdbcSourceConfig sourceConfig) { @Override public TableChanges.TableChange queryTableSchema(JdbcConnection jdbc, TableId tableId) { if (postgresSchema == null) { - postgresSchema = new PostgresSchema(); + postgresSchema = new PostgresSchema(sourceConfig.getDbzConnectorConfig()); } return postgresSchema.getTableSchema(jdbc, tableId); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSource.java similarity index 92% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSource.java index 100fed9c0b6..5a5d7caafe1 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSource.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSource.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source; import org.apache.seatunnel.api.configuration.Option; import org.apache.seatunnel.api.configuration.ReadonlyConfig; @@ -39,8 +39,8 @@ import org.apache.seatunnel.connectors.cdc.debezium.DeserializeFormat; import org.apache.seatunnel.connectors.cdc.debezium.row.DebeziumJsonDeserializeSchema; import org.apache.seatunnel.connectors.cdc.debezium.row.SeaTunnelRowDebeziumDeserializeSchema; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffsetFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset.LsnOffsetFactory; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.psql.PostgresCatalogFactory; @@ -48,6 +48,7 @@ import lombok.NoArgsConstructor; import java.time.ZoneId; +import java.util.List; @NoArgsConstructor @AutoService(SeaTunnelSource.class) @@ -56,8 +57,10 @@ public class PostgresIncrementalSource extends IncrementalSource dataType) { - super(options, dataType); + ReadonlyConfig options, + SeaTunnelDataType dataType, + List catalogTables) { + super(options, dataType, catalogTables); } @Override diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSourceFactory.java similarity index 73% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSourceFactory.java index 9f600534192..506bc0e37f2 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresIncrementalSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresIncrementalSourceFactory.java @@ -15,38 +15,34 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source; import org.apache.seatunnel.api.configuration.util.OptionRule; import org.apache.seatunnel.api.source.SeaTunnelSource; import org.apache.seatunnel.api.source.SourceSplit; import org.apache.seatunnel.api.table.catalog.CatalogOptions; import org.apache.seatunnel.api.table.catalog.CatalogTable; +import org.apache.seatunnel.api.table.catalog.CatalogTableUtil; import org.apache.seatunnel.api.table.connector.TableSource; import org.apache.seatunnel.api.table.factory.Factory; -import org.apache.seatunnel.api.table.factory.SupportMultipleTable; -import org.apache.seatunnel.api.table.factory.TableFactoryContext; import org.apache.seatunnel.api.table.factory.TableSourceFactory; -import org.apache.seatunnel.api.table.type.MultipleRowType; +import org.apache.seatunnel.api.table.factory.TableSourceFactoryContext; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRow; -import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.cdc.base.option.JdbcSourceOptions; import org.apache.seatunnel.connectors.cdc.base.option.SourceOptions; import org.apache.seatunnel.connectors.cdc.base.option.StartupMode; import org.apache.seatunnel.connectors.cdc.base.option.StopMode; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.option.PostgresOptions; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.option.PostgresOptions; import org.apache.seatunnel.connectors.seatunnel.jdbc.catalog.JdbcCatalogOptions; import com.google.auto.service.AutoService; import java.io.Serializable; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; +import java.util.List; @AutoService(Factory.class) -public class PostgresIncrementalSourceFactory implements TableSourceFactory, SupportMultipleTable { +public class PostgresIncrementalSourceFactory implements TableSourceFactory { @Override public String factoryIdentifier() { return PostgresIncrementalSource.IDENTIFIER; @@ -94,28 +90,15 @@ public Class getSourceClass() { @Override public - TableSource createSource(TableFactoryContext context) { + TableSource createSource(TableSourceFactoryContext context) { return () -> { - SeaTunnelDataType dataType; - if (context.getCatalogTables().size() == 1) { - dataType = - context.getCatalogTables().get(0).getTableSchema().toPhysicalRowDataType(); - } else { - Map rowTypeMap = new HashMap<>(); - for (CatalogTable catalogTable : context.getCatalogTables()) { - rowTypeMap.put( - catalogTable.getTableId().toTablePath().toString(), - catalogTable.getTableSchema().toPhysicalRowDataType()); - } - dataType = new MultipleRowType(rowTypeMap); - } + List catalogTables = + CatalogTableUtil.getCatalogTables( + context.getOptions(), context.getClassLoader()); + SeaTunnelDataType dataType = + CatalogTableUtil.convertToMultipleRowType(catalogTables); return (SeaTunnelSource) - new PostgresIncrementalSource<>(context.getOptions(), dataType); + new PostgresIncrementalSource<>(context.getOptions(), dataType, catalogTables); }; } - - @Override - public Result applyTables(TableFactoryContext context) { - return Result.of(context.getCatalogTables(), Collections.emptyList()); - } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresPooledDataSourceFactory.java similarity index 95% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresPooledDataSourceFactory.java index e1cfa4e912e..a4ac840e4bd 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresPooledDataSourceFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresPooledDataSourceFactory.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.relational.connection.JdbcConnectionPoolFactory; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresSourceOptions.java similarity index 97% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresSourceOptions.java index 014467638f5..e34167e320e 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/PostgresSourceOptions.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/PostgresSourceOptions.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source; import org.apache.seatunnel.api.configuration.Options; import org.apache.seatunnel.api.configuration.SingleChoiceOption; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/enumerator/PostgresChunkSplitter.java similarity index 93% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/enumerator/PostgresChunkSplitter.java index 3e00c101c57..3ccfb6a466d 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/enumerator/PostgresChunkSplitter.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/enumerator/PostgresChunkSplitter.java @@ -15,15 +15,15 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.enumerator; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.enumerator; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; import org.apache.seatunnel.connectors.cdc.base.dialect.JdbcDataSourceDialect; import org.apache.seatunnel.connectors.cdc.base.source.enumerator.splitter.AbstractJdbcSourceChunkSplitter; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresTypeUtils; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresTypeUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresUtils; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Column; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffset.java similarity index 97% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffset.java index f9df4b8d878..6010c1da023 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffset.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffset.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffsetFactory.java similarity index 85% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffsetFactory.java index 3633f41130a..9a07dfe03e8 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/offset/LsnOffsetFactory.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/offset/LsnOffsetFactory.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; import org.apache.seatunnel.connectors.cdc.base.source.offset.OffsetFactory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfigFactory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresDialect; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfigFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.PostgresDialect; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresUtils; import io.debezium.connector.postgresql.SourceInfo; import io.debezium.connector.postgresql.connection.Lsn; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/PostgresSourceFetchTaskContext.java similarity index 96% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/PostgresSourceFetchTaskContext.java index b297db40f76..241f5fa1853 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/PostgresSourceFetchTaskContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/PostgresSourceFetchTaskContext.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.connectors.cdc.base.config.JdbcSourceConfig; @@ -27,9 +27,9 @@ import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; import org.apache.seatunnel.connectors.cdc.debezium.EmbeddedDatabaseHistory; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.config.PostgresSourceConfig; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.config.PostgresSourceConfig; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresUtils; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.source.SourceRecord; @@ -68,7 +68,7 @@ import java.util.ArrayList; import java.util.List; -import static org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresConnectionUtils.createPostgresConnection; +import static org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresConnectionUtils.createPostgresConnection; @Slf4j public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { @@ -96,8 +96,7 @@ public class PostgresSourceFetchTaskContext extends JdbcSourceFetchTaskContext { public PostgresSourceFetchTaskContext( JdbcSourceConfig sourceConfig, JdbcDataSourceDialect dataSourceDialect) { super(sourceConfig, dataSourceDialect); - this.dataConnection = - createPostgresConnection(sourceConfig.getDbzConnectorConfig().getJdbcConfig()); + this.dataConnection = createPostgresConnection(sourceConfig.getDbzConfiguration()); this.metadataProvider = new PostgresEventMetadataProvider(); } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotFetchTask.java similarity index 94% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotFetchTask.java index 0a1d8fa851b..db2bbdccf01 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotFetchTask.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotFetchTask.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.snapshot; import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; @@ -23,7 +23,7 @@ import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.PostgresSourceFetchTaskContext; import io.debezium.pipeline.spi.SnapshotResult; import lombok.extern.slf4j.Slf4j; @@ -81,8 +81,6 @@ public void execute(FetchTask.Context context) throws Exception { } return; } - final IncrementalSplit backfillBinlogSplit = - createBackFillWalSplit(changeEventSourceContext); final IncrementalSplit backfillSplit = createBackFillWalSplit(changeEventSourceContext); // optimization that skip the binlog read when the low watermark equals high diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotSplitReadTask.java similarity index 97% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotSplitReadTask.java index ac40981d5a7..e4f4be7807c 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/PostgresSnapshotSplitReadTask.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/PostgresSnapshotSplitReadTask.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.snapshot; import org.apache.seatunnel.connectors.cdc.base.relational.JdbcSourceEventDispatcher; import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; import org.apache.seatunnel.connectors.cdc.base.source.split.wartermark.WatermarkKind; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils.PostgresUtils; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils.PostgresUtils; import org.apache.kafka.connect.errors.ConnectException; @@ -161,7 +161,6 @@ private void createDataEvents(PostgreSqlSnapshotContext snapshotContext, TableId EventDispatcher.SnapshotReceiver snapshotReceiver = dispatcher.getSnapshotChangeEventReceiver(); log.debug("Snapshotting table {}", tableId); - // todo pg 的 schema 不包含database TableId newTableId = new TableId(null, tableId.schema(), tableId.table()); createDataEventsForTable( snapshotContext, snapshotReceiver, databaseSchema.tableFor(newTableId)); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java similarity index 90% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java index 9aaa3f2ded5..8161cc31df9 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/snapshot/SnapshotSplitChangeEventSourceContext.java @@ -15,10 +15,10 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.snapshot; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.snapshot; import org.apache.seatunnel.connectors.cdc.base.source.split.SnapshotSplit; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset.LsnOffset; import io.debezium.pipeline.source.spi.ChangeEventSource; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/wal/PostgresWalFetchTask.java similarity index 94% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/wal/PostgresWalFetchTask.java index 340eaf5311a..b2b70dfcca9 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/source/reader/wal/PostgresWalFetchTask.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/source/reader/wal/PostgresWalFetchTask.java @@ -15,12 +15,12 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.wal; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.wal; import org.apache.seatunnel.connectors.cdc.base.source.reader.external.FetchTask; import org.apache.seatunnel.connectors.cdc.base.source.split.IncrementalSplit; import org.apache.seatunnel.connectors.cdc.base.source.split.SourceSplitBase; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.reader.PostgresSourceFetchTaskContext; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.reader.PostgresSourceFetchTaskContext; import io.debezium.connector.postgresql.PostgresStreamingChangeEventSource; import io.debezium.pipeline.source.spi.ChangeEventSource; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresConnectionUtils.java similarity index 74% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresConnectionUtils.java index 304e8835c76..130c86f73b7 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresConnectionUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresConnectionUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils; import io.debezium.config.Configuration; import io.debezium.connector.postgresql.PostgresConnectorConfig; @@ -29,16 +29,16 @@ public class PostgresConnectionUtils { public static PostgresConnection createPostgresConnection(Configuration dbzConfiguration) { - PostgresConnection heartbeatConnection = new PostgresConnection(dbzConfiguration); - final Charset databaseCharset = heartbeatConnection.getDatabaseCharset(); + final PostgresConnectorConfig connectorConfig = + new PostgresConnectorConfig(dbzConfiguration); + PostgresConnection heartbeatConnection = + new PostgresConnection(connectorConfig.getJdbcConfig()); + final Charset databaseCharset = heartbeatConnection.getDatabaseCharset(); final PostgresConnection.PostgresValueConverterBuilder valueConverterBuilder = (typeRegistry) -> - PostgresValueConverter.of( - new PostgresConnectorConfig(dbzConfiguration), - databaseCharset, - typeRegistry); + PostgresValueConverter.of(connectorConfig, databaseCharset, typeRegistry); - return new PostgresConnection(dbzConfiguration, valueConverterBuilder); + return new PostgresConnection(connectorConfig.getJdbcConfig(), valueConverterBuilder); } } diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresSchema.java similarity index 85% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresSchema.java index cb0910a4782..625c7a57409 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresSchema.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresSchema.java @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils; import org.apache.seatunnel.common.utils.SeaTunnelException; +import io.debezium.connector.postgresql.PostgresConnectorConfig; import io.debezium.connector.postgresql.connection.PostgresConnection; import io.debezium.jdbc.JdbcConnection; import io.debezium.relational.Table; @@ -36,9 +37,11 @@ public class PostgresSchema { private final Map schemasByTableId; + private final PostgresConnectorConfig connectorConfig; - public PostgresSchema() { + public PostgresSchema(PostgresConnectorConfig connectorConfig) { this.schemasByTableId = new ConcurrentHashMap<>(); + this.connectorConfig = connectorConfig; } public TableChanges.TableChange getTableSchema(JdbcConnection jdbc, TableId tableId) { @@ -66,7 +69,12 @@ private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId ta try { postgresConnection.readSchema( - tables, tableId.catalog(), tableId.schema(), null, null, false); + tables, + tableId.catalog(), + tableId.schema(), + connectorConfig.getTableFilters().dataCollectionFilter(), + null, + false); Table table = tables.forTable(tableId); TableChanges.TableChange tableChange = new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table); diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresTypeUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresTypeUtils.java new file mode 100644 index 00000000000..e05fb214c49 --- /dev/null +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresTypeUtils.java @@ -0,0 +1,148 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils; + +import org.apache.seatunnel.api.table.type.ArrayType; +import org.apache.seatunnel.api.table.type.BasicType; +import org.apache.seatunnel.api.table.type.DecimalType; +import org.apache.seatunnel.api.table.type.LocalTimeType; +import org.apache.seatunnel.api.table.type.PrimitiveByteArrayType; +import org.apache.seatunnel.api.table.type.SeaTunnelDataType; + +import io.debezium.relational.Column; + +public class PostgresTypeUtils { + + public static final String PG_SMALLSERIAL = "SMALLSERIAL"; + public static final String PG_SERIAL = "SERIAL"; + public static final String PG_BIGSERIAL = "BIGSERIAL"; + public static final String PG_BYTEA = "BYTEA"; + + public static final String PG_BIT = "BIT"; + public static final String PG_BYTEA_ARRAY = "_BYTEA"; + public static final String PG_SMALLINT = "INT2"; + public static final String PG_SMALLINT_ARRAY = "_INT2"; + public static final String PG_INTEGER = "INT4"; + public static final String PG_INTEGER_ARRAY = "_INT4"; + public static final String PG_BIGINT = "INT8"; + public static final String PG_BIGINT_ARRAY = "_INT8"; + public static final String PG_REAL = "FLOAT4"; + public static final String PG_REAL_ARRAY = "_FLOAT4"; + public static final String PG_DOUBLE_PRECISION = "FLOAT8"; + public static final String PG_DOUBLE_PRECISION_ARRAY = "_FLOAT8"; + public static final String PG_NUMERIC = "NUMERIC"; + public static final String PG_NUMERIC_ARRAY = "_NUMERIC"; + public static final String PG_BOOLEAN = "BOOL"; + public static final String PG_BOOLEAN_ARRAY = "_BOOL"; + public static final String PG_TIMESTAMP = "TIMESTAMP"; + public static final String PG_TIMESTAMP_ARRAY = "_TIMESTAMP"; + public static final String PG_TIMESTAMPTZ = "TIMESTAMPTZ"; + public static final String PG_TIMESTAMPTZ_ARRAY = "_TIMESTAMPTZ"; + public static final String PG_DATE = "DATE"; + public static final String PG_DATE_ARRAY = "_DATE"; + public static final String PG_TIME = "TIME"; + public static final String PG_TIME_ARRAY = "_TIME"; + public static final String PG_TEXT = "TEXT"; + public static final String PG_TEXT_ARRAY = "_TEXT"; + public static final String PG_CHAR = "BPCHAR"; + public static final String PG_CHAR_ARRAY = "_BPCHAR"; + public static final String PG_CHARACTER = "CHARACTER"; + public static final String PG_CHARACTER_ARRAY = "_CHARACTER"; + public static final String PG_CHARACTER_VARYING = "VARCHAR"; + public static final String PG_CHARACTER_VARYING_ARRAY = "_VARCHAR"; + public static final String PG_INTERVAL = "INTERVAL"; + public static final String PG_GEOMETRY = "GEOMETRY"; + public static final String PG_GEOGRAPHY = "GEOGRAPHY"; + public static final String PG_JSON = "JSON"; + public static final String PG_JSONB = "JSONB"; + public static final String PG_XML = "XML"; + + private PostgresTypeUtils() {} + + public static SeaTunnelDataType convertFromColumn(Column column) { + String typeName = column.typeName().toUpperCase(); + switch (typeName) { + case PG_CHAR: + case PG_CHARACTER: + case PG_CHARACTER_VARYING: + case PG_TEXT: + case PG_INTERVAL: + case PG_GEOMETRY: + case PG_GEOGRAPHY: + case PG_JSON: + case PG_JSONB: + case PG_XML: + return BasicType.STRING_TYPE; + case PG_BOOLEAN: + return BasicType.BOOLEAN_TYPE; + case PG_BOOLEAN_ARRAY: + return ArrayType.BOOLEAN_ARRAY_TYPE; + case PG_BYTEA: + case PG_BIT: + return PrimitiveByteArrayType.INSTANCE; + case PG_BYTEA_ARRAY: + return ArrayType.BYTE_ARRAY_TYPE; + case PG_SMALLINT: + case PG_SMALLSERIAL: + case PG_INTEGER: + case PG_SERIAL: + return BasicType.INT_TYPE; + case PG_SMALLINT_ARRAY: + case PG_INTEGER_ARRAY: + return ArrayType.INT_ARRAY_TYPE; + case PG_BIGINT: + case PG_BIGSERIAL: + return BasicType.LONG_TYPE; + case PG_BIGINT_ARRAY: + return ArrayType.LONG_ARRAY_TYPE; + case PG_REAL: + return BasicType.FLOAT_TYPE; + case PG_REAL_ARRAY: + return ArrayType.FLOAT_ARRAY_TYPE; + case PG_DOUBLE_PRECISION: + return BasicType.DOUBLE_TYPE; + case PG_DOUBLE_PRECISION_ARRAY: + return ArrayType.DOUBLE_ARRAY_TYPE; + case PG_NUMERIC: + return new DecimalType(column.length(), column.scale().orElse(0)); + case PG_CHAR_ARRAY: + case PG_CHARACTER_ARRAY: + case PG_CHARACTER_VARYING_ARRAY: + case PG_TEXT_ARRAY: + return ArrayType.STRING_ARRAY_TYPE; + case PG_TIMESTAMP: + case PG_TIMESTAMPTZ: + return LocalTimeType.LOCAL_DATE_TIME_TYPE; + case PG_TIME: + return LocalTimeType.LOCAL_TIME_TYPE; + case PG_DATE: + return LocalTimeType.LOCAL_DATE_TYPE; + + case PG_TIMESTAMP_ARRAY: + case PG_NUMERIC_ARRAY: + case PG_TIMESTAMPTZ_ARRAY: + case PG_TIME_ARRAY: + case PG_DATE_ARRAY: + default: + throw new UnsupportedOperationException( + String.format( + "Don't support Postgres type '%s' yet, jdbcType:'%s'.", + column.typeName(), column.jdbcType())); + } + } +} diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresUtils.java similarity index 89% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresUtils.java index 4f57d717679..f91d2c7bd61 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/PostgresUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/PostgresUtils.java @@ -15,14 +15,14 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils; import org.apache.seatunnel.api.table.type.SeaTunnelDataType; import org.apache.seatunnel.api.table.type.SeaTunnelRowType; import org.apache.seatunnel.common.utils.SeaTunnelException; import org.apache.seatunnel.connectors.cdc.base.source.offset.Offset; import org.apache.seatunnel.connectors.cdc.base.utils.SourceRecordUtils; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.offset.LsnOffset; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.offset.LsnOffset; import org.apache.kafka.connect.source.SourceRecord; @@ -32,10 +32,13 @@ import io.debezium.relational.Column; import io.debezium.relational.Table; import io.debezium.relational.TableId; +import lombok.extern.slf4j.Slf4j; import java.sql.Connection; import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.Statement; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -45,6 +48,7 @@ import java.util.Optional; /** The utils for Postgres data source. */ +@Slf4j public class PostgresUtils { private PostgresUtils() {} @@ -134,6 +138,56 @@ public static Object[] sampleDataFromColumn( }); } + public static Object[] skipReadAndSortSampleData( + JdbcConnection jdbc, TableId tableId, String columnName, int inverseSamplingRate) + throws SQLException { + final String sampleQuery = + String.format("SELECT %s FROM %s", quote(columnName), quote(tableId)); + + Statement stmt = null; + ResultSet rs = null; + + List results = new ArrayList<>(); + try { + stmt = + jdbc.connection() + .createStatement( + ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); + + stmt.setFetchSize(Integer.MIN_VALUE); + rs = stmt.executeQuery(sampleQuery); + + int count = 0; + while (rs.next()) { + count++; + if (count % 100000 == 0) { + log.info("Processing row index: {}", count); + } + if (count % inverseSamplingRate == 0) { + results.add(rs.getObject(1)); + } + } + } finally { + if (rs != null) { + try { + rs.close(); + } catch (SQLException e) { + log.error("Failed to close ResultSet", e); + } + } + if (stmt != null) { + try { + stmt.close(); + } catch (SQLException e) { + log.error("Failed to close Statement", e); + } + } + } + Object[] resultsArray = results.toArray(); + Arrays.sort(resultsArray); + return resultsArray; + } + /** * Returns the next LSN to be read from the database. This is the LSN of the last record that * was read from the database. diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/TableDiscoveryUtils.java similarity index 98% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/TableDiscoveryUtils.java index 5528c09bf8b..61dd1e0f4cc 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgres/utils/TableDiscoveryUtils.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/main/java/org/apache/seatunnel/connectors/seatunnel/cdc/postgresql/utils/TableDiscoveryUtils.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.connectors.seatunnel.cdc.postgres.utils; +package org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java similarity index 91% rename from seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java rename to seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java index 931dd10a06a..8e513bb6607 100644 --- a/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgres/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java +++ b/seatunnel-connectors-v2/connector-cdc/connector-cdc-postgresql/src/test/java/org/apache/seatunnel/connectors/seatunnel/cdc/mysql/source/PostgresIncrementalSourceFactoryTest.java @@ -17,7 +17,7 @@ package org.apache.seatunnel.connectors.seatunnel.cdc.mysql.source; -import org.apache.seatunnel.connectors.seatunnel.cdc.postgres.source.PostgresIncrementalSourceFactory; +import org.apache.seatunnel.connectors.seatunnel.cdc.postgresql.source.PostgresIncrementalSourceFactory; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/seatunnel-connectors-v2/connector-cdc/pom.xml b/seatunnel-connectors-v2/connector-cdc/pom.xml index c54d1bdc066..54ff7b9bfd7 100644 --- a/seatunnel-connectors-v2/connector-cdc/pom.xml +++ b/seatunnel-connectors-v2/connector-cdc/pom.xml @@ -34,7 +34,7 @@ connector-cdc-mysql connector-cdc-sqlserver connector-cdc-mongodb - connector-cdc-postgres + connector-cdc-postgresql diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/pom.xml similarity index 90% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/pom.xml index 77497df8577..4bbe99d8757 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/pom.xml @@ -22,14 +22,14 @@ ${revision} - connector-cdc-postgres-e2e - SeaTunnel : E2E : Connector V2 : CDC Postgres + connector-cdc-postgresql-e2e + SeaTunnel : E2E : Connector V2 : CDC PostgreSQL org.apache.seatunnel - connector-cdc-postgres + connector-jdbc ${project.version} pom import @@ -41,7 +41,7 @@ org.apache.seatunnel - connector-cdc-postgres + connector-cdc-postgresql ${project.version} test diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgresql/PostgreSQLCDCIT.java similarity index 96% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgresql/PostgreSQLCDCIT.java index eb172569d1c..b29160b533f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgres/PostgresCDCIT.java +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/java/org/apache/seatunnel/e2e/connector/cdc/postgresql/PostgreSQLCDCIT.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.seatunnel.e2e.connector.cdc.postgres; +package org.apache.seatunnel.e2e.connector.cdc.postgresql; import org.apache.seatunnel.e2e.common.TestResource; import org.apache.seatunnel.e2e.common.TestSuiteBase; @@ -49,9 +49,9 @@ @Slf4j @DisabledOnContainer( value = {}, - type = {EngineType.SPARK, EngineType.FLINK}, - disabledReason = "Currently SPARK and FLINK do not support cdc") -public class PostgresCDCIT extends TestSuiteBase implements TestResource { + type = {EngineType.SPARK}, + disabledReason = "Currently SPARK do not support cdc") +public class PostgreSQLCDCIT extends TestSuiteBase implements TestResource { // postgres private static final String POSTGRES_HOST = "postgres_cdc_e2e"; @@ -132,7 +132,7 @@ public void startUp() throws ClassNotFoundException, InterruptedException { } @TestTemplate - public void testpostgresCdcCheckDataE2e(TestContainer container) + public void testPostgresCDCCheckDataE2e(TestContainer container) throws IOException, InterruptedException { CompletableFuture executeJobFuture = @@ -146,7 +146,7 @@ public void testpostgresCdcCheckDataE2e(TestContainer container) } return null; }); - await().atMost(6000000, TimeUnit.MILLISECONDS) + await().atMost(600000, TimeUnit.MILLISECONDS) .untilAsserted( () -> { log.info(querySql(SINK_SQL).toString()); @@ -158,7 +158,7 @@ public void testpostgresCdcCheckDataE2e(TestContainer container) upsertDeleteSourceTable(); // stream stage - await().atMost(6000000, TimeUnit.MILLISECONDS) + await().atMost(600000, TimeUnit.MILLISECONDS) .untilAsserted( () -> { Assertions.assertIterableEquals( @@ -299,9 +299,9 @@ private void upsertDeleteSourceTable() { + " 21,\n" + " 'Leblanc');\n"); - executeSql("DELETE FROM public.postgres_cdc_e2e_source_table where gid = 1"); + executeSql("UPDATE public.postgres_cdc_e2e_source_table SET name = 'test' where gid = 2"); - executeSql("UPDATE public.postgres_cdc_e2e_source_table SET name = 'test' where gid = 3"); + executeSql("DELETE FROM public.postgres_cdc_e2e_source_table"); } @Override diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/ddl/postgres_cdc.sql similarity index 98% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/ddl/postgres_cdc.sql index 092a3665718..ac8dc47ee3f 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/ddl/postgres_cdc.sql +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/ddl/postgres_cdc.sql @@ -63,9 +63,6 @@ CREATE TABLE IF NOT EXISTS public.postgres_cdc_e2e_sink_table ( name VARCHAR(255) NOT null ); -truncate public.postgres_cdc_e2e_source_table; -truncate public.postgres_cdc_e2e_sink_table; - INSERT INTO public.postgres_cdc_e2e_source_table (gid, text_col, varchar_col, diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/docker/postgres.cnf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/docker/postgres.cnf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/docker/postgres.cnf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf b/seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/postgres_cdc_to_postgres.conf similarity index 100% rename from seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgres-e2e/src/test/resources/postgres_cdc_to_postgres.conf rename to seatunnel-e2e/seatunnel-connector-v2-e2e/connector-cdc-postgresql-e2e/src/test/resources/postgres_cdc_to_postgres.conf diff --git a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml index 9987b2afe68..0d0d70f585c 100644 --- a/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml +++ b/seatunnel-e2e/seatunnel-connector-v2-e2e/pom.xml @@ -63,7 +63,7 @@ connector-pulsar-e2e connector-paimon-e2e connector-kudu-e2e - connector-cdc-postgres-e2e + connector-cdc-postgresql-e2e