JDBC SQL 连接器

Scan Source: Bounded Lookup Source: Sync Mode Sink: Batch Sink: Streaming Append & Upsert Mode

JDBC 连接器允许使用 JDBC 驱动向任意类型的关系型数据库读取或者写入数据。本文档描述了针对关系型数据库如何通过建立 JDBC 连接器来执行 SQL 查询。

如果在 DDL 中定义了主键,JDBC sink 将以 upsert 模式与外部系统交换 UPDATE/DELETE 消息;否则,它将以 append 模式与外部系统交换消息且不支持消费 UPDATE/DELETE 消息。

依赖

In order to use the JDBC connector the following dependencies are required for both projects using a build automation tool (such as Maven or SBT) and SQL Client with SQL JAR bundles.

Maven dependencySQL Client
  1. <dependency>
  2. <groupId>org.apache.flink</groupId>
  3. <artifactId>flink-connector-jdbc_2.11</artifactId>
  4. <version>1.13.0</version>
  5. </dependency>
Copied to clipboard!
Download

在连接到具体数据库时,也需要对应的驱动依赖,目前支持的驱动如下:

DriverGroup IdArtifact IdJAR
MySQLmysqlmysql-connector-java下载
PostgreSQLorg.postgresqlpostgresql下载
Derbyorg.apache.derbyderby下载

当前,JDBC 连接器和驱动不在 Flink 二进制发布包中,请参阅这里了解在集群上执行时何连接它们。

如何创建 JDBC 表

JDBC table 可以按如下定义:

  1. -- Flink SQL 中注册一张 MySQL 'users'
  2. CREATE TABLE MyUserTable (
  3. id BIGINT,
  4. name STRING,
  5. age INT,
  6. status BOOLEAN,
  7. PRIMARY KEY (id) NOT ENFORCED
  8. ) WITH (
  9. 'connector' = 'jdbc',
  10. 'url' = 'jdbc:mysql://localhost:3306/mydatabase',
  11. 'table-name' = 'users'
  12. );
  13. -- 从另一张表 "T" 将数据写入到 JDBC 表中
  14. INSERT INTO MyUserTable
  15. SELECT id, name, age, status FROM T;
  16. -- 查看 JDBC 表中的数据
  17. SELECT id, name, age, status FROM MyUserTable;
  18. -- JDBC 表在时态表关联中作为维表
  19. SELECT * FROM myTopic
  20. LEFT JOIN MyUserTable FOR SYSTEM_TIME AS OF myTopic.proctime
  21. ON myTopic.key = MyUserTable.id;

连接器参数

参数是否必填默认值类型描述
connector
必填(none)String指定使用什么类型的连接器,这里应该是‘jdbc’
url
必填(none)StringJDBC 数据库 url。
table-name
必填(none)String连接到 JDBC 表的名称。
driver
可选(none)String用于连接到此 URL 的 JDBC 驱动类名,如果不设置,将自动从 URL 中推导。
username
可选(none)StringJDBC 用户名。如果指定了 ‘username’‘password’ 中的任一参数,则两者必须都被指定。
password
可选(none)StringJDBC 密码。
connection.max-retry-timeout
可选60sDuration最大重试超时时间,以秒为单位且不应该小于 1 秒。
scan.partition.column
可选(none)String用于将输入进行分区的列名。请参阅下面的分区扫描部分了解更多详情。
scan.partition.num
可选(none)Integer分区数。
scan.partition.lower-bound
可选(none)Integer第一个分区的最小值。
scan.partition.upper-bound
可选(none)Integer最后一个分区的最大值。
scan.fetch-size
可选0Integer每次循环读取时应该从数据库中获取的行数。如果指定的值为 ‘0’,则该配置项会被忽略。
scan.auto-commit
可选trueBoolean在 JDBC 驱动程序上设置 auto-commit 标志, 它决定了每个语句是否在事务中自动提交。有些 JDBC 驱动程序,特别是 Postgres,可能需要将此设置为 false 以便流化结果。
lookup.cache.max-rows
可选(none)Integerlookup cache 的最大行数,若超过该值,则最老的行记录将会过期。 默认情况下,lookup cache 是未开启的。请参阅下面的 Lookup Cache 部分了解更多详情。
lookup.cache.ttl
可选(none)Durationlookup cache 中每一行记录的最大存活时间,若超过该时间,则最老的行记录将会过期。 默认情况下,lookup cache 是未开启的。请参阅下面的 Lookup Cache 部分了解更多详情。
lookup.max-retries
可选3Integer查询数据库失败的最大重试时间。
sink.buffer-flush.max-rows
可选100Integerflush 前缓存记录的最大值,可以设置为 ‘0’ 来禁用它。
sink.buffer-flush.interval
可选1sDurationflush 间隔时间,超过该时间后异步线程将 flush 数据。可以设置为 ‘0’ 来禁用它。注意, 为了完全异步地处理缓存的 flush 事件,可以将 ‘sink.buffer-flush.max-rows’ 设置为 ‘0’ 并配置适当的 flush 时间间隔。
sink.max-retries
可选3Integer写入记录到数据库失败后的最大重试次数。
sink.parallelism
可选(none)Integer用于定义 JDBC sink 算子的并行度。默认情况下,并行度是由框架决定:使用与上游链式算子相同的并行度。

特性

键处理

当写入数据到外部数据库时,Flink 会使用 DDL 中定义的主键。如果定义了主键,则连接器将以 upsert 模式工作,否则连接器将以 append 模式工作。

在 upsert 模式下,Flink 将根据主键判断插入新行或者更新已存在的行,这种方式可以确保幂等性。为了确保输出结果是符合预期的,推荐为表定义主键并且确保主键是底层数据库中表的唯一键或主键。在 append 模式下,Flink 会把所有记录解释为 INSERT 消息,如果违反了底层数据库中主键或者唯一约束,INSERT 插入可能会失败。

有关 PRIMARY KEY 语法的更多详细信息,请参见 CREATE TABLE DDL

分区扫描

为了在并行 Source task 实例中加速读取数据,Flink 为 JDBC table 提供了分区扫描的特性。

如果下述分区扫描参数中的任一项被指定,则下述所有的分区扫描参数必须都被指定。这些参数描述了在多个 task 并行读取数据时如何对表进行分区。 scan.partition.column 必须是相关表中的数字、日期或时间戳列。注意,scan.partition.lower-boundscan.partition.upper-bound 用于决定分区的起始位置和过滤表中的数据。如果是批处理作业,也可以在提交 flink 作业之前获取最大值和最小值。

  • scan.partition.column:输入用于进行分区的列名。
  • scan.partition.num:分区数。
  • scan.partition.lower-bound:第一个分区的最小值。
  • scan.partition.upper-bound:最后一个分区的最大值。

Lookup Cache

JDBC 连接器可以用在时态表关联中作为一个可 lookup 的 source (又称为维表),当前只支持同步的查找模式。

默认情况下,lookup cache 是未启用的,你可以设置 lookup.cache.max-rows and lookup.cache.ttl 参数来启用。

lookup cache 的主要目的是用于提高时态表关联 JDBC 连接器的性能。默认情况下,lookup cache 不开启,所以所有请求都会发送到外部数据库。 当 lookup cache 被启用时,每个进程(即 TaskManager)将维护一个缓存。Flink 将优先查找缓存,只有当缓存未查找到时才向外部数据库发送请求,并使用返回的数据更新缓存。 当缓存命中最大缓存行 lookup.cache.max-rows 或当行超过最大存活时间 lookup.cache.ttl 时,缓存中最老的行将被设置为已过期。 缓存中的记录可能不是最新的,用户可以将 lookup.cache.ttl 设置为一个更小的值以获得更好的刷新数据,但这可能会增加发送到数据库的请求数。所以要做好吞吐量和正确性之间的平衡。

幂等写入

如果在 DDL 中定义了主键,JDBC sink 将使用 upsert 语义而不是普通的 INSERT 语句。upsert 语义指的是如果底层数据库中存在违反唯一性约束,则原子地添加新行或更新现有行,这种方式确保了幂等性。

如果出现故障,Flink 作业会从上次成功的 checkpoint 恢复并重新处理,这可能导致在恢复过程中重复处理消息。强烈推荐使用 upsert 模式,因为如果需要重复处理记录,它有助于避免违反数据库主键约束和产生重复数据。

除了故障恢复场景外,数据源(kafka topic)也可能随着时间的推移自然地包含多个具有相同主键的记录,这使得 upsert 模式是用户期待的。

由于 upsert 没有标准的语法,因此下表描述了不同数据库的 DML 语法:

DatabaseUpsert Grammar
MySQLINSERT .. ON DUPLICATE KEY UPDATE ..
PostgreSQLINSERT .. ON CONFLICT .. DO UPDATE SET ..

Postgres 数据库作为 Catalog

JdbcCatalog 允许用户通过 JDBC 协议将 Flink 连接到关系数据库。

目前,PostgresCatalog 是 JDBC Catalog 的唯一实现,PostgresCatalog 只支持有限的 Catalog 方法,包括:

  1. // Postgres Catalog 支持的方法
  2. PostgresCatalog.databaseExists(String databaseName)
  3. PostgresCatalog.listDatabases()
  4. PostgresCatalog.getDatabase(String databaseName)
  5. PostgresCatalog.listTables(String databaseName)
  6. PostgresCatalog.getTable(ObjectPath tablePath)
  7. PostgresCatalog.tableExists(ObjectPath tablePath)

其他的 Catalog 方法现在还是不支持的。

PostgresCatalog 的使用

请参阅 Dependencies 部分了解如何配置 JDBC 连接器和 Postgres 驱动。

Postgres catalog 支持以下参数:

  • name:必填,catalog 的名称。
  • default-database:必填,默认要连接的数据库。
  • username:必填,Postgres 账户的用户名。
  • password:必填,账户的密码。
  • base-url:必填,应该符合 "jdbc:postgresql://<ip>:<port>" 的格式,同时这里不应该包含数据库名。

    SQL

  1. CREATE CATALOG mypg WITH(
  2. 'type' = 'jdbc',
  3. 'default-database' = '...',
  4. 'username' = '...',
  5. 'password' = '...',
  6. 'base-url' = '...'
  7. );
  8. USE CATALOG mypg;

Java

  1. EnvironmentSettings settings = EnvironmentSettings.newInstance().inStreamingMode().build();
  2. TableEnvironment tableEnv = TableEnvironment.create(settings);
  3. String name = "mypg";
  4. String defaultDatabase = "mydb";
  5. String username = "...";
  6. String password = "...";
  7. String baseUrl = "..."
  8. JdbcCatalog catalog = new JdbcCatalog(name, defaultDatabase, username, password, baseUrl);
  9. tableEnv.registerCatalog("mypg", catalog);
  10. // 设置 JdbcCatalog 为会话的当前 catalog
  11. tableEnv.useCatalog("mypg");

Scala

  1. val settings = EnvironmentSettings.newInstance().inStreamingMode().build()
  2. val tableEnv = TableEnvironment.create(settings)
  3. val name = "mypg"
  4. val defaultDatabase = "mydb"
  5. val username = "..."
  6. val password = "..."
  7. val baseUrl = "..."
  8. val catalog = new JdbcCatalog(name, defaultDatabase, username, password, baseUrl)
  9. tableEnv.registerCatalog("mypg", catalog)
  10. // 设置 JdbcCatalog 为会话的当前 catalog
  11. tableEnv.useCatalog("mypg")

Python

  1. from pyflink.table.catalog import JdbcCatalog
  2. environment_settings = EnvironmentSettings.new_instance().in_streaming_mode().use_blink_planner().build()
  3. t_env = TableEnvironment.create(environment_settings)
  4. name = "mypg"
  5. default_database = "mydb"
  6. username = "..."
  7. password = "..."
  8. base_url = "..."
  9. catalog = JdbcCatalog(name, default_database, username, password, base_url)
  10. t_env.register_catalog("mypg", catalog)
  11. # 设置 JdbcCatalog 为会话的当前 catalog
  12. t_env.use_catalog("mypg")

YAML

  1. execution:
  2. planner: blink
  3. ...
  4. current-catalog: mypg # 设置 JdbcCatalog 为会话的当前 catalog
  5. current-database: mydb
  6. catalogs:
  7. - name: mypg
  8. type: jdbc
  9. default-database: mydb
  10. username: ...
  11. password: ...
  12. base-url: ...

PostgresSQL 元空间映射

除了数据库之外,postgresSQL 还有一个额外的命名空间 schema。一个 Postgres 实例可以拥有多个数据库,每个数据库可以拥有多个 schema,其中一个 schema 默认名为 “public”,每个 schema 可以包含多张表。 在 Flink 中,当查询由 Postgres catalog 注册的表时,用户可以使用 schema_name.table_name 或只有 table_name,其中 schema_name 是可选的,默认值为 “public”。

因此,Flink Catalog 和 Postgres 之间的元空间映射如下:

Flink Catalog Metaspace StructurePostgres Metaspace Structure
catalog name (defined in Flink only)N/A
database namedatabase name
table name[schema_name.]table_name

Flink 中的 Postgres 表的完整路径应该是 "<catalog>.<db>.`<schema.table>`"。如果指定了 schema,请注意需要转义 <schema.table>

这里提供了一些访问 Postgres 表的例子:

  1. -- 扫描 'public' schema(即默认 schema)中的 'test_table' 表,schema 名称可以省略
  2. SELECT * FROM mypg.mydb.test_table;
  3. SELECT * FROM mydb.test_table;
  4. SELECT * FROM test_table;
  5. -- 扫描 'custom_schema' schema 中的 'test_table2' 表,
  6. -- 自定义 schema 不能省略,并且必须与表一起转义。
  7. SELECT * FROM mypg.mydb.`custom_schema.test_table2`
  8. SELECT * FROM mydb.`custom_schema.test_table2`;
  9. SELECT * FROM `custom_schema.test_table2`;

数据类型映射

Flink 支持连接到多个使用方言(dialect)的数据库,如 MySQL、PostgresSQL、Derby 等。其中,Derby 通常是用于测试目的。下表列出了从关系数据库数据类型到 Flink SQL 数据类型的类型映射,映射表可以使得在 Flink 中定义 JDBC 表更加简单。

MySQL typePostgreSQL typeFlink SQL type
TINYINTTINYINT
SMALLINT
TINYINT UNSIGNED
SMALLINT
INT2
SMALLSERIAL
SERIAL2
SMALLINT
INT
MEDIUMINT
SMALLINT UNSIGNED
INTEGER
SERIAL
INT
BIGINT
INT UNSIGNED
BIGINT
BIGSERIAL
BIGINT
BIGINT UNSIGNEDDECIMAL(20, 0)
BIGINTBIGINTBIGINT
FLOATREAL
FLOAT4
FLOAT
DOUBLE
DOUBLE PRECISION
FLOAT8
DOUBLE PRECISION
DOUBLE
NUMERIC(p, s)
DECIMAL(p, s)
NUMERIC(p, s)
DECIMAL(p, s)
DECIMAL(p, s)
BOOLEAN
TINYINT(1)
BOOLEANBOOLEAN
DATEDATEDATE
TIME [(p)]TIME [(p)] [WITHOUT TIMEZONE]TIME [(p)] [WITHOUT TIMEZONE]
DATETIME [(p)]TIMESTAMP [(p)] [WITHOUT TIMEZONE]TIMESTAMP [(p)] [WITHOUT TIMEZONE]
CHAR(n)
VARCHAR(n)
TEXT
CHAR(n)
CHARACTER(n)
VARCHAR(n)
CHARACTER VARYING(n)
TEXT
STRING
BINARY
VARBINARY
BLOB
BYTEABYTES
ARRAYARRAY