From 7ac892cb32742fde70c66c9ae102a021fc454497 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Thu, 28 Dec 2023 13:01:54 +0800 Subject: [PATCH 1/5] [FLINK-33929][jdbc-connector] Support string slicing parallel reading --- .../docs/connectors/datastream/jdbc.md | 54 ++++ docs/content.zh/docs/connectors/table/jdbc.md | 47 ++- .../docs/connectors/datastream/jdbc.md | 56 ++++ docs/content/docs/connectors/table/jdbc.md | 48 ++- .../flink/connector/jdbc/JdbcInputFormat.java | 12 +- .../databases/db2/dialect/Db2Dialect.java | 5 + .../databases/derby/dialect/DerbyDialect.java | 5 + .../oracle/dialect/OracleDialect.java | 5 + .../postgres/dialect/PostgresDialect.java | 9 +- .../sqlserver/dialect/SqlServerDialect.java | 5 + .../databases/trino/dialect/TrinoDialect.java | 5 + .../connector/jdbc/dialect/JdbcDialect.java | 11 + .../internal/options/JdbcReadOptions.java | 41 ++- .../CompositeJdbcParameterValuesProvider.java | 5 +- .../JdbcNumericBetweenParametersProvider.java | 38 ++- .../jdbc/table/JdbcDynamicTableFactory.java | 33 +- .../jdbc/table/JdbcDynamicTableSource.java | 32 +- .../jdbc/table/JdbcRowDataInputFormat.java | 42 ++- .../connector/jdbc/JdbcInputFormatTest.java | 181 ++++++----- .../table/Db2DynamicTableSourceITCase.java | 2 + .../table/DerbyDynamicTableSourceITCase.java | 6 +- .../table/MySqlDynamicTableSourceITCase.java | 14 +- .../table/OracleDynamicTableSourceITCase.java | 4 +- .../PostgresDynamicTableSourceITCase.java | 9 +- .../SqlServerDynamicTableSourceITCase.java | 12 +- .../table/TrinoDynamicTableSourceITCase.java | 2 + .../connector/jdbc/internal/JdbcFullTest.java | 2 +- .../NumericBetweenParametersProviderTest.java | 54 ++-- .../table/JdbcDynamicTableSourceITCase.java | 306 +++++++----------- .../table/JdbcRowDataInputFormatTest.java | 219 +++---------- .../jdbc/testutils/databases/DbName.java | 19 ++ .../jdbc/testutils/tables/TableBase.java | 16 +- .../jdbc/testutils/tables/TableBuilder.java | 11 +- .../jdbc/testutils/tables/TableRow.java | 9 +- .../tables/templates/BooksTable.java | 1 + 35 files changed, 779 insertions(+), 541 deletions(-) create mode 100644 flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java diff --git a/docs/content.zh/docs/connectors/datastream/jdbc.md b/docs/content.zh/docs/connectors/datastream/jdbc.md index a57cf0bbe..6447cd0ac 100644 --- a/docs/content.zh/docs/connectors/datastream/jdbc.md +++ b/docs/content.zh/docs/connectors/datastream/jdbc.md @@ -84,6 +84,60 @@ env.from_collection( env.execute() ``` +### 支持字符串分片读取 + + + + + + + + + + + + + + + + + + + + + + + +
Support String Split DatabaseSlice Read String Grammar
MySQLABS(MD5(`column`) % `number_of_partitions`)
OracleMOD(ORA_HASH(`column`) , `number_of_partitions`)
PostgreSQL(ABS(HASHTEXT(`column`)) % `number_of_partitions`)
MS SQL ServerABS(HASHBYTES('MD5', `column`) % `number_of_partitions`)
+ + + +```java + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + Serializable[][] queryParameters = new Long[3][1]; + queryParameters[0] = new Long[]{0L}; + queryParameters[1] = new Long[]{1L}; + queryParameters[2] = new Long[]{2L}; + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes); + + JdbcInputFormat jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat() + .setDrivername("com.mysql.cj.jdbc.Driver") + .setDBUrl("jdbc:mysql://localhost:3306/test") + .setUsername("root") + .setPassword("12345678") + .setPartitionColumnTypeString(true) // 当读取字符串分区键一定设置为true + .setQuery("select * from fake_source_sink where ABS(MD5( `name`) % 2 ) = ?") // 根据不同数据库改造SQL语法 + .setRowTypeInfo(rowTypeInfo).setParametersProvider(new JdbcGenericParameterValuesProvider(queryParameters)).finish(); + DataStreamSource input = env.createInput(jdbcInputFormat); + env.execute(); +``` + {{< /tab >}} {{< /tabs >}} diff --git a/docs/content.zh/docs/connectors/table/jdbc.md b/docs/content.zh/docs/connectors/table/jdbc.md index 20447e16d..672e914eb 100644 --- a/docs/content.zh/docs/connectors/table/jdbc.md +++ b/docs/content.zh/docs/connectors/table/jdbc.md @@ -346,12 +346,57 @@ ON myTopic.key = MyUserTable.id; 为了在并行 `Source` task 实例中加速读取数据,Flink 为 JDBC table 提供了分区扫描的特性。 如果下述分区扫描参数中的任一项被指定,则下述所有的分区扫描参数必须都被指定。这些参数描述了在多个 task 并行读取数据时如何对表进行分区。 -`scan.partition.column` 必须是相关表中的数字、日期或时间戳列。注意,`scan.partition.lower-bound` 和 `scan.partition.upper-bound` 用于决定分区的起始位置和过滤表中的数据。如果是批处理作业,也可以在提交 flink 作业之前获取最大值和最小值。 +`scan.partition.column` 必须是相关表中的数字、日期或时间戳列、字符串类型。注意,`scan.partition.lower-bound` 和 `scan.partition.upper-bound` 用于决定分区的起始位置和过滤表中的数据。如果是批处理作业,也可以在提交 flink 作业之前获取最大值和最小值。 - `scan.partition.column`:输入用于进行分区的列名。 - `scan.partition.num`:分区数。 - `scan.partition.lower-bound`:第一个分区的最小值。 - `scan.partition.upper-bound`:最后一个分区的最大值。 +#### 支持字符串分片读取 + + + + + + + + + + + + + + + + + + + + + + + +
Support String Split DatabaseSlice Read String Grammar
MySQLABS(MD5(`column`) % `number_of_partitions`)
OracleMOD(ORA_HASH(`column`) , `number_of_partitions`)
PostgreSQL(ABS(HASHTEXT(`column`)) % `number_of_partitions`)
MS SQL ServerABS(HASHBYTES('MD5', `column`) % `number_of_partitions`)
+ +```SQL +CREATE TABLE my_split_string_read_table ( + id int, + name STRING, + age INT, + email STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'connector' = 'jdbc', + 'url' = 'jdbc:mysql://localhost:3306/test', + 'table-name' = 'mysql_table_name', + 'username'='root', + 'password'='12345678', + 'scan.partition.column'='name', + 'scan.partition.num'='3', + 'scan.partition.lower-bound'='0', -- 分别获取到 ABS(MD5(`name`) % 3) 0,1,2 的全表数据分区数据3 + 'scan.partition.upper-bound'='2' +) +``` diff --git a/docs/content/docs/connectors/datastream/jdbc.md b/docs/content/docs/connectors/datastream/jdbc.md index 41e46fb86..df3ae0b0b 100644 --- a/docs/content/docs/connectors/datastream/jdbc.md +++ b/docs/content/docs/connectors/datastream/jdbc.md @@ -64,6 +64,62 @@ JdbcSink.sink( {{< /tab >}} {{< /tabs >}} + +#### 分片读取 字符串列 目前只支持 部分数据库 + + + + + + + + + + + + + + + + + + + + + + + +
Support String Split DatabaseSlice Read String Grammar
MySQLABS(MD5(`column`) % `number_of_partitions`)
OracleMOD(ORA_HASH(`column`) , `number_of_partitions`)
PostgreSQL(ABS(HASHTEXT(`column`)) % `number_of_partitions`)
MS SQL ServerABS(HASHBYTES('MD5', `column`) % `number_of_partitions`)
+ + +### Supports Fragment Read String + +```java + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + Serializable[][] queryParameters = new Long[3][1]; + queryParameters[0] = new Long[]{0L}; + queryParameters[1] = new Long[]{1L}; + queryParameters[2] = new Long[]{2L}; + + TypeInformation[] fieldTypes = new TypeInformation[]{ + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO, + BasicTypeInfo.INT_TYPE_INFO, + BasicTypeInfo.STRING_TYPE_INFO}; + RowTypeInfo rowTypeInfo = new RowTypeInfo(fieldTypes); + + JdbcInputFormat jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat() + .setDrivername("com.mysql.cj.jdbc.Driver") + .setDBUrl("jdbc:mysql://localhost:3306/test") + .setUsername("root") + .setPassword("12345678") + .setPartitionColumnTypeString(true) // When reading string partitioning key must be set to true + .setQuery("select * from fake_source_sink where ABS(MD5( `name`) % 2 ) = ?") // Modify SQL syntax for different databases + .setRowTypeInfo(rowTypeInfo).setParametersProvider(new JdbcGenericParameterValuesProvider(queryParameters)).finish(); + DataStreamSource input = env.createInput(jdbcInputFormat); + env.execute(); +``` + ### SQL DML statement and JDBC statement builder The sink builds one [JDBC prepared statement](https://docs.oracle.com/en/java/javase/11/docs/api/java.sql/java/sql/PreparedStatement.html) from a user-provider SQL string, e.g.: diff --git a/docs/content/docs/connectors/table/jdbc.md b/docs/content/docs/connectors/table/jdbc.md index c1c6936a3..de33e53df 100644 --- a/docs/content/docs/connectors/table/jdbc.md +++ b/docs/content/docs/connectors/table/jdbc.md @@ -357,13 +357,59 @@ See [CREATE TABLE DDL]({{< ref "docs/dev/table/sql/create" >}}#create-table) for To accelerate reading data in parallel `Source` task instances, Flink provides partitioned scan feature for JDBC table. All the following scan partition options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple tasks. -The `scan.partition.column` must be a numeric, date, or timestamp column from the table in question. Notice that `scan.partition.lower-bound` and `scan.partition.upper-bound` are used to decide the partition stride and filter the rows in table. If it is a batch job, it also doable to get the max and min value first before submitting the flink job. +The `scan.partition.column` must be a numeric, date, or timestamp or string column from the table in question. Notice that `scan.partition.lower-bound` and `scan.partition.upper-bound` are used to decide the partition stride and filter the rows in table. If it is a batch job, it also doable to get the max and min value first before submitting the flink job. - `scan.partition.column`: The column name used for partitioning the input. - `scan.partition.num`: The number of partitions. - `scan.partition.lower-bound`: The smallest value of the first partition. - `scan.partition.upper-bound`: The largest value of the last partition. +#### Support String Fragment Reading + + + + + + + + + + + + + + + + + + + + + + + +
Support String Split DatabaseSlice Read String Grammar
MySQLABS(MD5(`column`) % `number_of_partitions`)
OracleMOD(ORA_HASH(`column`) , `number_of_partitions`)
PostgreSQL(ABS(HASHTEXT(`column`)) % `number_of_partitions`)
MS SQL ServerABS(HASHBYTES('MD5', `column`) % `number_of_partitions`)
+ +```SQL +CREATE TABLE my_split_string_read_table ( + id int, + name STRING, + age INT, + email STRING, + PRIMARY KEY (id) NOT ENFORCED +) WITH ( + 'connector' = 'jdbc', + 'url' = 'jdbc:mysql://localhost:3306/test', + 'table-name' = 'mysql_table_name', + 'username'='root', + 'password'='12345678', + 'scan.partition.column'='name', + 'scan.partition.num'='3', + 'scan.partition.lower-bound'='0', -- Obtain ABS(MD5(' name ') % 3) 0,1,2 Full table data partition data 3 + 'scan.partition.upper-bound'='2' +) +``` + ### Lookup Cache JDBC connector can be used in temporal join as a lookup source (aka. dimension table). Currently, only sync lookup mode is supported. diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java index e99c464d6..c670e9ffe 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java @@ -121,6 +121,7 @@ public class JdbcInputFormat extends RichInputFormat protected boolean hasNext; protected Object[][] parameterValues; + protected boolean isPartitionColumnTypeString; public JdbcInputFormat() {} @@ -190,7 +191,8 @@ public void closeInputFormat() { public void open(InputSplit inputSplit) throws IOException { try { if (inputSplit != null && parameterValues != null) { - for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) { + int parameterLength = isPartitionColumnTypeString ? 1 : parameterValues[inputSplit.getSplitNumber()].length; + for (int i = 0; i < parameterLength; i++) { Object param = parameterValues[inputSplit.getSplitNumber()][i]; if (param instanceof String) { statement.setString(i + 1, (String) param); @@ -265,7 +267,7 @@ public void close() throws IOException { * Checks whether all data has been read. * * @return boolean value indication whether all data has been read. - * @throws IOException + * @throws IOException if an I/O error occurs */ @Override public boolean reachedEnd() throws IOException { @@ -393,6 +395,11 @@ public JdbcInputFormatBuilder setParametersProvider( return this; } + public JdbcInputFormatBuilder setPartitionColumnTypeString(boolean partitionColumnTypeString) { + format.isPartitionColumnTypeString = partitionColumnTypeString; + return this; + } + public JdbcInputFormatBuilder setRowTypeInfo(RowTypeInfo rowTypeInfo) { format.rowTypeInfo = rowTypeInfo; return this; @@ -425,6 +432,7 @@ public JdbcInputFormat finish() { if (format.parameterValues == null) { LOG.debug("No input splitting configured (data will be read with parallelism 1)."); } + return format; } } diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java index e617a9fcc..1da75c275 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java @@ -56,6 +56,11 @@ public String quoteIdentifier(String identifier) { return identifier; } + @Override + public String hashModForField(String fieldName, int numPartitions) { + throw new IllegalArgumentException("The Db2 database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + } + @Override public String dialectName() { return "Db2"; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java index dad028f07..eab837945 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java @@ -59,6 +59,11 @@ public String quoteIdentifier(String identifier) { return identifier; } + @Override + public String hashModForField(String fieldName, int numPartitions) { + throw new IllegalArgumentException("The Derby database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + } + @Override public String dialectName() { return "Derby"; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java index 1ed061fee..9762dfc0e 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java @@ -71,6 +71,11 @@ public String quoteIdentifier(String identifier) { return identifier; } + @Override + public String hashModForField(String fieldName, int numPartitions) { + return "MOD(ORA_HASH(" + quoteIdentifier(fieldName) + ")," + numPartitions + ")"; + } + @Override public Optional getUpsertStatement( String tableName, String[] fieldNames, String[] uniqueKeyFields) { diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java index d0924aee8..9699c67b1 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java @@ -24,7 +24,9 @@ import java.util.Optional; -/** JDBC dialect for PostgreSQL. */ +/** + * JDBC dialect for PostgreSQL. + */ @Internal public class PostgresDialect extends AbstractPostgresCompatibleDialect { @@ -40,6 +42,11 @@ public Optional defaultDriverName() { return Optional.of("org.postgresql.Driver"); } + @Override + public String hashModForField(String fieldName, int numPartitions) { + return "(ABS(HASHTEXT(" + quoteIdentifier(fieldName) + ")) % " + numPartitions + ")"; + } + @Override public String dialectName() { return "PostgreSQL"; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java index 91469a6b3..1a9fa75b3 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java @@ -67,6 +67,11 @@ public String quoteIdentifier(String identifier) { return identifier; } + @Override + public String hashModForField(String fieldName, int numPartitions) { + return "ABS(HASHBYTES('MD5', " + quoteIdentifier(fieldName) + ") % " + numPartitions + ")"; + } + @Override public Optional getUpsertStatement( String tableName, String[] fieldNames, String[] uniqueKeyFields) { diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java index 041d24c33..835abbcdc 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java @@ -64,6 +64,11 @@ public String dialectName() { return "Trino"; } + @Override + public String hashModForField(String fieldName, int numPartitions) { + throw new IllegalArgumentException("The Trino database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + } + @Override public String quoteIdentifier(String identifier) { return identifier; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java index 6cc6bbd57..88be55aed 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/JdbcDialect.java @@ -84,6 +84,17 @@ default Optional defaultDriverName() { */ String quoteIdentifier(String identifier); + /** + * Computes the Hash value of the string. + * + * @param fieldName jdbc partition reads column names. + * @param numPartitions Set the number of partitions to be read. + * @return the quoted identifier. + */ + default String hashModForField(String fieldName, int numPartitions) { + return " ABS(MD5(" + quoteIdentifier(fieldName) + ") % " + numPartitions + ")"; + } + /** * Constructs the dialects upsert statement if supported; such as MySQL's {@code DUPLICATE KEY * UPDATE}, or PostgreSQL's {@code ON CONFLICT... DO UPDATE SET..}. If supported, the returned diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java index 064eb2a48..c0ad5db30 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java @@ -22,11 +22,14 @@ import java.util.Objects; import java.util.Optional; -/** Options for the JDBC scan. */ +/** + * Options for the JDBC scan. + */ public class JdbcReadOptions implements Serializable { private final String query; private final String partitionColumnName; + private final Boolean isPartitionColumnTypeString; private final Long partitionLowerBound; private final Long partitionUpperBound; private final Integer numPartitions; @@ -37,6 +40,7 @@ public class JdbcReadOptions implements Serializable { private JdbcReadOptions( String query, String partitionColumnName, + Boolean isPartitionColumnTypeString, Long partitionLowerBound, Long partitionUpperBound, Integer numPartitions, @@ -44,6 +48,7 @@ private JdbcReadOptions( boolean autoCommit) { this.query = query; this.partitionColumnName = partitionColumnName; + this.isPartitionColumnTypeString = isPartitionColumnTypeString; this.partitionLowerBound = partitionLowerBound; this.partitionUpperBound = partitionUpperBound; this.numPartitions = numPartitions; @@ -60,6 +65,10 @@ public Optional getPartitionColumnName() { return Optional.ofNullable(partitionColumnName); } + public Optional getPartitionColumnTypeString() { + return Optional.ofNullable(isPartitionColumnTypeString); + } + public Optional getPartitionLowerBound() { return Optional.ofNullable(partitionLowerBound); } @@ -100,10 +109,13 @@ public boolean equals(Object o) { } } - /** Builder of {@link JdbcReadOptions}. */ + /** + * Builder of {@link JdbcReadOptions}. + */ public static class Builder { protected String query; protected String partitionColumnName; + protected Boolean isPartitionColumnTypeString; protected Long partitionLowerBound; protected Long partitionUpperBound; protected Integer numPartitions; @@ -111,25 +123,37 @@ public static class Builder { protected int fetchSize = 0; protected boolean autoCommit = true; - /** optional, SQL query statement for this JDBC source. */ + /** + * optional, SQL query statement for this JDBC source. + */ public Builder setQuery(String query) { this.query = query; return this; } - /** optional, name of the column used for partitioning the input. */ + /** + * optional, name of the column used for partitioning the input. + */ public Builder setPartitionColumnName(String partitionColumnName) { this.partitionColumnName = partitionColumnName; return this; } - /** optional, the smallest value of the first partition. */ + public void setPartitionColumnTypeString(Boolean partitionColumnTypeString) { + isPartitionColumnTypeString = partitionColumnTypeString; + } + + /** + * optional, the smallest value of the first partition. + */ public Builder setPartitionLowerBound(long partitionLowerBound) { this.partitionLowerBound = partitionLowerBound; return this; } - /** optional, the largest value of the last partition. */ + /** + * optional, the largest value of the last partition. + */ public Builder setPartitionUpperBound(long partitionUpperBound) { this.partitionUpperBound = partitionUpperBound; return this; @@ -153,7 +177,9 @@ public Builder setFetchSize(int fetchSize) { return this; } - /** optional, whether to set auto commit on the JDBC driver. */ + /** + * optional, whether to set auto commit on the JDBC driver. + */ public Builder setAutoCommit(boolean autoCommit) { this.autoCommit = autoCommit; return this; @@ -163,6 +189,7 @@ public JdbcReadOptions build() { return new JdbcReadOptions( query, partitionColumnName, + isPartitionColumnTypeString, partitionLowerBound, partitionUpperBound, numPartitions, diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java index eebeac5ab..29d4e34eb 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java @@ -28,11 +28,10 @@ public class CompositeJdbcParameterValuesProvider implements JdbcParameterValuesProvider { JdbcParameterValuesProvider a; JdbcParameterValuesProvider b; - public CompositeJdbcParameterValuesProvider( - JdbcParameterValuesProvider a, JdbcParameterValuesProvider b) { + JdbcParameterValuesProvider a, JdbcParameterValuesProvider b, boolean isPartitionColumnTypeString) { Preconditions.checkArgument( - a.getParameterValues().length == b.getParameterValues().length, + isPartitionColumnTypeString || a.getParameterValues().length == b.getParameterValues().length, "Both JdbcParameterValuesProvider should have the same length."); this.a = a; this.b = b; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java index 75356e501..e46d0d30b 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java @@ -38,33 +38,39 @@ *

You can take advantage of this class to automatically generate the parameters of the BETWEEN * clause, based on the passed constructor parameters. */ + @Experimental public class JdbcNumericBetweenParametersProvider implements JdbcParameterValuesProvider { private final long minVal; private final long maxVal; + private boolean isPartitionColumnTypeString; + private long batchSize; private int batchNum; /** * NumericBetweenParametersProviderJdbc constructor. * - * @param minVal the lower bound of the produced "from" values - * @param maxVal the upper bound of the produced "to" values + * @param minVal the lower bound of the produced "from" values + * @param maxVal the upper bound of the produced "to" values + * @param isPartitionColumnTypeString Whether to use string types as slices to read fields */ - public JdbcNumericBetweenParametersProvider(long minVal, long maxVal) { + public JdbcNumericBetweenParametersProvider(long minVal, long maxVal, boolean isPartitionColumnTypeString) { Preconditions.checkArgument(minVal <= maxVal, "minVal must not be larger than maxVal"); + Preconditions.checkArgument(!isPartitionColumnTypeString || (minVal < Integer.MAX_VALUE && maxVal < Integer.MAX_VALUE), "When using a string as a shard field to set the upper and lower bounds lowerBound, upperBound exceeds the INT storage range, and the current logic uses the hash for each value within each shard boundary value"); this.minVal = minVal; this.maxVal = maxVal; + this.isPartitionColumnTypeString = isPartitionColumnTypeString; } /** * NumericBetweenParametersProviderJdbc constructor. * * @param fetchSize the max distance between the produced from/to pairs - * @param minVal the lower bound of the produced "from" values - * @param maxVal the upper bound of the produced "to" values + * @param minVal the lower bound of the produced "from" values + * @param maxVal the upper bound of the produced "to" values */ public JdbcNumericBetweenParametersProvider(long fetchSize, long minVal, long maxVal) { Preconditions.checkArgument(minVal <= maxVal, "minVal must not be larger than maxVal"); @@ -87,6 +93,7 @@ public JdbcNumericBetweenParametersProvider ofBatchSize(long batchSize) { public JdbcNumericBetweenParametersProvider ofBatchNum(int batchNum) { Preconditions.checkArgument(batchNum > 0, "Batch number must be positive"); + Preconditions.checkArgument(!isPartitionColumnTypeString || (minVal >= 0), "Ensure that lowerBound is greater than or equal to 0 and lowerBound, upperBound, is less than or equal to the number of partitions"); long maxElemCount = (maxVal - minVal) + 1; if (batchNum > maxElemCount) { @@ -102,16 +109,21 @@ public Serializable[][] getParameterValues() { Preconditions.checkState( batchSize > 0, "Batch size and batch number must be positive. Have you called `ofBatchSize` or `ofBatchNum`?"); - + Serializable[][] parameters = new Serializable[batchNum][2]; long maxElemCount = (maxVal - minVal) + 1; long bigBatchNum = maxElemCount - (batchSize - 1) * batchNum; - - Serializable[][] parameters = new Serializable[batchNum][2]; - long start = minVal; - for (int i = 0; i < batchNum; i++) { - long end = start + batchSize - 1 - (i >= bigBatchNum ? 1 : 0); - parameters[i] = new Long[] {start, end}; - start = end + 1; + if (!isPartitionColumnTypeString) { + long start = minVal; + for (int i = 0; i < batchNum; i++) { + long end = start + batchSize - 1 - (i >= bigBatchNum ? 1 : 0); + parameters[i] = new Long[]{start, end}; + start = end + 1; + } + } else { + for (int i = 0; i <= maxVal - minVal; i++) { + Long hashValue = minVal + i; + parameters[i] = new Long[]{hashValue, hashValue}; + } } return parameters; } diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java index 3434f6030..83451f617 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java @@ -37,7 +37,9 @@ import org.apache.flink.table.factories.DynamicTableSourceFactory; import org.apache.flink.table.factories.FactoryUtil; import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -45,6 +47,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.HashSet; +import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -111,14 +114,15 @@ public DynamicTableSource createDynamicTableSource(Context context) { helper.validate(); validateConfigOptions(config, context.getClassLoader()); + DataType physicalRowDataType = context.getPhysicalRowDataType(); validateDataTypeWithJdbcDialect( - context.getPhysicalRowDataType(), config.get(URL), context.getClassLoader()); + physicalRowDataType, config.get(URL), context.getClassLoader()); return new JdbcDynamicTableSource( getJdbcOptions(helper.getOptions(), context.getClassLoader()), - getJdbcReadOptions(helper.getOptions()), + getJdbcReadOptions(helper.getOptions(), physicalRowDataType), helper.getOptions().get(LookupOptions.MAX_RETRIES), getLookupCache(config), - context.getPhysicalRowDataType()); + physicalRowDataType); } private static void validateDataTypeWithJdbcDialect( @@ -146,12 +150,17 @@ private InternalJdbcConnectionOptions getJdbcOptions( return builder.build(); } - private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig) { + private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig, DataType physicalRowDataType) { final Optional partitionColumnName = readableConfig.getOptional(SCAN_PARTITION_COLUMN); + RowType logicalType = (RowType) physicalRowDataType.getLogicalType(); + Map fieldTypeMap = logicalType.getFields().stream().collect(Collectors.toMap(rowField -> rowField.getName(), rowField -> rowField.getType())); final JdbcReadOptions.Builder builder = JdbcReadOptions.builder(); if (partitionColumnName.isPresent()) { - builder.setPartitionColumnName(partitionColumnName.get()); + String partitionName = partitionColumnName.get(); + Boolean isPartitionColumnTypeString = fieldTypeMap.get(partitionName) instanceof VarCharType ? true : false; + builder.setPartitionColumnName(partitionName); + builder.setPartitionColumnTypeString(isPartitionColumnTypeString); builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND)); builder.setPartitionUpperBound(readableConfig.get(SCAN_PARTITION_UPPER_BOUND)); builder.setNumPartitions(readableConfig.get(SCAN_PARTITION_NUM)); @@ -270,15 +279,15 @@ private void validateConfigOptions(ReadableConfig config, ClassLoader classLoade String jdbcUrl = config.get(URL); JdbcDialectLoader.load(jdbcUrl, classLoader); - checkAllOrNone(config, new ConfigOption[] {USERNAME, PASSWORD}); + checkAllOrNone(config, new ConfigOption[]{USERNAME, PASSWORD}); checkAllOrNone( config, - new ConfigOption[] { - SCAN_PARTITION_COLUMN, - SCAN_PARTITION_NUM, - SCAN_PARTITION_LOWER_BOUND, - SCAN_PARTITION_UPPER_BOUND + new ConfigOption[]{ + SCAN_PARTITION_COLUMN, + SCAN_PARTITION_NUM, + SCAN_PARTITION_LOWER_BOUND, + SCAN_PARTITION_UPPER_BOUND }); if (config.getOptional(SCAN_PARTITION_LOWER_BOUND).isPresent() @@ -296,7 +305,7 @@ private void validateConfigOptions(ReadableConfig config, ClassLoader classLoade } } - checkAllOrNone(config, new ConfigOption[] {LOOKUP_CACHE_MAX_ROWS, LOOKUP_CACHE_TTL}); + checkAllOrNone(config, new ConfigOption[]{LOOKUP_CACHE_MAX_ROWS, LOOKUP_CACHE_TTL}); if (config.get(LOOKUP_MAX_RETRIES) < 0) { throw new IllegalArgumentException( diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java index 48e1702b7..2af2aa800 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java @@ -137,25 +137,36 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon options.getTableName(), DataType.getFieldNames(physicalRowDataType).toArray(new String[0]), new String[0]); - final List predicates = new ArrayList(); + final List predicates = new ArrayList<>(); if (readOptions.getPartitionColumnName().isPresent()) { long lowerBound = readOptions.getPartitionLowerBound().get(); long upperBound = readOptions.getPartitionUpperBound().get(); int numPartitions = readOptions.getNumPartitions().get(); - Serializable[][] allPushdownParams = replicatePushdownParamsForN(numPartitions); + Serializable[][] allPushDownParams = replicatePushdownParamsForN(numPartitions); + // Get here whether the type of the external configuration partition key is a string + boolean isPartitionColumnTypeString = readOptions.getPartitionColumnTypeString().get().booleanValue(); JdbcParameterValuesProvider allParams = new CompositeJdbcParameterValuesProvider( - new JdbcNumericBetweenParametersProvider(lowerBound, upperBound) + new JdbcNumericBetweenParametersProvider(lowerBound, upperBound, isPartitionColumnTypeString) .ofBatchNum(numPartitions), - new JdbcGenericParameterValuesProvider(allPushdownParams)); + new JdbcGenericParameterValuesProvider(allPushDownParams), isPartitionColumnTypeString); builder.setParametersProvider(allParams); + // Set partition type + builder.setPartitionColumnTypeString(isPartitionColumnTypeString); - predicates.add( - dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) - + " BETWEEN ? AND ?"); + String generatePredicates; + if (readOptions.getPartitionColumnTypeString().isPresent() && readOptions.getPartitionColumnTypeString().get()) { + generatePredicates = + dialect.hashModForField(readOptions.getPartitionColumnName().get(), numPartitions) + " = ? "; + } else { + generatePredicates = + dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) + + " BETWEEN ? AND ?"; + } + predicates.add(generatePredicates); } else { builder.setParametersProvider( new JdbcGenericParameterValuesProvider(replicatePushdownParamsForN(1))); @@ -182,7 +193,6 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon builder.setRowConverter(dialect.getRowConverter(rowType)); builder.setRowDataTypeInfo( runtimeProviderContext.createTypeInformation(physicalRowDataType)); - return InputFormatProvider.of(builder.build()); } @@ -287,10 +297,10 @@ private Optional parseFilterToPredicate(ResolvedExpressi } private Serializable[][] replicatePushdownParamsForN(int n) { - Serializable[][] allPushdownParams = new Serializable[n][pushdownParams.length]; + Serializable[][] allPushDownParams = new Serializable[n][pushdownParams.length]; for (int i = 0; i < n; i++) { - allPushdownParams[i] = this.pushdownParams; + allPushDownParams[i] = this.pushdownParams; } - return allPushdownParams; + return allPushDownParams; } } diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java index 76452684a..d334faf5f 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java @@ -52,7 +52,9 @@ import java.sql.Timestamp; import java.util.Arrays; -/** InputFormat for {@link JdbcDynamicTableSource}. */ +/** + * InputFormat for {@link JdbcDynamicTableSource}. + */ @Internal public class JdbcRowDataInputFormat extends RichInputFormat implements ResultTypeQueryable { @@ -60,15 +62,16 @@ public class JdbcRowDataInputFormat extends RichInputFormat private static final long serialVersionUID = 2L; private static final Logger LOG = LoggerFactory.getLogger(JdbcRowDataInputFormat.class); - private JdbcConnectionProvider connectionProvider; - private int fetchSize; - private Boolean autoCommit; + private final JdbcConnectionProvider connectionProvider; + private final int fetchSize; + private final Boolean autoCommit; private Object[][] parameterValues; - private String queryTemplate; - private int resultSetType; - private int resultSetConcurrency; - private JdbcRowConverter rowConverter; - private TypeInformation rowDataTypeInfo; + private final boolean isPartitionColumnTypeString; + private final String queryTemplate; + private final int resultSetType; + private final int resultSetConcurrency; + private final JdbcRowConverter rowConverter; + private final TypeInformation rowDataTypeInfo; private transient PreparedStatement statement; private transient ResultSet resultSet; @@ -79,6 +82,7 @@ private JdbcRowDataInputFormat( int fetchSize, Boolean autoCommit, Object[][] parameterValues, + boolean isPartitionColumnTypeString, String queryTemplate, int resultSetType, int resultSetConcurrency, @@ -88,6 +92,7 @@ private JdbcRowDataInputFormat( this.fetchSize = fetchSize; this.autoCommit = autoCommit; this.parameterValues = parameterValues; + this.isPartitionColumnTypeString = isPartitionColumnTypeString; this.queryTemplate = queryTemplate; this.resultSetType = resultSetType; this.resultSetConcurrency = resultSetConcurrency; @@ -147,14 +152,15 @@ public void closeInputFormat() { * fashion otherwise. * * @param inputSplit which is ignored if this InputFormat is executed as a non-parallel source, - * a "hook" to the query parameters otherwise (using its splitNumber) + * a "hook" to the query parameters otherwise (using its splitNumber) * @throws IOException if there's an error during the execution of the query */ @Override public void open(InputSplit inputSplit) throws IOException { try { if (inputSplit != null && parameterValues != null) { - for (int i = 0; i < parameterValues[inputSplit.getSplitNumber()].length; i++) { + int parameterLength = isPartitionColumnTypeString ? 1 : parameterValues[inputSplit.getSplitNumber()].length; + for (int i = 0; i < parameterLength; i++) { Object param = parameterValues[inputSplit.getSplitNumber()][i]; if (param instanceof String) { statement.setString(i + 1, (String) param); @@ -273,7 +279,7 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { if (parameterValues == null) { - return new GenericInputSplit[] {new GenericInputSplit(0, 1)}; + return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; } GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; for (int i = 0; i < ret.length; i++) { @@ -296,9 +302,11 @@ public static Builder builder() { return new Builder(); } - /** Builder for {@link JdbcRowDataInputFormat}. */ + /** + * Builder for {@link JdbcRowDataInputFormat}. + */ public static class Builder { - private JdbcConnectionOptions.JdbcConnectionOptionsBuilder connOptionsBuilder; + private final JdbcConnectionOptions.JdbcConnectionOptionsBuilder connOptionsBuilder; private int fetchSize; private Boolean autoCommit; private Object[][] parameterValues; @@ -307,6 +315,7 @@ public static class Builder { private TypeInformation rowDataTypeInfo; private int resultSetType = ResultSet.TYPE_FORWARD_ONLY; private int resultSetConcurrency = ResultSet.CONCUR_READ_ONLY; + private boolean isPartitionColumnTypeString = false; public Builder() { this.connOptionsBuilder = new JdbcConnectionOptions.JdbcConnectionOptionsBuilder(); @@ -376,6 +385,10 @@ public Builder setResultSetConcurrency(int resultSetConcurrency) { return this; } + public void setPartitionColumnTypeString(boolean partitionColumnTypeString) { + isPartitionColumnTypeString = partitionColumnTypeString; + } + public JdbcRowDataInputFormat build() { if (this.queryTemplate == null) { throw new NullPointerException("No query supplied"); @@ -391,6 +404,7 @@ public JdbcRowDataInputFormat build() { this.fetchSize, this.autoCommit, this.parameterValues, + this.isPartitionColumnTypeString, this.queryTemplate, this.resultSetType, this.resultSetConcurrency, diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java index f72434db7..5c1cbe2e3 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java @@ -44,7 +44,9 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Tests for the {@link JdbcInputFormat}. */ +/** + * Tests for the {@link JdbcInputFormat}. + */ class JdbcInputFormatTest extends JdbcDataTestBase { private JdbcInputFormat jdbcInputFormat; @@ -58,18 +60,53 @@ void tearDown() throws IOException { jdbcInputFormat = null; } + @Test + void testJdbcInputFormatspiltKeyString() throws IOException { + final int fetchSize = 1; + final long min = TEST_DATA[0].id; + final long max = TEST_DATA[TEST_DATA.length - fetchSize].id; + JdbcParameterValuesProvider pramProvider = + new JdbcNumericBetweenParametersProvider(min, max, true).ofBatchSize(fetchSize); + // check string The number of input slices for the scan boundary + + Serializable[][] correctParameters = new Serializable[10][2]; + correctParameters[0] = new Long[]{min, min}; + correctParameters[1] = new Long[]{min + 1, min + 1}; + correctParameters[2] = new Long[]{min + 2, min + 2}; + correctParameters[3] = new Long[]{min + 3, min + 3}; + correctParameters[4] = new Long[]{min + 4, min + 4}; + correctParameters[5] = new Long[]{min + 5, min + 5}; + correctParameters[6] = new Long[]{min + 6, min + 6}; + correctParameters[7] = new Long[]{min + 7, min + 7}; + correctParameters[8] = new Long[]{min + 8, min + 8}; + correctParameters[9] = new Long[]{min + 9, min + 9}; + + assertThat(pramProvider.getParameterValues().length).isEqualTo(max - min + 1); + assertThat(pramProvider.getParameterValues()).isEqualTo(correctParameters); + + // Test lower boundary values equal + JdbcParameterValuesProvider singleProvider = + new JdbcNumericBetweenParametersProvider(1, 1, true).ofBatchSize(fetchSize); + // check string The number of input slices for the scan boundary + + Serializable[][] singleCorrectParameters = new Serializable[1][2]; + singleCorrectParameters[0] = new Long[]{1L, 1L}; + + assertThat(singleProvider.getParameterValues()).isEqualTo(singleCorrectParameters); + } + @Test void testUntypedRowInfo() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("No RowTypeInfo supplied"); } @@ -77,62 +114,62 @@ void testUntypedRowInfo() { @Test void testInvalidDriver() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername("org.apache.derby.jdbc.idontexist") + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidURL() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidQuery() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery("iamnotsql") - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery("iamnotsql") + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testNoUrl() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("jdbc url is empty"); } @@ -140,14 +177,14 @@ void testNoUrl() { @Test void testNoQuery() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("No query supplied"); } @@ -155,16 +192,16 @@ void testNoQuery() { @Test void testInvalidFetchSize() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .setFetchSize(-7) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .setFetchSize(-7) + .finish(); + }) .isInstanceOf(IllegalArgumentException.class); } @@ -192,8 +229,8 @@ void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException { jdbcInputFormat.openInputFormat(); try (Connection dbConn = getMetadata().getConnection(); - Statement dbStatement = dbConn.createStatement(); - Statement inputStatement = jdbcInputFormat.getStatement()) { + Statement dbStatement = dbConn.createStatement(); + Statement inputStatement = jdbcInputFormat.getStatement()) { assertThat(inputStatement.getFetchSize()).isEqualTo(dbStatement.getFetchSize()); } } @@ -282,7 +319,7 @@ void testJdbcInputFormatWithParallelismAndNumericColumnSplitting() throws IOExce final long min = TEST_DATA[0].id; final long max = TEST_DATA[TEST_DATA.length - fetchSize].id; JdbcParameterValuesProvider pramProvider = - new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); + new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat() .setDrivername(getMetadata().getDriverClass()) @@ -320,7 +357,7 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE final long max = TEST_DATA[TEST_DATA.length - 1].id; final long fetchSize = max + 1; // generate a single split JdbcParameterValuesProvider pramProvider = - new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); + new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat() .setDrivername(getMetadata().getDriverClass()) @@ -355,8 +392,8 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE @Test void testJdbcInputFormatWithParallelismAndGenericSplitting() throws IOException { Serializable[][] queryParameters = new String[2][1]; - queryParameters[0] = new String[] {TEST_DATA[3].author}; - queryParameters[1] = new String[] {TEST_DATA[0].author}; + queryParameters[0] = new String[]{TEST_DATA[3].author}; + queryParameters[1] = new String[]{TEST_DATA[0].author}; JdbcParameterValuesProvider paramProvider = new JdbcGenericParameterValuesProvider(queryParameters); jdbcInputFormat = diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/db2/table/Db2DynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/db2/table/Db2DynamicTableSourceITCase.java index d92217a2f..0eafb4693 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/db2/table/Db2DynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/db2/table/Db2DynamicTableSourceITCase.java @@ -32,6 +32,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.flink.connector.jdbc.testutils.databases.DbName.DB2_DB; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.dbType; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; @@ -44,6 +45,7 @@ public class Db2DynamicTableSourceITCase extends JdbcDynamicTableSourceITCase protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DB2_DB, field("id", dbType("BIGINT"), DataTypes.BIGINT().notNull()), field("decimal_col", dbType("NUMERIC(10, 4)"), DataTypes.DECIMAL(10, 4)), field("timestamp6_col", dbType("TIMESTAMP(6)"), DataTypes.TIMESTAMP(6)), diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java index fa1d67350..878298026 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java @@ -31,17 +31,21 @@ import java.util.Arrays; import java.util.List; +import static org.apache.flink.connector.jdbc.testutils.databases.DbName.DERBY_DB; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.dbType; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; -/** The Table Source ITCase for {@link DerbyDialect}. */ +/** + * The Table Source ITCase for {@link DerbyDialect}. + */ public class DerbyDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements DerbyTestBase { protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DERBY_DB, field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", dbType("TIMESTAMP"), DataTypes.TIMESTAMP(6)), diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java index 967c7e841..61041284a 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.jdbc.databases.mysql.MySqlTestBase; import org.apache.flink.connector.jdbc.databases.mysql.dialect.MySqlDialect; import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.table.api.DataTypes; import org.apache.flink.types.Row; @@ -35,7 +36,9 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; -/** The Table Source ITCase for {@link MySqlDialect}. */ +/** + * The Table Source ITCase for {@link MySqlDialect}. + */ public class MySqlDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements MySqlTestBase { @@ -43,6 +46,7 @@ public class MySqlDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DbName.MYSQL_DB, field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", DataTypes.TIMESTAMP(6)), @@ -50,7 +54,8 @@ protected TableRow createInputTable() { field("real_col", dbType("REAL"), DataTypes.DOUBLE()), field("double_col", DataTypes.DOUBLE()), field("time_col", dbType("TIME"), DataTypes.TIME()), - field("timestamp9_col", DataTypes.TIMESTAMP(6))); + field("timestamp9_col", DataTypes.TIMESTAMP(6)), + field("string_col", DataTypes.VARCHAR(30))); } protected List getTestData() { @@ -62,7 +67,8 @@ protected List getTestData() { 1.175E-37D, 1.79769E308D, LocalTime.parse("15:35"), - LocalDateTime.parse("2020-01-01T15:35:00.123456")), + LocalDateTime.parse("2020-01-01T15:35:00.123456"), + "Leblanc_1"), Row.of( 2L, BigDecimal.valueOf(101.1234), @@ -70,6 +76,6 @@ protected List getTestData() { -1.175E-37D, -1.79769E308, LocalTime.parse("15:36:01"), - LocalDateTime.parse("2020-01-01T15:36:01.123456"))); + LocalDateTime.parse("2020-01-01T15:36:01.123456"), "Wade")); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSourceITCase.java index 9e758eeeb..b79c11a78 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSourceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.jdbc.databases.oracle.OracleTestBase; import org.apache.flink.connector.jdbc.databases.oracle.dialect.OracleDialect; import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.table.api.DataTypes; import org.apache.flink.types.Row; @@ -43,6 +44,7 @@ public class OracleDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DbName.ORACLE_DB, field("id", dbType("INTEGER"), DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", dbType("TIMESTAMP"), DataTypes.TIMESTAMP(6)), @@ -53,7 +55,7 @@ protected TableRow createInputTable() { field("binary_double_col", dbType("BINARY_DOUBLE"), DataTypes.DOUBLE()), field("char_col", dbType("CHAR"), DataTypes.CHAR(1)), field("nchar_col", dbType("NCHAR(3)"), DataTypes.VARCHAR(3)), - field("varchar2_col", dbType("VARCHAR2(30)"), DataTypes.VARCHAR(30)), + field("string_col", dbType("VARCHAR2(30)"), DataTypes.VARCHAR(30)), field("date_col", dbType("DATE"), DataTypes.DATE()), field("timestamp9_col", dbType("TIMESTAMP(9)"), DataTypes.TIMESTAMP(9)), field("clob_col", dbType("CLOB"), DataTypes.STRING())); diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java index 174be59d8..920a14d60 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.jdbc.databases.postgres.PostgresTestBase; import org.apache.flink.connector.jdbc.databases.postgres.dialect.PostgresDialect; import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.table.api.DataTypes; import org.apache.flink.types.Row; @@ -43,13 +44,15 @@ public class PostgresDynamicTableSourceITCase extends JdbcDynamicTableSourceITCa protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DbName.POSTGRES_DB, field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", DataTypes.TIMESTAMP(6)), // other fields field("real_col", dbType("REAL"), DataTypes.FLOAT()), field("double_col", dbType("DOUBLE PRECISION"), DataTypes.DOUBLE()), - field("time_col", dbType("TIME"), DataTypes.TIME())); + field("time_col", dbType("TIME"), DataTypes.TIME()), + field("string_col", DataTypes.VARCHAR(30))); } protected List getTestData() { @@ -60,13 +63,13 @@ protected List getTestData() { LocalDateTime.parse("2020-01-01T15:35:00.123456"), 1.175E-37F, 1.79769E308D, - LocalTime.parse("15:35")), + LocalTime.parse("15:35"), "Leblanc_1"), Row.of( 2L, BigDecimal.valueOf(101.1234), LocalDateTime.parse("2020-01-01T15:36:01.123456"), -1.175E-37F, -1.79769E308, - LocalTime.parse("15:36:01"))); + LocalTime.parse("15:36:01"), "Wade")); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java index f9444f45c..c51b74aee 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.jdbc.databases.sqlserver.SqlServerTestBase; import org.apache.flink.connector.jdbc.databases.sqlserver.dialect.SqlServerDialect; import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.table.api.DataTypes; import org.apache.flink.types.Row; @@ -39,7 +40,9 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** The Table Source ITCase for {@link SqlServerDialect}. */ +/** + * The Table Source ITCase for {@link SqlServerDialect}. + */ public class SqlServerDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements SqlServerTestBase { @@ -47,6 +50,7 @@ public class SqlServerDynamicTableSourceITCase extends JdbcDynamicTableSourceITC protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DbName.SQL_SERVER_DB, field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", dbType("DATETIME2"), DataTypes.TIMESTAMP(6)), @@ -63,7 +67,7 @@ protected TableRow createInputTable() { field("datetime2_col", dbType("DATETIME2"), DataTypes.TIMESTAMP()), field("char_col", dbType("CHAR"), DataTypes.STRING()), field("nchar_col", dbType("NCHAR(3)"), DataTypes.STRING()), - field("varchar2_col", dbType("VARCHAR(30)"), DataTypes.STRING()), + field("string_col", dbType("VARCHAR(30)"), DataTypes.STRING()), field("nvarchar2_col", dbType("NVARCHAR(30)"), DataTypes.STRING()), field("text_col", dbType("TEXT"), DataTypes.STRING()), field("ntext_col", dbType("NTEXT"), DataTypes.STRING())); @@ -87,7 +91,7 @@ protected List getTestData() { LocalDateTime.parse("2020-01-01T15:35:00.1234567"), "a", "abc", - "abcdef", + "Leblanc_1", "xyz", "Hello World", "World Hello"), @@ -110,7 +114,7 @@ protected List getTestData() { "abcdef", "xyz", "Hey Leonard", - "World Hello")); + "Wade")); } @Test diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/trino/table/TrinoDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/trino/table/TrinoDynamicTableSourceITCase.java index 39b6c3c5c..40b164e04 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/trino/table/TrinoDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/trino/table/TrinoDynamicTableSourceITCase.java @@ -21,6 +21,7 @@ import org.apache.flink.connector.jdbc.databases.trino.TrinoTestBase; import org.apache.flink.connector.jdbc.databases.trino.dialect.TrinoDialect; import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.table.api.DataTypes; import org.apache.flink.types.Row; @@ -47,6 +48,7 @@ class TrinoDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase impleme protected TableRow createInputTable() { return tableRow( "jdbDynamicTableSource", + DbName.TRINO_DB, field("id", dbType("INTEGER"), DataTypes.BIGINT()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", DataTypes.TIMESTAMP(6)), diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcFullTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcFullTest.java index f79e7f7eb..5c34f282e 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcFullTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcFullTest.java @@ -124,7 +124,7 @@ private void runTest(boolean exploitParallelism) throws Exception { inputBuilder .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID) .setParametersProvider( - new JdbcNumericBetweenParametersProvider(min, max) + new JdbcNumericBetweenParametersProvider(min, max, false) .ofBatchSize(fetchSize)); } DataSet source = environment.createInput(inputBuilder.finish()); diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java index 25e350e1a..1825ce2bf 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java @@ -24,13 +24,15 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Tests for {@link JdbcNumericBetweenParametersProvider}. */ +/** + * Tests for {@link JdbcNumericBetweenParametersProvider}. + */ class NumericBetweenParametersProviderTest { @Test void testBatchSizeDivisible() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(-5, 9).ofBatchSize(3); + new JdbcNumericBetweenParametersProvider(-5, 9, false).ofBatchSize(3); Serializable[][] actual = provider.getParameterValues(); long[][] expected = { @@ -46,15 +48,15 @@ void testBatchSizeDivisible() { @Test void testBatchSizeNotDivisible() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(-5, 11).ofBatchSize(4); + new JdbcNumericBetweenParametersProvider(-5, 11, false).ofBatchSize(4); Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[] {-5, -2}, - new long[] {-1, 2}, - new long[] {3, 5}, - new long[] {6, 8}, - new long[] {9, 11} + new long[]{-5, -2}, + new long[]{-1, 2}, + new long[]{3, 5}, + new long[]{6, 8}, + new long[]{9, 11} }; check(expected, actual); } @@ -62,25 +64,25 @@ void testBatchSizeNotDivisible() { @Test void testBatchSizeTooLarge() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(0, 2).ofBatchSize(5); + new JdbcNumericBetweenParametersProvider(0, 2, false).ofBatchSize(5); Serializable[][] actual = provider.getParameterValues(); - long[][] expected = {new long[] {0, 2}}; + long[][] expected = {new long[]{0, 2}}; check(expected, actual); } @Test void testBatchNumDivisible() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(-5, 9).ofBatchNum(5); + new JdbcNumericBetweenParametersProvider(-5, 9, false).ofBatchNum(5); Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[] {-5, -3}, - new long[] {-2, 0}, - new long[] {1, 3}, - new long[] {4, 6}, - new long[] {7, 9} + new long[]{-5, -3}, + new long[]{-2, 0}, + new long[]{1, 3}, + new long[]{4, 6}, + new long[]{7, 9} }; check(expected, actual); } @@ -88,15 +90,15 @@ void testBatchNumDivisible() { @Test void testBatchNumNotDivisible() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(-5, 11).ofBatchNum(5); + new JdbcNumericBetweenParametersProvider(-5, 11, false).ofBatchNum(5); Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[] {-5, -2}, - new long[] {-1, 2}, - new long[] {3, 5}, - new long[] {6, 8}, - new long[] {9, 11} + new long[]{-5, -2}, + new long[]{-1, 2}, + new long[]{3, 5}, + new long[]{6, 8}, + new long[]{9, 11} }; check(expected, actual); } @@ -104,13 +106,13 @@ void testBatchNumNotDivisible() { @Test void testBatchNumTooLarge() { JdbcNumericBetweenParametersProvider provider = - new JdbcNumericBetweenParametersProvider(0, 2).ofBatchNum(5); + new JdbcNumericBetweenParametersProvider(0, 2, false).ofBatchNum(5); Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[] {0, 0}, - new long[] {1, 1}, - new long[] {2, 2} + new long[]{0, 0}, + new long[]{1, 1}, + new long[]{2, 2} }; check(expected, actual); } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java index 51f75e34a..930b9738d 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java @@ -18,9 +18,18 @@ package org.apache.flink.connector.jdbc.table; +import org.apache.flink.api.java.ExecutionEnvironment; import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.jdbc.JdbcInputFormat; +import org.apache.flink.connector.jdbc.databases.mysql.dialect.MySqlDialect; +import org.apache.flink.connector.jdbc.databases.oracle.dialect.OracleDialect; +import org.apache.flink.connector.jdbc.databases.postgres.dialect.PostgresDialect; +import org.apache.flink.connector.jdbc.databases.sqlserver.dialect.SqlServerDialect; +import org.apache.flink.connector.jdbc.split.JdbcGenericParameterValuesProvider; +import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; import org.apache.flink.connector.jdbc.testutils.DatabaseTest; import org.apache.flink.connector.jdbc.testutils.TableManaged; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.tables.TableRow; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -47,12 +56,14 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import java.io.Serializable; import java.math.BigDecimal; import java.sql.Connection; import java.sql.SQLException; import java.time.LocalDateTime; import java.time.temporal.ChronoUnit; import java.time.temporal.TemporalUnit; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -65,28 +76,30 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; import static org.assertj.core.api.Assertions.assertThat; -/** ITCase for {@link JdbcDynamicTableSource}. */ +/** + * ITCase for {@link JdbcDynamicTableSource}. + */ public abstract class JdbcDynamicTableSourceITCase implements DatabaseTest { @RegisterExtension - static final MiniClusterExtension MINI_CLUSTER_RESOURCE = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setConfiguration(new Configuration()) - .build()); + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension(new MiniClusterResourceConfiguration.Builder().setNumberTaskManagers(1).setConfiguration(new Configuration()).build()); private final TableRow inputTable = createInputTable(); public static StreamExecutionEnvironment env; public static TableEnvironment tEnv; + private static final List dbNameList = new ArrayList<>(); + + static { + dbNameList.add(DbName.POSTGRES_DB); + dbNameList.add(DbName.MYSQL_DB); + dbNameList.add(DbName.ORACLE_DB); + dbNameList.add(DbName.SQL_SERVER_DB); + } + protected TableRow createInputTable() { - return tableRow( - "jdbDynamicTableSource", - field("id", DataTypes.BIGINT().notNull()), - field("decimal_col", DataTypes.DECIMAL(10, 4)), - field("timestamp6_col", DataTypes.TIMESTAMP(6))); + return tableRow("jdbDynamicTableSource", field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", DataTypes.TIMESTAMP(6))); } @Override @@ -95,15 +108,7 @@ public List getManagedTables() { } protected List getTestData() { - return Arrays.asList( - Row.of( - 1L, - BigDecimal.valueOf(100.1234), - truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456"))), - Row.of( - 2L, - BigDecimal.valueOf(101.1234), - truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); + return Arrays.asList(Row.of(1L, BigDecimal.valueOf(100.1234), truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456"))), Row.of(2L, BigDecimal.valueOf(101.1234), truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); } @BeforeEach @@ -122,34 +127,83 @@ void afterEach() { @Test void testJdbcSource() { + String fieldStr = "string_col"; + String fieldId = "id"; String testTable = "testTable"; - tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable)); + boolean isShardContainString = Arrays.asList(inputTable.getTableFields()).contains(fieldId); + if (!isShardContainString) { + throw new IllegalArgumentException("The data column in the jdbc table test must contain the `id`"); + } + boolean isPartitionColumnTypeString = (dbNameList.contains(createInputTable().getDbName()) && isShardContainString); + DatabaseMetadata metadata = getMetadata(); + // Non slice read + tEnv.executeSql(inputTable.getCreateQueryForFlink(metadata, testTable)); List collected = executeQuery("SELECT * FROM " + testTable); - assertThat(collected).containsExactlyInAnyOrderElementsOf(getTestData()); + + String testReadPartitionString = "testReadPartitionString"; + + //string slice read + List withParams = new ArrayList<>(); + + String partitionColumn = Arrays.asList(inputTable.getTableFields()).contains(fieldStr) ? fieldStr : fieldId; + int partitionNum = 10; + int lowerBound = 0; + int upperBound = 9; + withParams.add(String.format("'scan.partition.column'='%s'", partitionColumn)); + withParams.add(String.format("'scan.partition.num'='%s'", partitionNum)); + withParams.add(String.format("'scan.partition.lower-bound'='%s'", lowerBound)); + withParams.add(String.format("'scan.partition.upper-bound'='%s'", upperBound)); + + tEnv.executeSql(inputTable.getCreateQueryForFlink(metadata, testReadPartitionString, withParams)); + List collectedPartitionString = executeQuery("SELECT * FROM " + testReadPartitionString); + assertThat(collectedPartitionString).containsExactlyInAnyOrderElementsOf(getTestData()); + + Serializable[][] queryParameters = new Long[3][1]; + queryParameters[0] = new Long[]{0L}; + queryParameters[1] = new Long[]{1L}; + queryParameters[2] = new Long[]{2L}; + + String partitionKeyName = isPartitionColumnTypeString ? fieldStr : fieldId; + String sqlFilterField; + switch (createInputTable().getDbName()) { + case POSTGRES_DB: + sqlFilterField = new PostgresDialect().hashModForField(partitionKeyName, queryParameters.length); + break; + case MYSQL_DB: + sqlFilterField = new MySqlDialect().hashModForField(partitionKeyName, queryParameters.length); + break; + case ORACLE_DB: + sqlFilterField = new OracleDialect().hashModForField(partitionKeyName, queryParameters.length); + break; + case SQL_SERVER_DB: + sqlFilterField = new SqlServerDialect().hashModForField(partitionKeyName, queryParameters.length); + break; + default: + sqlFilterField = partitionKeyName; + break; + } + + ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment(); + JdbcInputFormat jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat().setDrivername(metadata.getDriverClass()).setDBUrl(metadata.getJdbcUrl()).setUsername(metadata.getUsername()).setPassword(metadata.getPassword()).setPartitionColumnTypeString(isPartitionColumnTypeString).setQuery("select * from " + inputTable.getTableName() + " where ( " + sqlFilterField + " ) = ?").setRowTypeInfo(inputTable.getTableRowTypeInfo()).setParametersProvider(new JdbcGenericParameterValuesProvider(queryParameters)).finish(); + try { + int jdbcInputRowSize = executionEnvironment.createInput(jdbcInputFormat).map(row -> row.getField(fieldId)).collect().size(); + assertThat(jdbcInputRowSize).isEqualTo(getTestData().size()); + } catch (Exception e) { + throw new RuntimeException(e); + } } @Test void testProject() { String testTable = "testTable"; - tEnv.executeSql( - inputTable.getCreateQueryForFlink( - getMetadata(), - testTable, - Arrays.asList( - "'scan.partition.column'='id'", - "'scan.partition.num'='2'", - "'scan.partition.lower-bound'='0'", - "'scan.partition.upper-bound'='100'"))); + tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='2'", "'scan.partition.lower-bound'='0'", "'scan.partition.upper-bound'='100'"))); String fields = String.join(",", Arrays.copyOfRange(inputTable.getTableFields(), 0, 3)); List collected = executeQuery(String.format("SELECT %s FROM %s", fields, testTable)); - List expected = - getTestData().stream() - .map(row -> Row.of(row.getField(0), row.getField(1), row.getField(2))) - .collect(Collectors.toList()); + List expected = getTestData().stream().map(row -> Row.of(row.getField(0), row.getField(1), row.getField(2))).collect(Collectors.toList()); assertThat(collected).containsExactlyInAnyOrderElementsOf(expected); } @@ -157,22 +211,12 @@ void testProject() { @Test public void testLimit() { String testTable = "testTable"; - tEnv.executeSql( - inputTable.getCreateQueryForFlink( - getMetadata(), - testTable, - Arrays.asList( - "'scan.partition.column'='id'", - "'scan.partition.num'='2'", - "'scan.partition.lower-bound'='1'", - "'scan.partition.upper-bound'='2'"))); + tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='2'", "'scan.partition.lower-bound'='1'", "'scan.partition.upper-bound'='2'"))); List collected = executeQuery("SELECT * FROM " + testTable + " LIMIT 1"); assertThat(collected).hasSize(1); - assertThat(getTestData()) - .as("The actual output is not a subset of the expected set.") - .containsAll(collected); + assertThat(getTestData()).as("The actual output is not a subset of the expected set.").containsAll(collected); } @Test @@ -182,95 +226,42 @@ public void testFilter() { // create a partitioned table to ensure no regression String partitionedTable = "PARTITIONED_TABLE"; - tEnv.executeSql( - inputTable.getCreateQueryForFlink( - getMetadata(), - partitionedTable, - Arrays.asList( - "'scan.partition.column'='id'", - "'scan.partition.num'='1'", - "'scan.partition.lower-bound'='1'", - "'scan.partition.upper-bound'='1'"))); + tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), partitionedTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='1'", "'scan.partition.lower-bound'='1'", "'scan.partition.upper-bound'='1'"))); // we create a VIEW here to test column remapping, ie. would filter push down work if we // create a view that depends on our source table - tEnv.executeSql( - String.format( - "CREATE VIEW FAKE_TABLE (idx, %s) as (SELECT * from %s )", - Arrays.stream(inputTable.getTableFields()) - .filter(f -> !f.equals("id")) - .collect(Collectors.joining(",")), - testTable)); - - Row onlyRow1 = - getTestData().stream() - .filter(row -> row.getFieldAs(0).equals(1L)) - .findAny() - .orElseThrow(NullPointerException::new); - - Row onlyRow2 = - getTestData().stream() - .filter(row -> row.getFieldAs(0).equals(2L)) - .findAny() - .orElseThrow(NullPointerException::new); + tEnv.executeSql(String.format("CREATE VIEW FAKE_TABLE (idx, %s) as (SELECT * from %s )", Arrays.stream(inputTable.getTableFields()).filter(f -> !f.equals("id")).collect(Collectors.joining(",")), testTable)); + + Row onlyRow1 = getTestData().stream().filter(row -> row.getFieldAs(0).equals(1L)).findAny().orElseThrow(NullPointerException::new); + + Row onlyRow2 = getTestData().stream().filter(row -> row.getFieldAs(0).equals(2L)).findAny().orElseThrow(NullPointerException::new); List twoRows = getTestData(); // test simple filter - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1")) - .containsExactly(onlyRow1); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1")).containsExactly(onlyRow1); // test TIMESTAMP filter - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE " - + "WHERE timestamp6_col > TIMESTAMP '2020-01-01 15:35:00'" - + " AND timestamp6_col < TIMESTAMP '2020-01-01 15:35:01'")) - .containsExactly(onlyRow1); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE " + "WHERE timestamp6_col > TIMESTAMP '2020-01-01 15:35:00'" + " AND timestamp6_col < TIMESTAMP '2020-01-01 15:35:01'")).containsExactly(onlyRow1); // test the IN operator - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE WHERE 1 = idx AND decimal_col IN (100.1234, 101.1234)")) - .containsExactly(onlyRow1); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE 1 = idx AND decimal_col IN (100.1234, 101.1234)")).containsExactly(onlyRow1); // test mixing AND and OR operator - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE WHERE idx = 1 AND decimal_col = 100.1234 OR decimal_col = 101.1234")) - .containsExactlyInAnyOrderElementsOf(twoRows); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1 AND decimal_col = 100.1234 OR decimal_col = 101.1234")).containsExactlyInAnyOrderElementsOf(twoRows); // test mixing AND/OR with parenthesis, and the swapping the operand of equal expression - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE WHERE (2 = idx AND decimal_col = 100.1234) OR decimal_col = 101.1234")) - .containsExactly(onlyRow2); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE (2 = idx AND decimal_col = 100.1234) OR decimal_col = 101.1234")).containsExactly(onlyRow2); // test Greater than, just to make sure we didnt break anything that we cannot pushdown - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE WHERE idx = 2 AND decimal_col > 100 OR decimal_col = 101.123")) - .containsExactly(onlyRow2); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 2 AND decimal_col > 100 OR decimal_col = 101.123")).containsExactly(onlyRow2); // One more test of parenthesis - assertThat( - executeQuery( - "SELECT * FROM FAKE_TABLE WHERE 2 = idx AND (decimal_col = 100.1234 OR decimal_col = 102.1234)")) - .isEmpty(); - - assertThat( - executeQuery( - "SELECT * FROM " - + partitionedTable - + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123")) - .isEmpty(); - - assertThat( - executeQuery( - "SELECT * FROM " - + partitionedTable - + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)")) - .containsExactly(onlyRow1); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE 2 = idx AND (decimal_col = 100.1234 OR decimal_col = 102.1234)")).isEmpty(); + + assertThat(executeQuery("SELECT * FROM " + partitionedTable + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123")).isEmpty(); + + assertThat(executeQuery("SELECT * FROM " + partitionedTable + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)")).containsExactly(onlyRow1); } @ParameterizedTest @@ -279,32 +270,13 @@ void testLookupJoin(Caching caching) { // Create JDBC lookup table List cachingOptions = Collections.emptyList(); if (caching.equals(Caching.ENABLE_CACHE)) { - cachingOptions = - Arrays.asList( - "'lookup.cache.max-rows' = '100'", "'lookup.cache.ttl' = '10min'"); + cachingOptions = Arrays.asList("'lookup.cache.max-rows' = '100'", "'lookup.cache.ttl' = '10min'"); } - tEnv.executeSql( - inputTable.getCreateQueryForFlink(getMetadata(), "jdbc_lookup", cachingOptions)); + tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), "jdbc_lookup", cachingOptions)); // Create and prepare a value source - String dataId = - TestValuesTableFactory.registerData( - Arrays.asList( - Row.of(1L, "Alice"), - Row.of(1L, "Alice"), - Row.of(2L, "Bob"), - Row.of(3L, "Charlie"))); - tEnv.executeSql( - String.format( - "CREATE TABLE value_source ( " - + " `id` BIGINT, " - + " `name` STRING, " - + " `proctime` AS PROCTIME()" - + ") WITH (" - + " 'connector' = 'values', " - + " 'data-id' = '%s'" - + ")", - dataId)); + String dataId = TestValuesTableFactory.registerData(Arrays.asList(Row.of(1L, "Alice"), Row.of(1L, "Alice"), Row.of(2L, "Bob"), Row.of(3L, "Charlie"))); + tEnv.executeSql(String.format("CREATE TABLE value_source ( " + " `id` BIGINT, " + " `name` STRING, " + " `proctime` AS PROCTIME()" + ") WITH (" + " 'connector' = 'values', " + " 'data-id' = '%s'" + ")", dataId)); if (caching == Caching.ENABLE_CACHE) { LookupCacheManager.keepCacheOnRelease(true); @@ -312,37 +284,13 @@ void testLookupJoin(Caching caching) { // Execute lookup join try { - List collected = - executeQuery( - "SELECT S.id, S.name, D.id, D.timestamp6_col, D.decimal_col FROM value_source" - + " AS S JOIN jdbc_lookup for system_time as of S.proctime AS D ON S.id = D.id"); + List collected = executeQuery("SELECT S.id, S.name, D.id, D.timestamp6_col, D.decimal_col FROM value_source" + " AS S JOIN jdbc_lookup for system_time as of S.proctime AS D ON S.id = D.id"); assertThat(collected).hasSize(3); - List expected = - Arrays.asList( - Row.of( - 1L, - "Alice", - 1L, - truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), - BigDecimal.valueOf(100.1234)), - Row.of( - 1L, - "Alice", - 1L, - truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), - BigDecimal.valueOf(100.1234)), - Row.of( - 2L, - "Bob", - 2L, - truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")), - BigDecimal.valueOf(101.1234))); - - assertThat(collected) - .as("The actual output is not a subset of the expected set") - .containsAll(expected); + List expected = Arrays.asList(Row.of(1L, "Alice", 1L, truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), BigDecimal.valueOf(100.1234)), Row.of(1L, "Alice", 1L, truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), BigDecimal.valueOf(100.1234)), Row.of(2L, "Bob", 2L, truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")), BigDecimal.valueOf(101.1234))); + + assertThat(collected).as("The actual output is not a subset of the expected set").containsAll(expected); if (caching == Caching.ENABLE_CACHE) { validateCachedValues(); @@ -370,26 +318,15 @@ private List executeQuery(String query) { private void validateCachedValues() { // Validate cache - Map managedCaches = - LookupCacheManager.getInstance().getManagedCaches(); + Map managedCaches = LookupCacheManager.getInstance().getManagedCaches(); assertThat(managedCaches).as("There should be only 1 shared cache registered").hasSize(1); LookupCache cache = managedCaches.get(managedCaches.keySet().iterator().next()).getCache(); // jdbc does support project push down, the cached row has been projected RowData key1 = GenericRowData.of(1L); - RowData value1 = - GenericRowData.of( - 1L, - DecimalData.fromBigDecimal(BigDecimal.valueOf(100.1234), 10, 4), - TimestampData.fromLocalDateTime( - truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")))); + RowData value1 = GenericRowData.of(1L, DecimalData.fromBigDecimal(BigDecimal.valueOf(100.1234), 10, 4), TimestampData.fromLocalDateTime(truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")))); RowData key2 = GenericRowData.of(2L); - RowData value2 = - GenericRowData.of( - 2L, - DecimalData.fromBigDecimal(BigDecimal.valueOf(101.1234), 10, 4), - TimestampData.fromLocalDateTime( - truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); + RowData value2 = GenericRowData.of(2L, DecimalData.fromBigDecimal(BigDecimal.valueOf(101.1234), 10, 4), TimestampData.fromLocalDateTime(truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); RowData key3 = GenericRowData.of(3L); @@ -402,7 +339,6 @@ private void validateCachedValues() { } private enum Caching { - ENABLE_CACHE, - DISABLE_CACHE + ENABLE_CACHE, DISABLE_CACHE } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java index b2e4deea8..55bd4ecdd 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java @@ -51,31 +51,16 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Test suite for {@link JdbcRowDataInputFormat}. */ +/** + * Test suite for {@link JdbcRowDataInputFormat}. + */ class JdbcRowDataInputFormatTest extends JdbcDataTestBase { private JdbcRowDataInputFormat inputFormat; - private static String[] fieldNames = new String[] {"id", "title", "author", "price", "qty"}; - private static DataType[] fieldDataTypes = - new DataType[] { - DataTypes.INT(), - DataTypes.STRING(), - DataTypes.STRING(), - DataTypes.DOUBLE(), - DataTypes.INT() - }; - final JdbcDialect dialect = - InternalJdbcConnectionOptions.builder() - .setDBUrl(getMetadata().getJdbcUrl()) - .setTableName(INPUT_TABLE) - .build() - .getDialect(); - final RowType rowType = - RowType.of( - Arrays.stream(fieldDataTypes) - .map(DataType::getLogicalType) - .toArray(LogicalType[]::new), - fieldNames); + private static final String[] fieldNames = new String[]{"id", "title", "author", "price", "qty"}; + private static final DataType[] fieldDataTypes = new DataType[]{DataTypes.INT(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.DOUBLE(), DataTypes.INT()}; + final JdbcDialect dialect = InternalJdbcConnectionOptions.builder().setDBUrl(getMetadata().getJdbcUrl()).setTableName(INPUT_TABLE).build().getDialect(); + final RowType rowType = RowType.of(Arrays.stream(fieldDataTypes).map(DataType::getLogicalType).toArray(LogicalType[]::new), fieldNames); @AfterEach void tearDown() throws IOException { @@ -88,134 +73,65 @@ void tearDown() throws IOException { @Test void testNoRowConverter() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .build(); - inputFormat.openInputFormat(); - }) - .isInstanceOf(NullPointerException.class); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).build(); + inputFormat.openInputFormat(); + }).isInstanceOf(NullPointerException.class); } @Test void testInvalidDriver() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); - inputFormat.openInputFormat(); - }) - .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername("org.apache.derby.jdbc.idontexist").setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat.openInputFormat(); + }).isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidURL() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") - .setQuery(SELECT_ALL_BOOKS) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); - inputFormat.openInputFormat(); - }) - .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl("jdbc:der:iamanerror:mory:ebookshop").setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat.openInputFormat(); + }).isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidQuery() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery("iamnotsql") - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); - inputFormat.openInputFormat(); - }) - .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery("iamnotsql").setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat.openInputFormat(); + }).isInstanceOf(IllegalArgumentException.class); } @Test void testNoQuery() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); - }) - .isInstanceOf(NullPointerException.class) - .hasMessage("No query supplied"); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setRowConverter(dialect.getRowConverter(rowType)).build(); + }).isInstanceOf(NullPointerException.class).hasMessage("No query supplied"); } @Test void testNoUrl() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setQuery(SELECT_ALL_BOOKS) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); - }) - .isInstanceOf(NullPointerException.class) - .hasMessage("jdbc url is empty"); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); + }).isInstanceOf(NullPointerException.class).hasMessage("jdbc url is empty"); } @Test void testInvalidFetchSize() { - assertThatThrownBy( - () -> { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setFetchSize(-7) - .build(); - }) - .isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy(() -> { + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setFetchSize(-7).build(); + }).isInstanceOf(IllegalArgumentException.class); } @Test void testValidFetchSizeIntegerMin() { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setFetchSize(Integer.MIN_VALUE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setFetchSize(Integer.MIN_VALUE).setRowConverter(dialect.getRowConverter(rowType)).build(); } @Test void testJdbcInputFormatWithoutParallelism() throws IOException { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); // this query does not exploit parallelism assertThat(inputFormat.createInputSplits(1)).hasSize(1); inputFormat.openInputFormat(); @@ -239,17 +155,8 @@ void testJdbcInputFormatWithParallelismAndNumericColumnSplitting() throws IOExce final int fetchSize = 1; final long min = TEST_DATA[0].id; final long max = TEST_DATA[TEST_DATA.length - fetchSize].id; - JdbcParameterValuesProvider pramProvider = - new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID) - .setParametersProvider(pramProvider) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + JdbcParameterValuesProvider pramProvider = new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID).setParametersProvider(pramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -277,17 +184,8 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE final long min = TEST_DATA[0].id; final long max = TEST_DATA[TEST_DATA.length - 1].id; final long fetchSize = max + 1; // generate a single split - JdbcParameterValuesProvider pramProvider = - new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize); - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID) - .setParametersProvider(pramProvider) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + JdbcParameterValuesProvider pramProvider = new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID).setParametersProvider(pramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -313,19 +211,10 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE @Test void testJdbcInputFormatWithParallelismAndGenericSplitting() throws IOException { Serializable[][] queryParameters = new String[2][1]; - queryParameters[0] = new String[] {TEST_DATA[3].author}; - queryParameters[1] = new String[] {TEST_DATA[0].author}; - JdbcParameterValuesProvider paramProvider = - new JdbcGenericParameterValuesProvider(queryParameters); - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR) - .setParametersProvider(paramProvider) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + queryParameters[0] = new String[]{TEST_DATA[3].author}; + queryParameters[1] = new String[]{TEST_DATA[0].author}; + JdbcParameterValuesProvider paramProvider = new JdbcGenericParameterValuesProvider(queryParameters); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR).setParametersProvider(paramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -359,14 +248,7 @@ private void verifySplit(InputSplit split, int expectedIDSum) throws IOException @Test void testEmptyResults() throws IOException { - inputFormat = - JdbcRowDataInputFormat.builder() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_EMPTY) - .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) - .setRowConverter(dialect.getRowConverter(rowType)) - .build(); + inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_EMPTY).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); try { inputFormat.openInputFormat(); @@ -379,15 +261,10 @@ void testEmptyResults() throws IOException { } private static void assertEquals(JdbcTestFixture.TestEntry expected, RowData actual) { - assertThat(actual.isNullAt(0) ? null : Integer.valueOf(actual.getInt(0))) - .isEqualTo(expected.id); - assertThat(actual.isNullAt(1) ? null : actual.getString(1).toString()) - .isEqualTo(expected.title); - assertThat(actual.isNullAt(2) ? null : actual.getString(2).toString()) - .isEqualTo(expected.author); - assertThat(actual.isNullAt(3) ? null : Double.valueOf(actual.getDouble(3))) - .isEqualTo(expected.price); - assertThat(actual.isNullAt(4) ? null : Integer.valueOf(actual.getInt(4))) - .isEqualTo(expected.qty); + assertThat(actual.isNullAt(0) ? null : Integer.valueOf(actual.getInt(0))).isEqualTo(expected.id); + assertThat(actual.isNullAt(1) ? null : actual.getString(1).toString()).isEqualTo(expected.title); + assertThat(actual.isNullAt(2) ? null : actual.getString(2).toString()).isEqualTo(expected.author); + assertThat(actual.isNullAt(3) ? null : Double.valueOf(actual.getDouble(3))).isEqualTo(expected.price); + assertThat(actual.isNullAt(4) ? null : Integer.valueOf(actual.getInt(4))).isEqualTo(expected.qty); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java new file mode 100644 index 000000000..2788b3966 --- /dev/null +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java @@ -0,0 +1,19 @@ +package org.apache.flink.connector.jdbc.testutils.databases; + +// + +/** + * Define a database name. * + */ +public enum DbName { + + POSTGRES_DB, + MYSQL_DB, + ORACLE_DB, + SQL_SERVER_DB, + TRINO_DB, + DB2_DB, + DERBY_DB, + + +} diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java index d8cbd7937..c2d70f00f 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java @@ -23,6 +23,7 @@ import org.apache.flink.connector.jdbc.JdbcStatementBuilder; import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; import org.apache.flink.connector.jdbc.testutils.TableManaged; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.functions.JdbcResultSetBuilder; import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil; import org.apache.flink.table.api.DataTypes; @@ -49,17 +50,22 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -/** Base table operations. * */ +/** + * Base table operations. * + */ public abstract class TableBase implements TableManaged { private final String name; + + private final DbName dbName; private final TableField[] fields; - protected TableBase(String name, TableField[] fields) { + protected TableBase(String name, DbName dbName, TableField[] fields) { Preconditions.checkArgument(name != null && !name.isEmpty(), "Table name must be defined"); Preconditions.checkArgument( fields != null && fields.length != 0, "Table fields must be defined"); this.name = name; + this.dbName = dbName; this.fields = fields; } @@ -104,6 +110,10 @@ public RowTypeInfo getTableRowTypeInfo() { return new RowTypeInfo(typesArray, fieldsArray); } + public DbName getDbName() { + return dbName; + } + public RowType getTableRowType() { LogicalType[] typesArray = getStreamDataTypes().map(DataType::getLogicalType).toArray(LogicalType[]::new); @@ -265,7 +275,7 @@ protected void executeUpdate(Connection conn, String sql) throws SQLException { protected List executeStatement( Connection conn, String sql, JdbcResultSetBuilder rsGetter) throws SQLException { try (Statement st = conn.createStatement(); - ResultSet rs = st.executeQuery(sql)) { + ResultSet rs = st.executeQuery(sql)) { return rsGetter.accept(rs); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java index 6a8f80d3b..659fc19ee 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java @@ -18,13 +18,20 @@ package org.apache.flink.connector.jdbc.testutils.tables; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.table.types.DataType; -/** Table builder. * */ +/** + * Table builder. * + */ public final class TableBuilder { public static TableRow tableRow(String name, TableField... fields) { - return new TableRow(name, fields); + return new TableRow(name, null, fields); + } + + public static TableRow tableRow(String name, DbName dbName, TableField... fields) { + return new TableRow(name, dbName, fields); } public static TableField field(String name, DataType dataType) { diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java index 87308fb70..5040118ec 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java @@ -20,6 +20,7 @@ import org.apache.flink.connector.jdbc.JdbcStatementBuilder; import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; +import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.connector.jdbc.testutils.functions.JdbcResultSetBuilder; import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil; import org.apache.flink.table.api.DataTypes; @@ -40,11 +41,13 @@ import static org.assertj.core.api.Assertions.assertThat; -/** Row table. * */ +/** + * Row table. * + */ public class TableRow extends TableBase { - protected TableRow(String name, TableField[] fields) { - super(name, fields); + protected TableRow(String name, DbName dbName, TableField[] fields) { + super(name, dbName, fields); } protected JdbcResultSetBuilder getResultSetBuilder() { diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/templates/BooksTable.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/templates/BooksTable.java index c1bb4d8f9..9207d851b 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/templates/BooksTable.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/templates/BooksTable.java @@ -77,6 +77,7 @@ public class BooksTable extends TableBase implements Table public BooksTable(String name) { super( name, + null, Arrays.asList( pkField("id", INT().notNull()), field("title", VARCHAR(50)), From 9b1dec27b11e56b3c94b94a85e22be7d668b656c Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 29 Dec 2023 11:47:31 +0800 Subject: [PATCH 2/5] [FLINK-33929][jdbc-connector] Support string slicing parallel reading --- .../JdbcNumericBetweenParametersProvider.java | 4 +--- .../jdbc/testutils/databases/DbName.java | 20 +++++++++++++++++-- 2 files changed, 19 insertions(+), 5 deletions(-) diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java index e46d0d30b..810ac6e5b 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java @@ -106,9 +106,7 @@ public JdbcNumericBetweenParametersProvider ofBatchNum(int batchNum) { @Override public Serializable[][] getParameterValues() { - Preconditions.checkState( - batchSize > 0, - "Batch size and batch number must be positive. Have you called `ofBatchSize` or `ofBatchNum`?"); + Preconditions.checkState(batchSize > 0, "Batch size and batch number must be positive. Have you called `ofBatchSize` or `ofBatchNum`?"); Serializable[][] parameters = new Serializable[batchNum][2]; long maxElemCount = (maxVal - minVal) + 1; long bigBatchNum = maxElemCount - (batchSize - 1) * batchNum; diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java index 2788b3966..f9241ad50 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java @@ -1,6 +1,22 @@ -package org.apache.flink.connector.jdbc.testutils.databases; +/* + * 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.flink.connector.jdbc.testutils.databases; /** * Define a database name. * From 2f5787e624a2b852ad2b8d3778a45f1c3bcfe299 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Fri, 29 Dec 2023 14:05:46 +0800 Subject: [PATCH 3/5] [FLINK-33929][jdbc-connector] Support string slicing parallel reading --- .../apache/flink/connector/jdbc/testutils/databases/DbName.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java index f9241ad50..338f453e0 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java @@ -29,7 +29,7 @@ public enum DbName { SQL_SERVER_DB, TRINO_DB, DB2_DB, - DERBY_DB, + DERBY_DB } From ae9255b814f301769a3e0ab8374c94f6fbe2d405 Mon Sep 17 00:00:00 2001 From: zhilinli Date: Mon, 15 Jan 2024 11:45:44 +0800 Subject: [PATCH 4/5] [FLINK-33929][jdbc-connector] Support string slicing parallel reading --- .../flink/connector/jdbc/JdbcInputFormat.java | 10 +- .../databases/db2/dialect/Db2Dialect.java | 5 +- .../databases/derby/dialect/DerbyDialect.java | 5 +- .../postgres/dialect/PostgresDialect.java | 4 +- .../databases/trino/dialect/TrinoDialect.java | 5 +- .../internal/options/JdbcReadOptions.java | 28 +- .../CompositeJdbcParameterValuesProvider.java | 8 +- .../JdbcNumericBetweenParametersProvider.java | 29 +- .../jdbc/table/JdbcDynamicTableFactory.java | 27 +- .../jdbc/table/JdbcDynamicTableSource.java | 16 +- .../jdbc/table/JdbcRowDataInputFormat.java | 17 +- .../connector/jdbc/JdbcInputFormatTest.java | 164 +++++----- .../table/DerbyDynamicTableSourceITCase.java | 4 +- .../table/MySqlDynamicTableSourceITCase.java | 7 +- .../PostgresDynamicTableSourceITCase.java | 6 +- .../SqlServerDynamicTableSourceITCase.java | 4 +- .../NumericBetweenParametersProviderTest.java | 42 ++- .../table/JdbcDynamicTableSourceITCase.java | 286 ++++++++++++++---- .../table/JdbcRowDataInputFormatTest.java | 220 +++++++++++--- .../jdbc/testutils/databases/DbName.java | 7 +- .../jdbc/testutils/tables/TableBase.java | 6 +- .../jdbc/testutils/tables/TableBuilder.java | 4 +- .../jdbc/testutils/tables/TableRow.java | 4 +- 23 files changed, 606 insertions(+), 302 deletions(-) diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java index c670e9ffe..e0d9977c7 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/JdbcInputFormat.java @@ -121,7 +121,7 @@ public class JdbcInputFormat extends RichInputFormat protected boolean hasNext; protected Object[][] parameterValues; - protected boolean isPartitionColumnTypeString; + protected boolean isPartitionColumnTypeString; public JdbcInputFormat() {} @@ -191,7 +191,10 @@ public void closeInputFormat() { public void open(InputSplit inputSplit) throws IOException { try { if (inputSplit != null && parameterValues != null) { - int parameterLength = isPartitionColumnTypeString ? 1 : parameterValues[inputSplit.getSplitNumber()].length; + int parameterLength = + isPartitionColumnTypeString + ? 1 + : parameterValues[inputSplit.getSplitNumber()].length; for (int i = 0; i < parameterLength; i++) { Object param = parameterValues[inputSplit.getSplitNumber()][i]; if (param instanceof String) { @@ -395,7 +398,8 @@ public JdbcInputFormatBuilder setParametersProvider( return this; } - public JdbcInputFormatBuilder setPartitionColumnTypeString(boolean partitionColumnTypeString) { + public JdbcInputFormatBuilder setPartitionColumnTypeString( + boolean partitionColumnTypeString) { format.isPartitionColumnTypeString = partitionColumnTypeString; return this; } diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java index 1da75c275..6fbd58949 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/db2/dialect/Db2Dialect.java @@ -58,7 +58,10 @@ public String quoteIdentifier(String identifier) { @Override public String hashModForField(String fieldName, int numPartitions) { - throw new IllegalArgumentException("The Db2 database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + throw new IllegalArgumentException( + "The Db2 database itself is not supported by the hash md5 syntax " + + fieldName + + "Cannot be read in fragments"); } @Override diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java index eab837945..1b33e40fe 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java @@ -61,7 +61,10 @@ public String quoteIdentifier(String identifier) { @Override public String hashModForField(String fieldName, int numPartitions) { - throw new IllegalArgumentException("The Derby database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + throw new IllegalArgumentException( + "The Derby database itself is not supported by the hash md5 syntax " + + fieldName + + "Cannot be read in fragments"); } @Override diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java index 9699c67b1..af733d210 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java @@ -24,9 +24,7 @@ import java.util.Optional; -/** - * JDBC dialect for PostgreSQL. - */ +/** JDBC dialect for PostgreSQL. */ @Internal public class PostgresDialect extends AbstractPostgresCompatibleDialect { diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java index 835abbcdc..dd36cdd5d 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/trino/dialect/TrinoDialect.java @@ -66,7 +66,10 @@ public String dialectName() { @Override public String hashModForField(String fieldName, int numPartitions) { - throw new IllegalArgumentException("The Trino database itself is not supported by the hash md5 syntax " + fieldName + "Cannot be read in fragments"); + throw new IllegalArgumentException( + "The Trino database itself is not supported by the hash md5 syntax " + + fieldName + + "Cannot be read in fragments"); } @Override diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java index c0ad5db30..50e6d2a84 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcReadOptions.java @@ -22,9 +22,7 @@ import java.util.Objects; import java.util.Optional; -/** - * Options for the JDBC scan. - */ +/** Options for the JDBC scan. */ public class JdbcReadOptions implements Serializable { private final String query; @@ -109,9 +107,7 @@ public boolean equals(Object o) { } } - /** - * Builder of {@link JdbcReadOptions}. - */ + /** Builder of {@link JdbcReadOptions}. */ public static class Builder { protected String query; protected String partitionColumnName; @@ -123,17 +119,13 @@ public static class Builder { protected int fetchSize = 0; protected boolean autoCommit = true; - /** - * optional, SQL query statement for this JDBC source. - */ + /** optional, SQL query statement for this JDBC source. */ public Builder setQuery(String query) { this.query = query; return this; } - /** - * optional, name of the column used for partitioning the input. - */ + /** optional, name of the column used for partitioning the input. */ public Builder setPartitionColumnName(String partitionColumnName) { this.partitionColumnName = partitionColumnName; return this; @@ -143,17 +135,13 @@ public void setPartitionColumnTypeString(Boolean partitionColumnTypeString) { isPartitionColumnTypeString = partitionColumnTypeString; } - /** - * optional, the smallest value of the first partition. - */ + /** optional, the smallest value of the first partition. */ public Builder setPartitionLowerBound(long partitionLowerBound) { this.partitionLowerBound = partitionLowerBound; return this; } - /** - * optional, the largest value of the last partition. - */ + /** optional, the largest value of the last partition. */ public Builder setPartitionUpperBound(long partitionUpperBound) { this.partitionUpperBound = partitionUpperBound; return this; @@ -177,9 +165,7 @@ public Builder setFetchSize(int fetchSize) { return this; } - /** - * optional, whether to set auto commit on the JDBC driver. - */ + /** optional, whether to set auto commit on the JDBC driver. */ public Builder setAutoCommit(boolean autoCommit) { this.autoCommit = autoCommit; return this; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java index 29d4e34eb..8f826270d 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/CompositeJdbcParameterValuesProvider.java @@ -28,10 +28,14 @@ public class CompositeJdbcParameterValuesProvider implements JdbcParameterValuesProvider { JdbcParameterValuesProvider a; JdbcParameterValuesProvider b; + public CompositeJdbcParameterValuesProvider( - JdbcParameterValuesProvider a, JdbcParameterValuesProvider b, boolean isPartitionColumnTypeString) { + JdbcParameterValuesProvider a, + JdbcParameterValuesProvider b, + boolean isPartitionColumnTypeString) { Preconditions.checkArgument( - isPartitionColumnTypeString || a.getParameterValues().length == b.getParameterValues().length, + isPartitionColumnTypeString + || a.getParameterValues().length == b.getParameterValues().length, "Both JdbcParameterValuesProvider should have the same length."); this.a = a; this.b = b; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java index 810ac6e5b..cc8b14618 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/split/JdbcNumericBetweenParametersProvider.java @@ -38,7 +38,6 @@ *

You can take advantage of this class to automatically generate the parameters of the BETWEEN * clause, based on the passed constructor parameters. */ - @Experimental public class JdbcNumericBetweenParametersProvider implements JdbcParameterValuesProvider { @@ -53,13 +52,17 @@ public class JdbcNumericBetweenParametersProvider implements JdbcParameterValues /** * NumericBetweenParametersProviderJdbc constructor. * - * @param minVal the lower bound of the produced "from" values - * @param maxVal the upper bound of the produced "to" values + * @param minVal the lower bound of the produced "from" values + * @param maxVal the upper bound of the produced "to" values * @param isPartitionColumnTypeString Whether to use string types as slices to read fields */ - public JdbcNumericBetweenParametersProvider(long minVal, long maxVal, boolean isPartitionColumnTypeString) { + public JdbcNumericBetweenParametersProvider( + long minVal, long maxVal, boolean isPartitionColumnTypeString) { Preconditions.checkArgument(minVal <= maxVal, "minVal must not be larger than maxVal"); - Preconditions.checkArgument(!isPartitionColumnTypeString || (minVal < Integer.MAX_VALUE && maxVal < Integer.MAX_VALUE), "When using a string as a shard field to set the upper and lower bounds lowerBound, upperBound exceeds the INT storage range, and the current logic uses the hash for each value within each shard boundary value"); + Preconditions.checkArgument( + !isPartitionColumnTypeString + || (minVal < Integer.MAX_VALUE && maxVal < Integer.MAX_VALUE), + "When using a string as a shard field to set the upper and lower bounds lowerBound, upperBound exceeds the INT storage range, and the current logic uses the hash for each value within each shard boundary value"); this.minVal = minVal; this.maxVal = maxVal; this.isPartitionColumnTypeString = isPartitionColumnTypeString; @@ -69,8 +72,8 @@ public JdbcNumericBetweenParametersProvider(long minVal, long maxVal, boolean is * NumericBetweenParametersProviderJdbc constructor. * * @param fetchSize the max distance between the produced from/to pairs - * @param minVal the lower bound of the produced "from" values - * @param maxVal the upper bound of the produced "to" values + * @param minVal the lower bound of the produced "from" values + * @param maxVal the upper bound of the produced "to" values */ public JdbcNumericBetweenParametersProvider(long fetchSize, long minVal, long maxVal) { Preconditions.checkArgument(minVal <= maxVal, "minVal must not be larger than maxVal"); @@ -93,7 +96,9 @@ public JdbcNumericBetweenParametersProvider ofBatchSize(long batchSize) { public JdbcNumericBetweenParametersProvider ofBatchNum(int batchNum) { Preconditions.checkArgument(batchNum > 0, "Batch number must be positive"); - Preconditions.checkArgument(!isPartitionColumnTypeString || (minVal >= 0), "Ensure that lowerBound is greater than or equal to 0 and lowerBound, upperBound, is less than or equal to the number of partitions"); + Preconditions.checkArgument( + !isPartitionColumnTypeString || (minVal >= 0), + "Ensure that lowerBound is greater than or equal to 0 and lowerBound, upperBound, is less than or equal to the number of partitions"); long maxElemCount = (maxVal - minVal) + 1; if (batchNum > maxElemCount) { @@ -106,7 +111,9 @@ public JdbcNumericBetweenParametersProvider ofBatchNum(int batchNum) { @Override public Serializable[][] getParameterValues() { - Preconditions.checkState(batchSize > 0, "Batch size and batch number must be positive. Have you called `ofBatchSize` or `ofBatchNum`?"); + Preconditions.checkState( + batchSize > 0, + "Batch size and batch number must be positive. Have you called `ofBatchSize` or `ofBatchNum`?"); Serializable[][] parameters = new Serializable[batchNum][2]; long maxElemCount = (maxVal - minVal) + 1; long bigBatchNum = maxElemCount - (batchSize - 1) * batchNum; @@ -114,13 +121,13 @@ public Serializable[][] getParameterValues() { long start = minVal; for (int i = 0; i < batchNum; i++) { long end = start + batchSize - 1 - (i >= bigBatchNum ? 1 : 0); - parameters[i] = new Long[]{start, end}; + parameters[i] = new Long[] {start, end}; start = end + 1; } } else { for (int i = 0; i <= maxVal - minVal; i++) { Long hashValue = minVal + i; - parameters[i] = new Long[]{hashValue, hashValue}; + parameters[i] = new Long[] {hashValue, hashValue}; } } return parameters; diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java index 83451f617..23bf0786e 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java @@ -150,15 +150,22 @@ private InternalJdbcConnectionOptions getJdbcOptions( return builder.build(); } - private JdbcReadOptions getJdbcReadOptions(ReadableConfig readableConfig, DataType physicalRowDataType) { + private JdbcReadOptions getJdbcReadOptions( + ReadableConfig readableConfig, DataType physicalRowDataType) { final Optional partitionColumnName = readableConfig.getOptional(SCAN_PARTITION_COLUMN); RowType logicalType = (RowType) physicalRowDataType.getLogicalType(); - Map fieldTypeMap = logicalType.getFields().stream().collect(Collectors.toMap(rowField -> rowField.getName(), rowField -> rowField.getType())); + Map fieldTypeMap = + logicalType.getFields().stream() + .collect( + Collectors.toMap( + rowField -> rowField.getName(), + rowField -> rowField.getType())); final JdbcReadOptions.Builder builder = JdbcReadOptions.builder(); if (partitionColumnName.isPresent()) { String partitionName = partitionColumnName.get(); - Boolean isPartitionColumnTypeString = fieldTypeMap.get(partitionName) instanceof VarCharType ? true : false; + Boolean isPartitionColumnTypeString = + fieldTypeMap.get(partitionName) instanceof VarCharType ? true : false; builder.setPartitionColumnName(partitionName); builder.setPartitionColumnTypeString(isPartitionColumnTypeString); builder.setPartitionLowerBound(readableConfig.get(SCAN_PARTITION_LOWER_BOUND)); @@ -279,15 +286,15 @@ private void validateConfigOptions(ReadableConfig config, ClassLoader classLoade String jdbcUrl = config.get(URL); JdbcDialectLoader.load(jdbcUrl, classLoader); - checkAllOrNone(config, new ConfigOption[]{USERNAME, PASSWORD}); + checkAllOrNone(config, new ConfigOption[] {USERNAME, PASSWORD}); checkAllOrNone( config, - new ConfigOption[]{ - SCAN_PARTITION_COLUMN, - SCAN_PARTITION_NUM, - SCAN_PARTITION_LOWER_BOUND, - SCAN_PARTITION_UPPER_BOUND + new ConfigOption[] { + SCAN_PARTITION_COLUMN, + SCAN_PARTITION_NUM, + SCAN_PARTITION_LOWER_BOUND, + SCAN_PARTITION_UPPER_BOUND }); if (config.getOptional(SCAN_PARTITION_LOWER_BOUND).isPresent() @@ -305,7 +312,7 @@ private void validateConfigOptions(ReadableConfig config, ClassLoader classLoade } } - checkAllOrNone(config, new ConfigOption[]{LOOKUP_CACHE_MAX_ROWS, LOOKUP_CACHE_TTL}); + checkAllOrNone(config, new ConfigOption[] {LOOKUP_CACHE_MAX_ROWS, LOOKUP_CACHE_TTL}); if (config.get(LOOKUP_MAX_RETRIES) < 0) { throw new IllegalArgumentException( diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java index 2af2aa800..9b1747e3a 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSource.java @@ -146,21 +146,27 @@ public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderCon Serializable[][] allPushDownParams = replicatePushdownParamsForN(numPartitions); // Get here whether the type of the external configuration partition key is a string - boolean isPartitionColumnTypeString = readOptions.getPartitionColumnTypeString().get().booleanValue(); + boolean isPartitionColumnTypeString = + readOptions.getPartitionColumnTypeString().get().booleanValue(); JdbcParameterValuesProvider allParams = new CompositeJdbcParameterValuesProvider( - new JdbcNumericBetweenParametersProvider(lowerBound, upperBound, isPartitionColumnTypeString) + new JdbcNumericBetweenParametersProvider( + lowerBound, upperBound, isPartitionColumnTypeString) .ofBatchNum(numPartitions), - new JdbcGenericParameterValuesProvider(allPushDownParams), isPartitionColumnTypeString); + new JdbcGenericParameterValuesProvider(allPushDownParams), + isPartitionColumnTypeString); builder.setParametersProvider(allParams); // Set partition type builder.setPartitionColumnTypeString(isPartitionColumnTypeString); String generatePredicates; - if (readOptions.getPartitionColumnTypeString().isPresent() && readOptions.getPartitionColumnTypeString().get()) { + if (readOptions.getPartitionColumnTypeString().isPresent() + && readOptions.getPartitionColumnTypeString().get()) { generatePredicates = - dialect.hashModForField(readOptions.getPartitionColumnName().get(), numPartitions) + " = ? "; + dialect.hashModForField( + readOptions.getPartitionColumnName().get(), numPartitions) + + " = ? "; } else { generatePredicates = dialect.quoteIdentifier(readOptions.getPartitionColumnName().get()) diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java index d334faf5f..5ecb7793c 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormat.java @@ -52,9 +52,7 @@ import java.sql.Timestamp; import java.util.Arrays; -/** - * InputFormat for {@link JdbcDynamicTableSource}. - */ +/** InputFormat for {@link JdbcDynamicTableSource}. */ @Internal public class JdbcRowDataInputFormat extends RichInputFormat implements ResultTypeQueryable { @@ -152,14 +150,17 @@ public void closeInputFormat() { * fashion otherwise. * * @param inputSplit which is ignored if this InputFormat is executed as a non-parallel source, - * a "hook" to the query parameters otherwise (using its splitNumber) + * a "hook" to the query parameters otherwise (using its splitNumber) * @throws IOException if there's an error during the execution of the query */ @Override public void open(InputSplit inputSplit) throws IOException { try { if (inputSplit != null && parameterValues != null) { - int parameterLength = isPartitionColumnTypeString ? 1 : parameterValues[inputSplit.getSplitNumber()].length; + int parameterLength = + isPartitionColumnTypeString + ? 1 + : parameterValues[inputSplit.getSplitNumber()].length; for (int i = 0; i < parameterLength; i++) { Object param = parameterValues[inputSplit.getSplitNumber()][i]; if (param instanceof String) { @@ -279,7 +280,7 @@ public BaseStatistics getStatistics(BaseStatistics cachedStatistics) throws IOEx @Override public InputSplit[] createInputSplits(int minNumSplits) throws IOException { if (parameterValues == null) { - return new GenericInputSplit[]{new GenericInputSplit(0, 1)}; + return new GenericInputSplit[] {new GenericInputSplit(0, 1)}; } GenericInputSplit[] ret = new GenericInputSplit[parameterValues.length]; for (int i = 0; i < ret.length; i++) { @@ -302,9 +303,7 @@ public static Builder builder() { return new Builder(); } - /** - * Builder for {@link JdbcRowDataInputFormat}. - */ + /** Builder for {@link JdbcRowDataInputFormat}. */ public static class Builder { private final JdbcConnectionOptions.JdbcConnectionOptionsBuilder connOptionsBuilder; private int fetchSize; diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java index 5c1cbe2e3..8bf9f56de 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcInputFormatTest.java @@ -44,9 +44,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** - * Tests for the {@link JdbcInputFormat}. - */ +/** Tests for the {@link JdbcInputFormat}. */ class JdbcInputFormatTest extends JdbcDataTestBase { private JdbcInputFormat jdbcInputFormat; @@ -70,16 +68,16 @@ void testJdbcInputFormatspiltKeyString() throws IOException { // check string The number of input slices for the scan boundary Serializable[][] correctParameters = new Serializable[10][2]; - correctParameters[0] = new Long[]{min, min}; - correctParameters[1] = new Long[]{min + 1, min + 1}; - correctParameters[2] = new Long[]{min + 2, min + 2}; - correctParameters[3] = new Long[]{min + 3, min + 3}; - correctParameters[4] = new Long[]{min + 4, min + 4}; - correctParameters[5] = new Long[]{min + 5, min + 5}; - correctParameters[6] = new Long[]{min + 6, min + 6}; - correctParameters[7] = new Long[]{min + 7, min + 7}; - correctParameters[8] = new Long[]{min + 8, min + 8}; - correctParameters[9] = new Long[]{min + 9, min + 9}; + correctParameters[0] = new Long[] {min, min}; + correctParameters[1] = new Long[] {min + 1, min + 1}; + correctParameters[2] = new Long[] {min + 2, min + 2}; + correctParameters[3] = new Long[] {min + 3, min + 3}; + correctParameters[4] = new Long[] {min + 4, min + 4}; + correctParameters[5] = new Long[] {min + 5, min + 5}; + correctParameters[6] = new Long[] {min + 6, min + 6}; + correctParameters[7] = new Long[] {min + 7, min + 7}; + correctParameters[8] = new Long[] {min + 8, min + 8}; + correctParameters[9] = new Long[] {min + 9, min + 9}; assertThat(pramProvider.getParameterValues().length).isEqualTo(max - min + 1); assertThat(pramProvider.getParameterValues()).isEqualTo(correctParameters); @@ -90,7 +88,7 @@ void testJdbcInputFormatspiltKeyString() throws IOException { // check string The number of input slices for the scan boundary Serializable[][] singleCorrectParameters = new Serializable[1][2]; - singleCorrectParameters[0] = new Long[]{1L, 1L}; + singleCorrectParameters[0] = new Long[] {1L, 1L}; assertThat(singleProvider.getParameterValues()).isEqualTo(singleCorrectParameters); } @@ -98,15 +96,15 @@ void testJdbcInputFormatspiltKeyString() throws IOException { @Test void testUntypedRowInfo() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("No RowTypeInfo supplied"); } @@ -114,62 +112,62 @@ void testUntypedRowInfo() { @Test void testInvalidDriver() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername("org.apache.derby.jdbc.idontexist") - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername("org.apache.derby.jdbc.idontexist") + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidURL() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidQuery() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery("iamnotsql") - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - jdbcInputFormat.openInputFormat(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery("iamnotsql") + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + jdbcInputFormat.openInputFormat(); + }) .isInstanceOf(IllegalArgumentException.class); } @Test void testNoUrl() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("jdbc url is empty"); } @@ -177,14 +175,14 @@ void testNoUrl() { @Test void testNoQuery() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setRowTypeInfo(ROW_TYPE_INFO) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setRowTypeInfo(ROW_TYPE_INFO) + .finish(); + }) .isInstanceOf(NullPointerException.class) .hasMessage("No query supplied"); } @@ -192,16 +190,16 @@ void testNoQuery() { @Test void testInvalidFetchSize() { assertThatThrownBy( - () -> { - jdbcInputFormat = - JdbcInputFormat.buildJdbcInputFormat() - .setDrivername(getMetadata().getDriverClass()) - .setDBUrl(getMetadata().getJdbcUrl()) - .setQuery(SELECT_ALL_BOOKS) - .setRowTypeInfo(ROW_TYPE_INFO) - .setFetchSize(-7) - .finish(); - }) + () -> { + jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setRowTypeInfo(ROW_TYPE_INFO) + .setFetchSize(-7) + .finish(); + }) .isInstanceOf(IllegalArgumentException.class); } @@ -229,8 +227,8 @@ void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException { jdbcInputFormat.openInputFormat(); try (Connection dbConn = getMetadata().getConnection(); - Statement dbStatement = dbConn.createStatement(); - Statement inputStatement = jdbcInputFormat.getStatement()) { + Statement dbStatement = dbConn.createStatement(); + Statement inputStatement = jdbcInputFormat.getStatement()) { assertThat(inputStatement.getFetchSize()).isEqualTo(dbStatement.getFetchSize()); } } @@ -392,8 +390,8 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE @Test void testJdbcInputFormatWithParallelismAndGenericSplitting() throws IOException { Serializable[][] queryParameters = new String[2][1]; - queryParameters[0] = new String[]{TEST_DATA[3].author}; - queryParameters[1] = new String[]{TEST_DATA[0].author}; + queryParameters[0] = new String[] {TEST_DATA[3].author}; + queryParameters[1] = new String[] {TEST_DATA[0].author}; JdbcParameterValuesProvider paramProvider = new JdbcGenericParameterValuesProvider(queryParameters); jdbcInputFormat = diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java index 878298026..215932159 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java @@ -36,9 +36,7 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; -/** - * The Table Source ITCase for {@link DerbyDialect}. - */ +/** The Table Source ITCase for {@link DerbyDialect}. */ public class DerbyDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements DerbyTestBase { diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java index 61041284a..95a245f8c 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java @@ -36,9 +36,7 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field; import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; -/** - * The Table Source ITCase for {@link MySqlDialect}. - */ +/** The Table Source ITCase for {@link MySqlDialect}. */ public class MySqlDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements MySqlTestBase { @@ -76,6 +74,7 @@ protected List getTestData() { -1.175E-37D, -1.79769E308, LocalTime.parse("15:36:01"), - LocalDateTime.parse("2020-01-01T15:36:01.123456"), "Wade")); + LocalDateTime.parse("2020-01-01T15:36:01.123456"), + "Wade")); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java index 920a14d60..66d6d5bec 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java @@ -63,13 +63,15 @@ protected List getTestData() { LocalDateTime.parse("2020-01-01T15:35:00.123456"), 1.175E-37F, 1.79769E308D, - LocalTime.parse("15:35"), "Leblanc_1"), + LocalTime.parse("15:35"), + "Leblanc_1"), Row.of( 2L, BigDecimal.valueOf(101.1234), LocalDateTime.parse("2020-01-01T15:36:01.123456"), -1.175E-37F, -1.79769E308, - LocalTime.parse("15:36:01"), "Wade")); + LocalTime.parse("15:36:01"), + "Wade")); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java index c51b74aee..0a072e022 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java @@ -40,9 +40,7 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** - * The Table Source ITCase for {@link SqlServerDialect}. - */ +/** The Table Source ITCase for {@link SqlServerDialect}. */ public class SqlServerDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase implements SqlServerTestBase { diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java index 1825ce2bf..72f20f362 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/split/NumericBetweenParametersProviderTest.java @@ -24,9 +24,7 @@ import static org.assertj.core.api.Assertions.assertThat; -/** - * Tests for {@link JdbcNumericBetweenParametersProvider}. - */ +/** Tests for {@link JdbcNumericBetweenParametersProvider}. */ class NumericBetweenParametersProviderTest { @Test @@ -52,11 +50,11 @@ void testBatchSizeNotDivisible() { Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[]{-5, -2}, - new long[]{-1, 2}, - new long[]{3, 5}, - new long[]{6, 8}, - new long[]{9, 11} + new long[] {-5, -2}, + new long[] {-1, 2}, + new long[] {3, 5}, + new long[] {6, 8}, + new long[] {9, 11} }; check(expected, actual); } @@ -67,7 +65,7 @@ void testBatchSizeTooLarge() { new JdbcNumericBetweenParametersProvider(0, 2, false).ofBatchSize(5); Serializable[][] actual = provider.getParameterValues(); - long[][] expected = {new long[]{0, 2}}; + long[][] expected = {new long[] {0, 2}}; check(expected, actual); } @@ -78,11 +76,11 @@ void testBatchNumDivisible() { Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[]{-5, -3}, - new long[]{-2, 0}, - new long[]{1, 3}, - new long[]{4, 6}, - new long[]{7, 9} + new long[] {-5, -3}, + new long[] {-2, 0}, + new long[] {1, 3}, + new long[] {4, 6}, + new long[] {7, 9} }; check(expected, actual); } @@ -94,11 +92,11 @@ void testBatchNumNotDivisible() { Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[]{-5, -2}, - new long[]{-1, 2}, - new long[]{3, 5}, - new long[]{6, 8}, - new long[]{9, 11} + new long[] {-5, -2}, + new long[] {-1, 2}, + new long[] {3, 5}, + new long[] {6, 8}, + new long[] {9, 11} }; check(expected, actual); } @@ -110,9 +108,9 @@ void testBatchNumTooLarge() { Serializable[][] actual = provider.getParameterValues(); long[][] expected = { - new long[]{0, 0}, - new long[]{1, 1}, - new long[]{2, 2} + new long[] {0, 0}, + new long[] {1, 1}, + new long[] {2, 2} }; check(expected, actual); } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java index 930b9738d..6dfec7e9e 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSourceITCase.java @@ -76,13 +76,16 @@ import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow; import static org.assertj.core.api.Assertions.assertThat; -/** - * ITCase for {@link JdbcDynamicTableSource}. - */ +/** ITCase for {@link JdbcDynamicTableSource}. */ public abstract class JdbcDynamicTableSourceITCase implements DatabaseTest { @RegisterExtension - static final MiniClusterExtension MINI_CLUSTER_RESOURCE = new MiniClusterExtension(new MiniClusterResourceConfiguration.Builder().setNumberTaskManagers(1).setConfiguration(new Configuration()).build()); + static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setConfiguration(new Configuration()) + .build()); private final TableRow inputTable = createInputTable(); @@ -99,7 +102,11 @@ public abstract class JdbcDynamicTableSourceITCase implements DatabaseTest { } protected TableRow createInputTable() { - return tableRow("jdbDynamicTableSource", field("id", DataTypes.BIGINT().notNull()), field("decimal_col", DataTypes.DECIMAL(10, 4)), field("timestamp6_col", DataTypes.TIMESTAMP(6))); + return tableRow( + "jdbDynamicTableSource", + field("id", DataTypes.BIGINT().notNull()), + field("decimal_col", DataTypes.DECIMAL(10, 4)), + field("timestamp6_col", DataTypes.TIMESTAMP(6))); } @Override @@ -108,7 +115,15 @@ public List getManagedTables() { } protected List getTestData() { - return Arrays.asList(Row.of(1L, BigDecimal.valueOf(100.1234), truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456"))), Row.of(2L, BigDecimal.valueOf(101.1234), truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); + return Arrays.asList( + Row.of( + 1L, + BigDecimal.valueOf(100.1234), + truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456"))), + Row.of( + 2L, + BigDecimal.valueOf(101.1234), + truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); } @BeforeEach @@ -132,9 +147,11 @@ void testJdbcSource() { String testTable = "testTable"; boolean isShardContainString = Arrays.asList(inputTable.getTableFields()).contains(fieldId); if (!isShardContainString) { - throw new IllegalArgumentException("The data column in the jdbc table test must contain the `id`"); + throw new IllegalArgumentException( + "The data column in the jdbc table test must contain the `id`"); } - boolean isPartitionColumnTypeString = (dbNameList.contains(createInputTable().getDbName()) && isShardContainString); + boolean isPartitionColumnTypeString = + (dbNameList.contains(createInputTable().getDbName()) && isShardContainString); DatabaseMetadata metadata = getMetadata(); // Non slice read @@ -144,10 +161,11 @@ void testJdbcSource() { String testReadPartitionString = "testReadPartitionString"; - //string slice read + // string slice read List withParams = new ArrayList<>(); - String partitionColumn = Arrays.asList(inputTable.getTableFields()).contains(fieldStr) ? fieldStr : fieldId; + String partitionColumn = + Arrays.asList(inputTable.getTableFields()).contains(fieldStr) ? fieldStr : fieldId; int partitionNum = 10; int lowerBound = 0; int upperBound = 9; @@ -156,29 +174,39 @@ void testJdbcSource() { withParams.add(String.format("'scan.partition.lower-bound'='%s'", lowerBound)); withParams.add(String.format("'scan.partition.upper-bound'='%s'", upperBound)); - tEnv.executeSql(inputTable.getCreateQueryForFlink(metadata, testReadPartitionString, withParams)); - List collectedPartitionString = executeQuery("SELECT * FROM " + testReadPartitionString); + tEnv.executeSql( + inputTable.getCreateQueryForFlink(metadata, testReadPartitionString, withParams)); + List collectedPartitionString = + executeQuery("SELECT * FROM " + testReadPartitionString); assertThat(collectedPartitionString).containsExactlyInAnyOrderElementsOf(getTestData()); Serializable[][] queryParameters = new Long[3][1]; - queryParameters[0] = new Long[]{0L}; - queryParameters[1] = new Long[]{1L}; - queryParameters[2] = new Long[]{2L}; + queryParameters[0] = new Long[] {0L}; + queryParameters[1] = new Long[] {1L}; + queryParameters[2] = new Long[] {2L}; String partitionKeyName = isPartitionColumnTypeString ? fieldStr : fieldId; String sqlFilterField; switch (createInputTable().getDbName()) { case POSTGRES_DB: - sqlFilterField = new PostgresDialect().hashModForField(partitionKeyName, queryParameters.length); + sqlFilterField = + new PostgresDialect() + .hashModForField(partitionKeyName, queryParameters.length); break; case MYSQL_DB: - sqlFilterField = new MySqlDialect().hashModForField(partitionKeyName, queryParameters.length); + sqlFilterField = + new MySqlDialect() + .hashModForField(partitionKeyName, queryParameters.length); break; case ORACLE_DB: - sqlFilterField = new OracleDialect().hashModForField(partitionKeyName, queryParameters.length); + sqlFilterField = + new OracleDialect() + .hashModForField(partitionKeyName, queryParameters.length); break; case SQL_SERVER_DB: - sqlFilterField = new SqlServerDialect().hashModForField(partitionKeyName, queryParameters.length); + sqlFilterField = + new SqlServerDialect() + .hashModForField(partitionKeyName, queryParameters.length); break; default: sqlFilterField = partitionKeyName; @@ -186,9 +214,30 @@ void testJdbcSource() { } ExecutionEnvironment executionEnvironment = ExecutionEnvironment.getExecutionEnvironment(); - JdbcInputFormat jdbcInputFormat = JdbcInputFormat.buildJdbcInputFormat().setDrivername(metadata.getDriverClass()).setDBUrl(metadata.getJdbcUrl()).setUsername(metadata.getUsername()).setPassword(metadata.getPassword()).setPartitionColumnTypeString(isPartitionColumnTypeString).setQuery("select * from " + inputTable.getTableName() + " where ( " + sqlFilterField + " ) = ?").setRowTypeInfo(inputTable.getTableRowTypeInfo()).setParametersProvider(new JdbcGenericParameterValuesProvider(queryParameters)).finish(); + JdbcInputFormat jdbcInputFormat = + JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(metadata.getDriverClass()) + .setDBUrl(metadata.getJdbcUrl()) + .setUsername(metadata.getUsername()) + .setPassword(metadata.getPassword()) + .setPartitionColumnTypeString(isPartitionColumnTypeString) + .setQuery( + "select * from " + + inputTable.getTableName() + + " where ( " + + sqlFilterField + + " ) = ?") + .setRowTypeInfo(inputTable.getTableRowTypeInfo()) + .setParametersProvider( + new JdbcGenericParameterValuesProvider(queryParameters)) + .finish(); try { - int jdbcInputRowSize = executionEnvironment.createInput(jdbcInputFormat).map(row -> row.getField(fieldId)).collect().size(); + int jdbcInputRowSize = + executionEnvironment + .createInput(jdbcInputFormat) + .map(row -> row.getField(fieldId)) + .collect() + .size(); assertThat(jdbcInputRowSize).isEqualTo(getTestData().size()); } catch (Exception e) { throw new RuntimeException(e); @@ -198,12 +247,23 @@ void testJdbcSource() { @Test void testProject() { String testTable = "testTable"; - tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='2'", "'scan.partition.lower-bound'='0'", "'scan.partition.upper-bound'='100'"))); + tEnv.executeSql( + inputTable.getCreateQueryForFlink( + getMetadata(), + testTable, + Arrays.asList( + "'scan.partition.column'='id'", + "'scan.partition.num'='2'", + "'scan.partition.lower-bound'='0'", + "'scan.partition.upper-bound'='100'"))); String fields = String.join(",", Arrays.copyOfRange(inputTable.getTableFields(), 0, 3)); List collected = executeQuery(String.format("SELECT %s FROM %s", fields, testTable)); - List expected = getTestData().stream().map(row -> Row.of(row.getField(0), row.getField(1), row.getField(2))).collect(Collectors.toList()); + List expected = + getTestData().stream() + .map(row -> Row.of(row.getField(0), row.getField(1), row.getField(2))) + .collect(Collectors.toList()); assertThat(collected).containsExactlyInAnyOrderElementsOf(expected); } @@ -211,12 +271,22 @@ void testProject() { @Test public void testLimit() { String testTable = "testTable"; - tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='2'", "'scan.partition.lower-bound'='1'", "'scan.partition.upper-bound'='2'"))); + tEnv.executeSql( + inputTable.getCreateQueryForFlink( + getMetadata(), + testTable, + Arrays.asList( + "'scan.partition.column'='id'", + "'scan.partition.num'='2'", + "'scan.partition.lower-bound'='1'", + "'scan.partition.upper-bound'='2'"))); List collected = executeQuery("SELECT * FROM " + testTable + " LIMIT 1"); assertThat(collected).hasSize(1); - assertThat(getTestData()).as("The actual output is not a subset of the expected set.").containsAll(collected); + assertThat(getTestData()) + .as("The actual output is not a subset of the expected set.") + .containsAll(collected); } @Test @@ -226,42 +296,95 @@ public void testFilter() { // create a partitioned table to ensure no regression String partitionedTable = "PARTITIONED_TABLE"; - tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), partitionedTable, Arrays.asList("'scan.partition.column'='id'", "'scan.partition.num'='1'", "'scan.partition.lower-bound'='1'", "'scan.partition.upper-bound'='1'"))); + tEnv.executeSql( + inputTable.getCreateQueryForFlink( + getMetadata(), + partitionedTable, + Arrays.asList( + "'scan.partition.column'='id'", + "'scan.partition.num'='1'", + "'scan.partition.lower-bound'='1'", + "'scan.partition.upper-bound'='1'"))); // we create a VIEW here to test column remapping, ie. would filter push down work if we // create a view that depends on our source table - tEnv.executeSql(String.format("CREATE VIEW FAKE_TABLE (idx, %s) as (SELECT * from %s )", Arrays.stream(inputTable.getTableFields()).filter(f -> !f.equals("id")).collect(Collectors.joining(",")), testTable)); - - Row onlyRow1 = getTestData().stream().filter(row -> row.getFieldAs(0).equals(1L)).findAny().orElseThrow(NullPointerException::new); - - Row onlyRow2 = getTestData().stream().filter(row -> row.getFieldAs(0).equals(2L)).findAny().orElseThrow(NullPointerException::new); + tEnv.executeSql( + String.format( + "CREATE VIEW FAKE_TABLE (idx, %s) as (SELECT * from %s )", + Arrays.stream(inputTable.getTableFields()) + .filter(f -> !f.equals("id")) + .collect(Collectors.joining(",")), + testTable)); + + Row onlyRow1 = + getTestData().stream() + .filter(row -> row.getFieldAs(0).equals(1L)) + .findAny() + .orElseThrow(NullPointerException::new); + + Row onlyRow2 = + getTestData().stream() + .filter(row -> row.getFieldAs(0).equals(2L)) + .findAny() + .orElseThrow(NullPointerException::new); List twoRows = getTestData(); // test simple filter - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1")).containsExactly(onlyRow1); + assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1")) + .containsExactly(onlyRow1); // test TIMESTAMP filter - assertThat(executeQuery("SELECT * FROM FAKE_TABLE " + "WHERE timestamp6_col > TIMESTAMP '2020-01-01 15:35:00'" + " AND timestamp6_col < TIMESTAMP '2020-01-01 15:35:01'")).containsExactly(onlyRow1); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE " + + "WHERE timestamp6_col > TIMESTAMP '2020-01-01 15:35:00'" + + " AND timestamp6_col < TIMESTAMP '2020-01-01 15:35:01'")) + .containsExactly(onlyRow1); // test the IN operator - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE 1 = idx AND decimal_col IN (100.1234, 101.1234)")).containsExactly(onlyRow1); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE WHERE 1 = idx AND decimal_col IN (100.1234, 101.1234)")) + .containsExactly(onlyRow1); // test mixing AND and OR operator - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1 AND decimal_col = 100.1234 OR decimal_col = 101.1234")).containsExactlyInAnyOrderElementsOf(twoRows); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE WHERE idx = 1 AND decimal_col = 100.1234 OR decimal_col = 101.1234")) + .containsExactlyInAnyOrderElementsOf(twoRows); // test mixing AND/OR with parenthesis, and the swapping the operand of equal expression - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE (2 = idx AND decimal_col = 100.1234) OR decimal_col = 101.1234")).containsExactly(onlyRow2); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE WHERE (2 = idx AND decimal_col = 100.1234) OR decimal_col = 101.1234")) + .containsExactly(onlyRow2); // test Greater than, just to make sure we didnt break anything that we cannot pushdown - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 2 AND decimal_col > 100 OR decimal_col = 101.123")).containsExactly(onlyRow2); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE WHERE idx = 2 AND decimal_col > 100 OR decimal_col = 101.123")) + .containsExactly(onlyRow2); // One more test of parenthesis - assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE 2 = idx AND (decimal_col = 100.1234 OR decimal_col = 102.1234)")).isEmpty(); - - assertThat(executeQuery("SELECT * FROM " + partitionedTable + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123")).isEmpty(); - - assertThat(executeQuery("SELECT * FROM " + partitionedTable + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)")).containsExactly(onlyRow1); + assertThat( + executeQuery( + "SELECT * FROM FAKE_TABLE WHERE 2 = idx AND (decimal_col = 100.1234 OR decimal_col = 102.1234)")) + .isEmpty(); + + assertThat( + executeQuery( + "SELECT * FROM " + + partitionedTable + + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123")) + .isEmpty(); + + assertThat( + executeQuery( + "SELECT * FROM " + + partitionedTable + + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)")) + .containsExactly(onlyRow1); } @ParameterizedTest @@ -270,13 +393,32 @@ void testLookupJoin(Caching caching) { // Create JDBC lookup table List cachingOptions = Collections.emptyList(); if (caching.equals(Caching.ENABLE_CACHE)) { - cachingOptions = Arrays.asList("'lookup.cache.max-rows' = '100'", "'lookup.cache.ttl' = '10min'"); + cachingOptions = + Arrays.asList( + "'lookup.cache.max-rows' = '100'", "'lookup.cache.ttl' = '10min'"); } - tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), "jdbc_lookup", cachingOptions)); + tEnv.executeSql( + inputTable.getCreateQueryForFlink(getMetadata(), "jdbc_lookup", cachingOptions)); // Create and prepare a value source - String dataId = TestValuesTableFactory.registerData(Arrays.asList(Row.of(1L, "Alice"), Row.of(1L, "Alice"), Row.of(2L, "Bob"), Row.of(3L, "Charlie"))); - tEnv.executeSql(String.format("CREATE TABLE value_source ( " + " `id` BIGINT, " + " `name` STRING, " + " `proctime` AS PROCTIME()" + ") WITH (" + " 'connector' = 'values', " + " 'data-id' = '%s'" + ")", dataId)); + String dataId = + TestValuesTableFactory.registerData( + Arrays.asList( + Row.of(1L, "Alice"), + Row.of(1L, "Alice"), + Row.of(2L, "Bob"), + Row.of(3L, "Charlie"))); + tEnv.executeSql( + String.format( + "CREATE TABLE value_source ( " + + " `id` BIGINT, " + + " `name` STRING, " + + " `proctime` AS PROCTIME()" + + ") WITH (" + + " 'connector' = 'values', " + + " 'data-id' = '%s'" + + ")", + dataId)); if (caching == Caching.ENABLE_CACHE) { LookupCacheManager.keepCacheOnRelease(true); @@ -284,13 +426,37 @@ void testLookupJoin(Caching caching) { // Execute lookup join try { - List collected = executeQuery("SELECT S.id, S.name, D.id, D.timestamp6_col, D.decimal_col FROM value_source" + " AS S JOIN jdbc_lookup for system_time as of S.proctime AS D ON S.id = D.id"); + List collected = + executeQuery( + "SELECT S.id, S.name, D.id, D.timestamp6_col, D.decimal_col FROM value_source" + + " AS S JOIN jdbc_lookup for system_time as of S.proctime AS D ON S.id = D.id"); assertThat(collected).hasSize(3); - List expected = Arrays.asList(Row.of(1L, "Alice", 1L, truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), BigDecimal.valueOf(100.1234)), Row.of(1L, "Alice", 1L, truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), BigDecimal.valueOf(100.1234)), Row.of(2L, "Bob", 2L, truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")), BigDecimal.valueOf(101.1234))); - - assertThat(collected).as("The actual output is not a subset of the expected set").containsAll(expected); + List expected = + Arrays.asList( + Row.of( + 1L, + "Alice", + 1L, + truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), + BigDecimal.valueOf(100.1234)), + Row.of( + 1L, + "Alice", + 1L, + truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")), + BigDecimal.valueOf(100.1234)), + Row.of( + 2L, + "Bob", + 2L, + truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")), + BigDecimal.valueOf(101.1234))); + + assertThat(collected) + .as("The actual output is not a subset of the expected set") + .containsAll(expected); if (caching == Caching.ENABLE_CACHE) { validateCachedValues(); @@ -318,15 +484,26 @@ private List executeQuery(String query) { private void validateCachedValues() { // Validate cache - Map managedCaches = LookupCacheManager.getInstance().getManagedCaches(); + Map managedCaches = + LookupCacheManager.getInstance().getManagedCaches(); assertThat(managedCaches).as("There should be only 1 shared cache registered").hasSize(1); LookupCache cache = managedCaches.get(managedCaches.keySet().iterator().next()).getCache(); // jdbc does support project push down, the cached row has been projected RowData key1 = GenericRowData.of(1L); - RowData value1 = GenericRowData.of(1L, DecimalData.fromBigDecimal(BigDecimal.valueOf(100.1234), 10, 4), TimestampData.fromLocalDateTime(truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")))); + RowData value1 = + GenericRowData.of( + 1L, + DecimalData.fromBigDecimal(BigDecimal.valueOf(100.1234), 10, 4), + TimestampData.fromLocalDateTime( + truncateTime(LocalDateTime.parse("2020-01-01T15:35:00.123456")))); RowData key2 = GenericRowData.of(2L); - RowData value2 = GenericRowData.of(2L, DecimalData.fromBigDecimal(BigDecimal.valueOf(101.1234), 10, 4), TimestampData.fromLocalDateTime(truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); + RowData value2 = + GenericRowData.of( + 2L, + DecimalData.fromBigDecimal(BigDecimal.valueOf(101.1234), 10, 4), + TimestampData.fromLocalDateTime( + truncateTime(LocalDateTime.parse("2020-01-01T15:36:01.123456")))); RowData key3 = GenericRowData.of(3L); @@ -339,6 +516,7 @@ private void validateCachedValues() { } private enum Caching { - ENABLE_CACHE, DISABLE_CACHE + ENABLE_CACHE, + DISABLE_CACHE } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java index 55bd4ecdd..4c93f8c0e 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataInputFormatTest.java @@ -51,16 +51,32 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** - * Test suite for {@link JdbcRowDataInputFormat}. - */ +/** Test suite for {@link JdbcRowDataInputFormat}. */ class JdbcRowDataInputFormatTest extends JdbcDataTestBase { private JdbcRowDataInputFormat inputFormat; - private static final String[] fieldNames = new String[]{"id", "title", "author", "price", "qty"}; - private static final DataType[] fieldDataTypes = new DataType[]{DataTypes.INT(), DataTypes.STRING(), DataTypes.STRING(), DataTypes.DOUBLE(), DataTypes.INT()}; - final JdbcDialect dialect = InternalJdbcConnectionOptions.builder().setDBUrl(getMetadata().getJdbcUrl()).setTableName(INPUT_TABLE).build().getDialect(); - final RowType rowType = RowType.of(Arrays.stream(fieldDataTypes).map(DataType::getLogicalType).toArray(LogicalType[]::new), fieldNames); + private static final String[] fieldNames = + new String[] {"id", "title", "author", "price", "qty"}; + private static final DataType[] fieldDataTypes = + new DataType[] { + DataTypes.INT(), + DataTypes.STRING(), + DataTypes.STRING(), + DataTypes.DOUBLE(), + DataTypes.INT() + }; + final JdbcDialect dialect = + InternalJdbcConnectionOptions.builder() + .setDBUrl(getMetadata().getJdbcUrl()) + .setTableName(INPUT_TABLE) + .build() + .getDialect(); + final RowType rowType = + RowType.of( + Arrays.stream(fieldDataTypes) + .map(DataType::getLogicalType) + .toArray(LogicalType[]::new), + fieldNames); @AfterEach void tearDown() throws IOException { @@ -73,65 +89,134 @@ void tearDown() throws IOException { @Test void testNoRowConverter() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).build(); - inputFormat.openInputFormat(); - }).isInstanceOf(NullPointerException.class); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .build(); + inputFormat.openInputFormat(); + }) + .isInstanceOf(NullPointerException.class); } @Test void testInvalidDriver() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername("org.apache.derby.jdbc.idontexist").setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); - inputFormat.openInputFormat(); - }).isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername("org.apache.derby.jdbc.idontexist") + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); + inputFormat.openInputFormat(); + }) + .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidURL() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl("jdbc:der:iamanerror:mory:ebookshop").setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); - inputFormat.openInputFormat(); - }).isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl("jdbc:der:iamanerror:mory:ebookshop") + .setQuery(SELECT_ALL_BOOKS) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); + inputFormat.openInputFormat(); + }) + .isInstanceOf(IllegalArgumentException.class); } @Test void testInvalidQuery() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery("iamnotsql").setRowConverter(dialect.getRowConverter(rowType)).build(); - inputFormat.openInputFormat(); - }).isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery("iamnotsql") + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); + inputFormat.openInputFormat(); + }) + .isInstanceOf(IllegalArgumentException.class); } @Test void testNoQuery() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setRowConverter(dialect.getRowConverter(rowType)).build(); - }).isInstanceOf(NullPointerException.class).hasMessage("No query supplied"); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); + }) + .isInstanceOf(NullPointerException.class) + .hasMessage("No query supplied"); } @Test void testNoUrl() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setQuery(SELECT_ALL_BOOKS).setRowConverter(dialect.getRowConverter(rowType)).build(); - }).isInstanceOf(NullPointerException.class).hasMessage("jdbc url is empty"); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setQuery(SELECT_ALL_BOOKS) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); + }) + .isInstanceOf(NullPointerException.class) + .hasMessage("jdbc url is empty"); } @Test void testInvalidFetchSize() { - assertThatThrownBy(() -> { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setFetchSize(-7).build(); - }).isInstanceOf(IllegalArgumentException.class); + assertThatThrownBy( + () -> { + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setFetchSize(-7) + .build(); + }) + .isInstanceOf(IllegalArgumentException.class); } @Test void testValidFetchSizeIntegerMin() { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setFetchSize(Integer.MIN_VALUE).setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setFetchSize(Integer.MIN_VALUE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); } @Test void testJdbcInputFormatWithoutParallelism() throws IOException { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); // this query does not exploit parallelism assertThat(inputFormat.createInputSplits(1)).hasSize(1); inputFormat.openInputFormat(); @@ -155,8 +240,17 @@ void testJdbcInputFormatWithParallelismAndNumericColumnSplitting() throws IOExce final int fetchSize = 1; final long min = TEST_DATA[0].id; final long max = TEST_DATA[TEST_DATA.length - fetchSize].id; - JdbcParameterValuesProvider pramProvider = new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID).setParametersProvider(pramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); + JdbcParameterValuesProvider pramProvider = + new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setParametersProvider(pramProvider) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -184,8 +278,17 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE final long min = TEST_DATA[0].id; final long max = TEST_DATA[TEST_DATA.length - 1].id; final long fetchSize = max + 1; // generate a single split - JdbcParameterValuesProvider pramProvider = new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID).setParametersProvider(pramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); + JdbcParameterValuesProvider pramProvider = + new JdbcNumericBetweenParametersProvider(min, max, false).ofBatchSize(fetchSize); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID) + .setParametersProvider(pramProvider) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -211,10 +314,19 @@ void testJdbcInputFormatWithoutParallelismAndNumericColumnSplitting() throws IOE @Test void testJdbcInputFormatWithParallelismAndGenericSplitting() throws IOException { Serializable[][] queryParameters = new String[2][1]; - queryParameters[0] = new String[]{TEST_DATA[3].author}; - queryParameters[1] = new String[]{TEST_DATA[0].author}; - JdbcParameterValuesProvider paramProvider = new JdbcGenericParameterValuesProvider(queryParameters); - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR).setParametersProvider(paramProvider).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); + queryParameters[0] = new String[] {TEST_DATA[3].author}; + queryParameters[1] = new String[] {TEST_DATA[0].author}; + JdbcParameterValuesProvider paramProvider = + new JdbcGenericParameterValuesProvider(queryParameters); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR) + .setParametersProvider(paramProvider) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); inputFormat.openInputFormat(); InputSplit[] splits = inputFormat.createInputSplits(1); @@ -248,7 +360,14 @@ private void verifySplit(InputSplit split, int expectedIDSum) throws IOException @Test void testEmptyResults() throws IOException { - inputFormat = JdbcRowDataInputFormat.builder().setDrivername(getMetadata().getDriverClass()).setDBUrl(getMetadata().getJdbcUrl()).setQuery(SELECT_EMPTY).setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE).setRowConverter(dialect.getRowConverter(rowType)).build(); + inputFormat = + JdbcRowDataInputFormat.builder() + .setDrivername(getMetadata().getDriverClass()) + .setDBUrl(getMetadata().getJdbcUrl()) + .setQuery(SELECT_EMPTY) + .setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE) + .setRowConverter(dialect.getRowConverter(rowType)) + .build(); try { inputFormat.openInputFormat(); @@ -261,10 +380,15 @@ void testEmptyResults() throws IOException { } private static void assertEquals(JdbcTestFixture.TestEntry expected, RowData actual) { - assertThat(actual.isNullAt(0) ? null : Integer.valueOf(actual.getInt(0))).isEqualTo(expected.id); - assertThat(actual.isNullAt(1) ? null : actual.getString(1).toString()).isEqualTo(expected.title); - assertThat(actual.isNullAt(2) ? null : actual.getString(2).toString()).isEqualTo(expected.author); - assertThat(actual.isNullAt(3) ? null : Double.valueOf(actual.getDouble(3))).isEqualTo(expected.price); - assertThat(actual.isNullAt(4) ? null : Integer.valueOf(actual.getInt(4))).isEqualTo(expected.qty); + assertThat(actual.isNullAt(0) ? null : Integer.valueOf(actual.getInt(0))) + .isEqualTo(expected.id); + assertThat(actual.isNullAt(1) ? null : actual.getString(1).toString()) + .isEqualTo(expected.title); + assertThat(actual.isNullAt(2) ? null : actual.getString(2).toString()) + .isEqualTo(expected.author); + assertThat(actual.isNullAt(3) ? null : Double.valueOf(actual.getDouble(3))) + .isEqualTo(expected.price); + assertThat(actual.isNullAt(4) ? null : Integer.valueOf(actual.getInt(4))) + .isEqualTo(expected.qty); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java index 338f453e0..0953a3886 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/DbName.java @@ -18,11 +18,8 @@ package org.apache.flink.connector.jdbc.testutils.databases; -/** - * Define a database name. * - */ +/** Define a database name. * */ public enum DbName { - POSTGRES_DB, MYSQL_DB, ORACLE_DB, @@ -30,6 +27,4 @@ public enum DbName { TRINO_DB, DB2_DB, DERBY_DB - - } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java index c2d70f00f..fabe09639 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java @@ -50,9 +50,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -/** - * Base table operations. * - */ +/** Base table operations. * */ public abstract class TableBase implements TableManaged { private final String name; @@ -275,7 +273,7 @@ protected void executeUpdate(Connection conn, String sql) throws SQLException { protected List executeStatement( Connection conn, String sql, JdbcResultSetBuilder rsGetter) throws SQLException { try (Statement st = conn.createStatement(); - ResultSet rs = st.executeQuery(sql)) { + ResultSet rs = st.executeQuery(sql)) { return rsGetter.accept(rs); } } diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java index 659fc19ee..4c95a03dd 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java @@ -21,9 +21,7 @@ import org.apache.flink.connector.jdbc.testutils.databases.DbName; import org.apache.flink.table.types.DataType; -/** - * Table builder. * - */ +/** Table builder. * */ public final class TableBuilder { public static TableRow tableRow(String name, TableField... fields) { diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java index 5040118ec..2812ab66b 100644 --- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java +++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java @@ -41,9 +41,7 @@ import static org.assertj.core.api.Assertions.assertThat; -/** - * Row table. * - */ +/** Row table. * */ public class TableRow extends TableBase { protected TableRow(String name, DbName dbName, TableField[] fields) { From fc60ad3e567e610e2fdeb9a62c8130ae60ed77ef Mon Sep 17 00:00:00 2001 From: zhilinli Date: Wed, 6 Mar 2024 16:04:37 +0800 Subject: [PATCH 5/5] fix CI error --- .../flink/connector/jdbc/table/JdbcDynamicTableFactory.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java index 48560f64b..b76c07684 100644 --- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java +++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableFactory.java @@ -124,13 +124,12 @@ public DynamicTableSource createDynamicTableSource(Context context) { config.get(URL), config.get(COMPATIBLE_MODE), context.getClassLoader()); - physicalRowDataType, config.get(URL), context.getClassLoader()); return new JdbcDynamicTableSource( getJdbcOptions(helper.getOptions(), context.getClassLoader()), getJdbcReadOptions(helper.getOptions(), physicalRowDataType), helper.getOptions().get(LookupOptions.MAX_RETRIES), getLookupCache(config), - physicalRowDataType); + context.getPhysicalRowDataType()); } private static void validateDataTypeWithJdbcDialect(