[FLINK-30790][Connector/JDBC] Add DatabaseExtension with TableManaged for testing. This closes #22
* [FLINK-30790] Introduce DatabaseExtension
* [FLINK-30790] Adapt MySql to DatabaseExtension
* [FLINK-30790] Adapt Oracle to DatabaseExtension
* [FLINK-30790] Adapt Postgres to DatabaseExtension
* [FLINK-30790] Adapt SqlServer to DatabaseExtension
* [FLINK-30790] Adapt test databases (Derby and H2) to DatabaseExtension
* [FLINK-15462] Cleaning and fixing JdbcExactlyOnceSinkE2eTest
* [FLINK-15462] Cleaning and fixing JdbcDynamicTableSinkITCase
* [FLINK-15462] Cleaning and fixing JdbcDynamicTableSourceITCase
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/JdbcCatalogUtils.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/JdbcCatalogUtils.java
index 28bea80..a5c6c1e 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/JdbcCatalogUtils.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/JdbcCatalogUtils.java
@@ -18,10 +18,12 @@
package org.apache.flink.connector.jdbc.catalog;
+import org.apache.flink.connector.jdbc.databases.mysql.catalog.MySqlCatalog;
+import org.apache.flink.connector.jdbc.databases.mysql.dialect.MySqlDialect;
+import org.apache.flink.connector.jdbc.databases.postgres.catalog.PostgresCatalog;
+import org.apache.flink.connector.jdbc.databases.postgres.dialect.PostgresDialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectLoader;
-import org.apache.flink.connector.jdbc.dialect.mysql.MySqlDialect;
-import org.apache.flink.connector.jdbc.dialect.psql.PostgresDialect;
import static org.apache.flink.util.Preconditions.checkArgument;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialect.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java
index a701f4b..dad028f 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialect.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialect.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.derby;
+package org.apache.flink.connector.jdbc.databases.derby.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
-import org.apache.flink.connector.jdbc.internal.converter.DerbyRowConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
@@ -28,7 +28,9 @@
import java.util.Optional;
import java.util.Set;
-class DerbyDialect extends AbstractDialect {
+/** JDBC dialect for Derby. */
+@Internal
+public class DerbyDialect extends AbstractDialect {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialectFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialectFactory.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialectFactory.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialectFactory.java
index 39b0b1c..280038d 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/derby/DerbyDialectFactory.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyDialectFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.derby;
+package org.apache.flink.connector.jdbc.databases.derby.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/DerbyRowConverter.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyRowConverter.java
similarity index 91%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/DerbyRowConverter.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyRowConverter.java
index b2d4dc3..7f05a30 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/DerbyRowConverter.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/derby/dialect/DerbyRowConverter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.internal.converter;
+package org.apache.flink.connector.jdbc.databases.derby.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
import org.apache.flink.table.types.logical.RowType;
@@ -25,6 +26,7 @@
* Runtime converter that responsible to convert between JDBC object and Flink internal object for
* Derby.
*/
+@Internal
public class DerbyRowConverter extends AbstractJdbcRowConverter {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalog.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalog.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalog.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalog.java
index d88b228..b54aa23 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalog.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalog.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.mysql.catalog;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.catalog.AbstractJdbcCatalog;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeMapper;
-import org.apache.flink.connector.jdbc.dialect.mysql.MySqlTypeMapper;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlTypeMapper.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlTypeMapper.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlTypeMapper.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlTypeMapper.java
index 732630b..995e091 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlTypeMapper.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlTypeMapper.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.mysql.catalog;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeMapper;
@@ -119,7 +119,7 @@
// BINARY is not supported in MySqlDialect now.
// VARBINARY(n) is not supported in MySqlDialect when 'n' is not equals to
// Integer.MAX_VALUE. Please see
- // org.apache.flink.connector.jdbc.dialect.mysql.MySqlDialect#supportedTypes and
+ // org.apache.flink.connector.jdbc.databases.mysql.dialect.MySqlDialect#supportedTypes and
// org.apache.flink.connector.jdbc.dialect.AbstractDialect#validate for more
// details.
return DataTypes.BYTES();
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/MySQLRowConverter.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySQLRowConverter.java
similarity index 91%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/MySQLRowConverter.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySQLRowConverter.java
index c9a77d6..ef5d0fd 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/MySQLRowConverter.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySQLRowConverter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.internal.converter;
+package org.apache.flink.connector.jdbc.databases.mysql.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
import org.apache.flink.table.types.logical.RowType;
@@ -25,6 +26,7 @@
* Runtime converter that responsible to convert between JDBC object and Flink internal object for
* MySQL.
*/
+@Internal
public class MySQLRowConverter extends AbstractJdbcRowConverter {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialect.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialect.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialect.java
index b441559..6d79d30 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialect.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialect.java
@@ -16,12 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.mysql.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
-import org.apache.flink.connector.jdbc.internal.converter.MySQLRowConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectFactory.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectFactory.java
index 6ec301a..71ae04c 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.mysql.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java
similarity index 96%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialect.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java
index b8f229e..1ed061f 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialect.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialect.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.oracle;
+package org.apache.flink.connector.jdbc.databases.oracle.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
-import org.apache.flink.connector.jdbc.internal.converter.OracleRowConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
@@ -31,7 +31,8 @@
import java.util.stream.Collectors;
/** JDBC dialect for Oracle. */
-class OracleDialect extends AbstractDialect {
+@Internal
+public class OracleDialect extends AbstractDialect {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectFactory.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectFactory.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectFactory.java
index 7a3ca11..b7bc60e 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectFactory.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.oracle;
+package org.apache.flink.connector.jdbc.databases.oracle.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/OracleRowConverter.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleRowConverter.java
similarity index 98%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/OracleRowConverter.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleRowConverter.java
index 34e30a8..b476e66 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/OracleRowConverter.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleRowConverter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.internal.converter;
+package org.apache.flink.connector.jdbc.databases.oracle.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.StringData;
@@ -48,6 +49,7 @@
* Runtime converter that responsible to convert between JDBC object and Flink internal object for
* Oracle.
*/
+@Internal
public class OracleRowConverter extends AbstractJdbcRowConverter {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalog.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalog.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalog.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalog.java
index de41c01..0b5cbeb 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalog.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalog.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
import org.apache.flink.annotation.Internal;
+import org.apache.flink.connector.jdbc.catalog.AbstractJdbcCatalog;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeMapper;
-import org.apache.flink.connector.jdbc.dialect.psql.PostgresTypeMapper;
import org.apache.flink.table.catalog.ObjectPath;
import org.apache.flink.table.catalog.exceptions.CatalogException;
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
@@ -70,7 +70,7 @@
private final JdbcDialectTypeMapper dialectTypeMapper;
- protected PostgresCatalog(
+ public PostgresCatalog(
ClassLoader userClassLoader,
String catalogName,
String defaultDatabase,
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePath.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePath.java
similarity index 93%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePath.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePath.java
index 1c84b42..199a081 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePath.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePath.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.util.StringUtils;
import java.util.Objects;
@@ -25,9 +26,10 @@
import static org.apache.flink.util.Preconditions.checkArgument;
/**
- * Table path of PostgreSQL in Flink. Can be of formats "table_name" or "schema_name.table_name".
+ * Table path of PostgresSQL in Flink. Can be of formats "table_name" or "schema_name.table_name".
* When it's "table_name", the schema name defaults to "public".
*/
+@Internal
public class PostgresTablePath {
private static final String DEFAULT_POSTGRES_SCHEMA_NAME = "public";
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresTypeMapper.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTypeMapper.java
similarity index 98%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresTypeMapper.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTypeMapper.java
index b6c478a..b2769d7 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresTypeMapper.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTypeMapper.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.psql;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeMapper;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java
similarity index 96%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialect.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java
index cec0434..f5b4af2 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialect.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialect.java
@@ -16,11 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.psql;
+package org.apache.flink.connector.jdbc.databases.postgres.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
-import org.apache.flink.connector.jdbc.internal.converter.PostgresRowConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
@@ -30,7 +30,8 @@
import java.util.Set;
import java.util.stream.Collectors;
-/** JDBC dialect for PostgreSQL. */
+/** JDBC dialect for PostgresSQL. */
+@Internal
public class PostgresDialect extends AbstractDialect {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialectFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectFactory.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialectFactory.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectFactory.java
index 374f101..6286f14 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/psql/PostgresDialectFactory.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.psql;
+package org.apache.flink.connector.jdbc.databases.postgres.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/PostgresRowConverter.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresRowConverter.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/PostgresRowConverter.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresRowConverter.java
index 265476a..4f302cc 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/PostgresRowConverter.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresRowConverter.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.internal.converter;
+package org.apache.flink.connector.jdbc.databases.postgres.dialect;
import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
import org.apache.flink.table.data.GenericArrayData;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialect.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java
similarity index 97%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialect.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java
index 22663c1..91469a6 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialect.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialect.java
@@ -16,12 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.sqlserver.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.JdbcRowConverter;
import org.apache.flink.connector.jdbc.dialect.AbstractDialect;
-import org.apache.flink.connector.jdbc.internal.converter.SqlServerRowConverter;
import org.apache.flink.table.types.logical.LogicalTypeRoot;
import org.apache.flink.table.types.logical.RowType;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialectFactory.java
similarity index 94%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialectFactory.java
index 3e71166..9b6ee18 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerDialectFactory.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.sqlserver.dialect;
import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/SqlServerRowConverter.java b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerRowConverter.java
similarity index 93%
rename from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/SqlServerRowConverter.java
rename to flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerRowConverter.java
index 4e8a1af..5f3ac2b 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/internal/converter/SqlServerRowConverter.java
+++ b/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerRowConverter.java
@@ -16,8 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.internal.converter;
+package org.apache.flink.connector.jdbc.databases.sqlserver.dialect;
+import org.apache.flink.annotation.Internal;
import org.apache.flink.connector.jdbc.converter.AbstractJdbcRowConverter;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
@@ -26,6 +27,7 @@
* Runtime converter that responsible to convert between JDBC object and Flink internal object for
* MsSql.
*/
+@Internal
public class SqlServerRowConverter extends AbstractJdbcRowConverter {
private static final long serialVersionUID = 1L;
diff --git a/flink-connector-jdbc/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory b/flink-connector-jdbc/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory
index 20c1f6a..4d12a47 100644
--- a/flink-connector-jdbc/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory
+++ b/flink-connector-jdbc/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory
@@ -13,8 +13,8 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.flink.connector.jdbc.dialect.derby.DerbyDialectFactory
-org.apache.flink.connector.jdbc.dialect.mysql.MySqlDialectFactory
-org.apache.flink.connector.jdbc.dialect.psql.PostgresDialectFactory
-org.apache.flink.connector.jdbc.dialect.oracle.OracleDialectFactory
-org.apache.flink.connector.jdbc.dialect.sqlserver.SqlServerDialectFactory
+org.apache.flink.connector.jdbc.databases.derby.dialect.DerbyDialectFactory
+org.apache.flink.connector.jdbc.databases.mysql.dialect.MySqlDialectFactory
+org.apache.flink.connector.jdbc.databases.postgres.dialect.PostgresDialectFactory
+org.apache.flink.connector.jdbc.databases.oracle.dialect.OracleDialectFactory
+org.apache.flink.connector.jdbc.databases.sqlserver.dialect.SqlServerDialectFactory
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcDataTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcDataTestBase.java
index de23d88..a3ec29f 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcDataTestBase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcDataTestBase.java
@@ -19,8 +19,8 @@
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.common.functions.RuntimeContext;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
+import org.apache.flink.connector.jdbc.testutils.databases.derby.DerbyMetadata;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
@@ -31,12 +31,11 @@
import java.sql.SQLException;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.mockito.Mockito.doReturn;
/**
- * Base class for JDBC test using data from {@link JdbcTestFixture}. It uses {@link
- * org.apache.flink.connector.jdbc.databases.derby.DerbyMetadata} and inserts data before each test.
+ * Base class for JDBC test using data from {@link JdbcTestFixture}. It uses {@link DerbyMetadata}
+ * and inserts data before each test.
*/
public abstract class JdbcDataTestBase extends JdbcTestBase {
@BeforeEach
@@ -44,11 +43,6 @@
JdbcTestFixture.initData(getMetadata());
}
- @Override
- public DatabaseMetadata getMetadata() {
- return DERBY_EBOOKSHOP_DB;
- }
-
public static Row toRow(JdbcTestFixture.TestEntry entry) {
Row row = new Row(5);
row.setField(0, entry.id);
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
index 7dfc03f..ead2c44 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcITCase.java
@@ -19,7 +19,6 @@
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.util.function.FunctionWithException;
@@ -40,7 +39,6 @@
import static org.apache.flink.configuration.PipelineOptions.OBJECT_REUSE;
import static org.apache.flink.connector.jdbc.JdbcConnectionOptions.JdbcConnectionOptionsBuilder;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INSERT_TEMPLATE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.TEST_DATA;
@@ -168,11 +166,6 @@
return result;
}
- @Override
- public DatabaseMetadata getMetadata() {
- return DERBY_EBOOKSHOP_DB;
- }
-
private static <T> T getNullable(
ResultSet rs, FunctionWithException<ResultSet, T, SQLException> get)
throws SQLException {
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 ae45787..f72434d 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
@@ -34,7 +34,6 @@
import java.sql.SQLException;
import java.sql.Statement;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.ROW_TYPE_INFO;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR;
@@ -65,8 +64,8 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.finish();
jdbcInputFormat.openInputFormat();
@@ -82,7 +81,7 @@
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
.setDrivername("org.apache.derby.jdbc.idontexist")
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
@@ -97,7 +96,7 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
@@ -113,8 +112,8 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery("iamnotsql")
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
@@ -129,7 +128,7 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
@@ -144,8 +143,8 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
})
@@ -159,8 +158,8 @@
() -> {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.setFetchSize(-7)
@@ -173,8 +172,8 @@
void testValidFetchSizeIntegerMin() {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.setFetchSize(Integer.MIN_VALUE)
@@ -185,14 +184,14 @@
void testDefaultFetchSizeIsUsedIfNotConfiguredOtherwise() throws SQLException {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
jdbcInputFormat.openInputFormat();
- try (Connection dbConn = DERBY_EBOOKSHOP_DB.getConnection();
+ try (Connection dbConn = getMetadata().getConnection();
Statement dbStatement = dbConn.createStatement();
Statement inputStatement = jdbcInputFormat.getStatement()) {
assertThat(inputStatement.getFetchSize()).isEqualTo(dbStatement.getFetchSize());
@@ -204,8 +203,8 @@
final int desiredFetchSize = 10_000;
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.setFetchSize(desiredFetchSize)
@@ -220,14 +219,14 @@
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.finish();
jdbcInputFormat.openInputFormat();
- final boolean defaultAutoCommit = DERBY_EBOOKSHOP_DB.getConnection().getAutoCommit();
+ final boolean defaultAutoCommit = getMetadata().getConnection().getAutoCommit();
assertThat(jdbcInputFormat.getDbConn().getAutoCommit()).isEqualTo(defaultAutoCommit);
}
@@ -238,8 +237,8 @@
final boolean desiredAutoCommit = false;
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.setAutoCommit(desiredAutoCommit)
@@ -253,8 +252,8 @@
void testJdbcInputFormatWithoutParallelism() throws IOException {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowTypeInfo(ROW_TYPE_INFO)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
@@ -286,8 +285,8 @@
new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize);
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID)
.setRowTypeInfo(ROW_TYPE_INFO)
.setParametersProvider(pramProvider)
@@ -324,8 +323,8 @@
new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize);
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID)
.setRowTypeInfo(ROW_TYPE_INFO)
.setParametersProvider(pramProvider)
@@ -362,8 +361,8 @@
new JdbcGenericParameterValuesProvider(queryParameters);
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR)
.setRowTypeInfo(ROW_TYPE_INFO)
.setParametersProvider(paramProvider)
@@ -403,8 +402,8 @@
void testEmptyResults() throws IOException {
jdbcInputFormat =
JdbcInputFormat.buildJdbcInputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_EMPTY)
.setRowTypeInfo(ROW_TYPE_INFO)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcRowOutputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcRowOutputFormatTest.java
index fca5d11..692d741 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcRowOutputFormatTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcRowOutputFormatTest.java
@@ -33,7 +33,6 @@
import java.sql.Statement;
import java.sql.Types;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INSERT_TEMPLATE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.OUTPUT_TABLE;
@@ -62,7 +61,7 @@
}
jdbcOutputFormat = null;
- try (Connection conn = DERBY_EBOOKSHOP_DB.getConnection();
+ try (Connection conn = getMetadata().getConnection();
Statement stat = conn.createStatement()) {
stat.execute("DELETE FROM " + OUTPUT_TABLE);
}
@@ -75,7 +74,7 @@
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
.setDrivername("org.apache.derby.jdbc.idontexist")
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
.finish();
jdbcOutputFormat.open(0, 1);
@@ -91,7 +90,7 @@
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
.finish();
@@ -106,8 +105,8 @@
try {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery("iamnotsql")
.finish();
setRuntimeContext(jdbcOutputFormat, true);
@@ -124,7 +123,7 @@
try {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
.finish();
} catch (Exception e) {
@@ -139,8 +138,8 @@
try {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, INPUT_TABLE))
.finish();
setRuntimeContext(jdbcOutputFormat, true);
@@ -167,8 +166,8 @@
try {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE))
.setSqlTypes(
new int[] {
@@ -203,8 +202,8 @@
try {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE))
.setSqlTypes(
new int[] {
@@ -240,8 +239,8 @@
void testJdbcOutputFormat() throws IOException, SQLException {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE))
.finish();
setRuntimeContext(jdbcOutputFormat, true);
@@ -253,7 +252,7 @@
jdbcOutputFormat.close();
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS);
ResultSet resultSet = statement.executeQuery()) {
int recordCount = 0;
@@ -274,13 +273,13 @@
void testFlush() throws SQLException, IOException {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE_2))
.setBatchSize(3)
.finish();
setRuntimeContext(jdbcOutputFormat, true);
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS_2)) {
jdbcOutputFormat.open(0, 1);
for (int i = 0; i < 2; ++i) {
@@ -314,8 +313,8 @@
void testInvalidConnectionInJdbcOutputFormat() throws IOException, SQLException {
jdbcOutputFormat =
JdbcRowOutputFormat.buildJdbcOutputFormat()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(String.format(INSERT_TEMPLATE, OUTPUT_TABLE_3))
.finish();
setRuntimeContext(jdbcOutputFormat, true);
@@ -335,7 +334,7 @@
jdbcOutputFormat.close();
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS_3);
ResultSet resultSet = statement.executeQuery()) {
int recordCount = 0;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestBase.java
index a8b673c..6959fb9 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestBase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestBase.java
@@ -17,7 +17,7 @@
package org.apache.flink.connector.jdbc;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
@@ -26,7 +26,7 @@
* Base class for JDBC test using DDL from {@link JdbcTestFixture}. It uses create tables before
* each test and drops afterwards.
*/
-public abstract class JdbcTestBase implements DatabaseTest {
+public abstract class JdbcTestBase implements DerbyTestBase {
@BeforeEach
public void before() throws Exception {
@@ -35,7 +35,6 @@
@AfterEach
public void after() throws Exception {
- JdbcTestFixture.cleanupData(getMetadata().getJdbcUrl());
JdbcTestFixture.cleanUpDatabasesStatic(getMetadata());
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestFixture.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestFixture.java
index ae1c688..4c7519d 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestFixture.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/JdbcTestFixture.java
@@ -20,8 +20,8 @@
import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
import org.apache.flink.api.java.typeutils.RowTypeInfo;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.derby.DerbyDatabase;
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import org.apache.flink.table.types.logical.RowType;
import java.io.OutputStream;
@@ -36,7 +36,7 @@
/** Test data and helper objects for JDBC tests. */
@SuppressWarnings("SpellCheckingInspection")
-public class JdbcTestFixture implements DerbyDatabase {
+public class JdbcTestFixture implements DerbyTestBase {
public static final JdbcTestCheckpoint CP0 = new JdbcTestCheckpoint(0, 1, 2, 3);
public static final JdbcTestCheckpoint CP1 = new JdbcTestCheckpoint(1, 4, 5, 6);
@@ -74,8 +74,6 @@
new TestEntry(1010, ("A Teaspoon of Java 1.8"), ("Kevin Jones"), null, 1010)
};
- public static final DatabaseMetadata DERBY_EBOOKSHOP_DB = DerbyDatabase.startDatabase();
-
/** TestEntry. */
public static class TestEntry implements Serializable {
public final Integer id;
@@ -187,7 +185,7 @@
public static void initSchema(DatabaseMetadata metadata) throws SQLException {
try (Connection conn = metadata.getConnection()) {
- createTable(conn, JdbcTestFixture.INPUT_TABLE);
+ createTable(conn, INPUT_TABLE);
createTable(conn, OUTPUT_TABLE);
createTable(conn, OUTPUT_TABLE_2);
createTable(conn, OUTPUT_TABLE_3);
@@ -238,8 +236,8 @@
}
}
- static void cleanupData(String url) throws Exception {
- try (Connection conn = DriverManager.getConnection(url)) {
+ static void cleanupData(DatabaseMetadata dbMetadata) throws Exception {
+ try (Connection conn = dbMetadata.getConnection()) {
executeUpdate(conn, "delete from " + INPUT_TABLE);
executeUpdate(conn, "delete from " + WORDS_TABLE);
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java
deleted file mode 100644
index 3480fa7..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogITCase.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.catalog;
-
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlDatabase;
-
-import org.testcontainers.containers.MySQLContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-/** E2E test for {@link MySqlCatalog}. */
-@Testcontainers
-public class MySqlCatalogITCase extends MySqlCatalogTestBase {
-
- @Container
- private static final MySQLContainer<?> CONTAINER = createContainer(MySqlDatabase.MYSQL_8_0);
-
- @Override
- protected String getDatabaseUrl() {
- return CONTAINER.getJdbcUrl().substring(0, CONTAINER.getJdbcUrl().lastIndexOf("/"));
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
deleted file mode 100644
index b1f04aa..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySqlCatalogTestBase.java
+++ /dev/null
@@ -1,463 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.catalog;
-
-import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Schema;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.catalog.CatalogBaseTable;
-import org.apache.flink.table.catalog.ObjectPath;
-import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
-import org.apache.flink.table.catalog.exceptions.TableNotExistException;
-import org.apache.flink.types.Row;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.CollectionUtil;
-
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.testcontainers.containers.MySQLContainer;
-import org.testcontainers.containers.output.Slf4jLogConsumer;
-import org.testcontainers.utility.DockerImageName;
-
-import java.math.BigDecimal;
-import java.sql.Date;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
-import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
-import static org.assertj.core.api.Assertions.assertThat;
-import static org.assertj.core.api.Assertions.assertThatThrownBy;
-
-/** Test base for {@link MySqlCatalog}. */
-abstract class MySqlCatalogTestBase implements JdbcITCaseBase {
-
- public static final Logger LOG = LoggerFactory.getLogger(MySqlCatalogTestBase.class);
- protected static final String TEST_CATALOG_NAME = "mysql_catalog";
- protected static final String TEST_USERNAME = "mysql";
- protected static final String TEST_PWD = "mysql";
- protected static final String TEST_DB = "test";
- protected static final String TEST_DB2 = "test2";
- protected static final String TEST_TABLE_ALL_TYPES = "t_all_types";
- protected static final String TEST_SINK_TABLE_ALL_TYPES = "t_all_types_sink";
- protected static final String TEST_TABLE_SINK_FROM_GROUPED_BY = "t_grouped_by_sink";
- protected static final String TEST_TABLE_PK = "t_pk";
- protected static final String MYSQL_INIT_SCRIPT = "mysql-scripts/catalog-init-for-test.sql";
- protected static final Map<String, String> DEFAULT_CONTAINER_ENV_MAP =
- new HashMap<String, String>() {
- {
- put("MYSQL_ROOT_HOST", "%");
- }
- };
-
- protected static final Schema TABLE_SCHEMA =
- Schema.newBuilder()
- .column("pid", DataTypes.BIGINT().notNull())
- .column("col_bigint", DataTypes.BIGINT())
- .column("col_bigint_unsigned", DataTypes.DECIMAL(20, 0))
- .column("col_binary", DataTypes.BYTES())
- .column("col_bit", DataTypes.BOOLEAN())
- .column("col_blob", DataTypes.BYTES())
- .column("col_char", DataTypes.CHAR(10))
- .column("col_date", DataTypes.DATE())
- .column("col_datetime", DataTypes.TIMESTAMP(0))
- .column("col_decimal", DataTypes.DECIMAL(10, 0))
- .column("col_decimal_unsigned", DataTypes.DECIMAL(11, 0))
- .column("col_double", DataTypes.DOUBLE())
- .column("col_double_unsigned", DataTypes.DOUBLE())
- .column("col_enum", DataTypes.CHAR(6))
- .column("col_float", DataTypes.FLOAT())
- .column("col_float_unsigned", DataTypes.FLOAT())
- .column("col_int", DataTypes.INT())
- .column("col_int_unsigned", DataTypes.BIGINT())
- .column("col_integer", DataTypes.INT())
- .column("col_integer_unsigned", DataTypes.BIGINT())
- .column("col_longblob", DataTypes.BYTES())
- .column("col_longtext", DataTypes.STRING())
- .column("col_mediumblob", DataTypes.BYTES())
- .column("col_mediumint", DataTypes.INT())
- .column("col_mediumint_unsigned", DataTypes.INT())
- .column("col_mediumtext", DataTypes.VARCHAR(5592405))
- .column("col_numeric", DataTypes.DECIMAL(10, 0))
- .column("col_numeric_unsigned", DataTypes.DECIMAL(11, 0))
- .column("col_real", DataTypes.DOUBLE())
- .column("col_real_unsigned", DataTypes.DOUBLE())
- .column("col_set", DataTypes.CHAR(18))
- .column("col_smallint", DataTypes.SMALLINT())
- .column("col_smallint_unsigned", DataTypes.INT())
- .column("col_text", DataTypes.VARCHAR(21845))
- .column("col_time", DataTypes.TIME(0))
- .column("col_timestamp", DataTypes.TIMESTAMP(0))
- .column("col_tinytext", DataTypes.VARCHAR(85))
- .column("col_tinyint", DataTypes.TINYINT())
- .column("col_tinyint_unsinged", DataTypes.SMALLINT())
- .column("col_tinyblob", DataTypes.BYTES())
- .column("col_varchar", DataTypes.VARCHAR(255))
- .column("col_datetime_p3", DataTypes.TIMESTAMP(3).notNull())
- .column("col_time_p3", DataTypes.TIME(3))
- .column("col_timestamp_p3", DataTypes.TIMESTAMP(3))
- .column("col_varbinary", DataTypes.BYTES())
- .primaryKeyNamed("PRIMARY", Collections.singletonList("pid"))
- .build();
-
- protected static final List<Row> TABLE_ROWS =
- Arrays.asList(
- Row.ofKind(
- RowKind.INSERT,
- 1L,
- -1L,
- new BigDecimal(1),
- null,
- true,
- null,
- "hello",
- Date.valueOf("2021-08-04").toLocalDate(),
- Timestamp.valueOf("2021-08-04 01:54:16").toLocalDateTime(),
- new BigDecimal(-1),
- new BigDecimal(1),
- -1.0d,
- 1.0d,
- "enum2",
- -9.1f,
- 9.1f,
- -1,
- 1L,
- -1,
- 1L,
- null,
- "col_longtext",
- null,
- -1,
- 1,
- "col_mediumtext",
- new BigDecimal(-99),
- new BigDecimal(99),
- -1.0d,
- 1.0d,
- "set_ele1",
- Short.parseShort("-1"),
- 1,
- "col_text",
- Time.valueOf("10:32:34").toLocalTime(),
- Timestamp.valueOf("2021-08-04 01:54:16").toLocalDateTime(),
- "col_tinytext",
- Byte.parseByte("-1"),
- Short.parseShort("1"),
- null,
- "col_varchar",
- Timestamp.valueOf("2021-08-04 01:54:16.463").toLocalDateTime(),
- Time.valueOf("09:33:43").toLocalTime(),
- Timestamp.valueOf("2021-08-04 01:54:16.463").toLocalDateTime(),
- null),
- Row.ofKind(
- RowKind.INSERT,
- 2L,
- -1L,
- new BigDecimal(1),
- null,
- true,
- null,
- "hello",
- Date.valueOf("2021-08-04").toLocalDate(),
- Timestamp.valueOf("2021-08-04 01:53:19").toLocalDateTime(),
- new BigDecimal(-1),
- new BigDecimal(1),
- -1.0d,
- 1.0d,
- "enum2",
- -9.1f,
- 9.1f,
- -1,
- 1L,
- -1,
- 1L,
- null,
- "col_longtext",
- null,
- -1,
- 1,
- "col_mediumtext",
- new BigDecimal(-99),
- new BigDecimal(99),
- -1.0d,
- 1.0d,
- "set_ele1,set_ele12",
- Short.parseShort("-1"),
- 1,
- "col_text",
- Time.valueOf("10:32:34").toLocalTime(),
- Timestamp.valueOf("2021-08-04 01:53:19").toLocalDateTime(),
- "col_tinytext",
- Byte.parseByte("-1"),
- Short.parseShort("1"),
- null,
- "col_varchar",
- Timestamp.valueOf("2021-08-04 01:53:19.098").toLocalDateTime(),
- Time.valueOf("09:33:43").toLocalTime(),
- Timestamp.valueOf("2021-08-04 01:53:19.098").toLocalDateTime(),
- null));
-
- private MySqlCatalog catalog;
- private TableEnvironment tEnv;
-
- protected static MySQLContainer<?> createContainer(String dockerImage) {
- return new MySQLContainer<>(DockerImageName.parse(dockerImage))
- .withUsername("root")
- .withPassword("")
- .withEnv(DEFAULT_CONTAINER_ENV_MAP)
- .withInitScript(MYSQL_INIT_SCRIPT)
- .withLogConsumer(new Slf4jLogConsumer(LOG));
- }
-
- protected abstract String getDatabaseUrl();
-
- @BeforeEach
- void setup() {
- catalog =
- new MySqlCatalog(
- Thread.currentThread().getContextClassLoader(),
- TEST_CATALOG_NAME,
- TEST_DB,
- TEST_USERNAME,
- TEST_PWD,
- getDatabaseUrl());
-
- this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
- tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
-
- // Use mysql catalog.
- tEnv.registerCatalog(TEST_CATALOG_NAME, catalog);
- tEnv.useCatalog(TEST_CATALOG_NAME);
- }
-
- @Test
- void testGetDb_DatabaseNotExistException() {
- String databaseNotExist = "nonexistent";
- assertThatThrownBy(() -> catalog.getDatabase(databaseNotExist))
- .satisfies(
- anyCauseMatches(
- DatabaseNotExistException.class,
- String.format(
- "Database %s does not exist in Catalog",
- databaseNotExist)));
- }
-
- @Test
- void testListDatabases() {
- List<String> actual = catalog.listDatabases();
- assertThat(actual).containsExactly(TEST_DB, TEST_DB2);
- }
-
- @Test
- void testDbExists() {
- String databaseNotExist = "nonexistent";
- assertThat(catalog.databaseExists(databaseNotExist)).isFalse();
- assertThat(catalog.databaseExists(TEST_DB)).isTrue();
- }
-
- // ------ tables ------
-
- @Test
- void testListTables() throws DatabaseNotExistException {
- List<String> actual = catalog.listTables(TEST_DB);
- assertThat(actual)
- .isEqualTo(
- Arrays.asList(
- TEST_TABLE_ALL_TYPES,
- TEST_SINK_TABLE_ALL_TYPES,
- TEST_TABLE_SINK_FROM_GROUPED_BY,
- TEST_TABLE_PK));
- }
-
- @Test
- void testListTables_DatabaseNotExistException() {
- String anyDatabase = "anyDatabase";
- assertThatThrownBy(() -> catalog.listTables(anyDatabase))
- .satisfies(
- anyCauseMatches(
- DatabaseNotExistException.class,
- String.format(
- "Database %s does not exist in Catalog", anyDatabase)));
- }
-
- @Test
- void testTableExists() {
- String tableNotExist = "nonexist";
- assertThat(catalog.tableExists(new ObjectPath(TEST_DB, tableNotExist))).isFalse();
- assertThat(catalog.tableExists(new ObjectPath(TEST_DB, TEST_TABLE_ALL_TYPES))).isTrue();
- }
-
- @Test
- void testGetTables_TableNotExistException() {
- String anyTableNotExist = "anyTable";
- assertThatThrownBy(() -> catalog.getTable(new ObjectPath(TEST_DB, anyTableNotExist)))
- .satisfies(
- anyCauseMatches(
- TableNotExistException.class,
- String.format(
- "Table (or view) %s.%s does not exist in Catalog",
- TEST_DB, anyTableNotExist)));
- }
-
- @Test
- void testGetTables_TableNotExistException_NoDb() {
- String databaseNotExist = "nonexistdb";
- String tableNotExist = "anyTable";
- assertThatThrownBy(() -> catalog.getTable(new ObjectPath(databaseNotExist, tableNotExist)))
- .satisfies(
- anyCauseMatches(
- TableNotExistException.class,
- String.format(
- "Table (or view) %s.%s does not exist in Catalog",
- databaseNotExist, tableNotExist)));
- }
-
- @Test
- void testGetTable() throws TableNotExistException {
- CatalogBaseTable table = catalog.getTable(new ObjectPath(TEST_DB, TEST_TABLE_ALL_TYPES));
- assertThat(table.getUnresolvedSchema()).isEqualTo(TABLE_SCHEMA);
- }
-
- @Test
- void testGetTablePrimaryKey() throws TableNotExistException {
- // test the PK of test.t_user
- Schema tableSchemaTestPK1 =
- Schema.newBuilder()
- .column("uid", DataTypes.BIGINT().notNull())
- .column("col_bigint", DataTypes.BIGINT())
- .primaryKeyNamed("PRIMARY", Collections.singletonList("uid"))
- .build();
- CatalogBaseTable tablePK1 = catalog.getTable(new ObjectPath(TEST_DB, TEST_TABLE_PK));
- assertThat(tableSchemaTestPK1.getPrimaryKey())
- .contains(tablePK1.getUnresolvedSchema().getPrimaryKey().get());
-
- // test the PK of test2.t_user
- Schema tableSchemaTestPK2 =
- Schema.newBuilder()
- .column("pid", DataTypes.INT().notNull())
- .column("col_varchar", DataTypes.VARCHAR(255))
- .primaryKeyNamed("PRIMARY", Collections.singletonList("pid"))
- .build();
- CatalogBaseTable tablePK2 = catalog.getTable(new ObjectPath(TEST_DB2, TEST_TABLE_PK));
- assertThat(tableSchemaTestPK2.getPrimaryKey())
- .contains(tablePK2.getUnresolvedSchema().getPrimaryKey().get());
- }
-
- // ------ test select query. ------
-
- @Test
- void testSelectField() {
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(String.format("select pid from %s", TEST_TABLE_ALL_TYPES))
- .execute()
- .collect());
- assertThat(results)
- .isEqualTo(
- Arrays.asList(
- Row.ofKind(RowKind.INSERT, 1L), Row.ofKind(RowKind.INSERT, 2L)));
- }
-
- @Test
- void testWithoutCatalogDB() {
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(String.format("select * from %s", TEST_TABLE_ALL_TYPES))
- .execute()
- .collect());
-
- assertThat(results).isEqualTo(TABLE_ROWS);
- }
-
- @Test
- void testWithoutCatalog() {
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(
- String.format(
- "select * from `%s`.`%s`",
- TEST_DB, TEST_TABLE_ALL_TYPES))
- .execute()
- .collect());
- assertThat(results).isEqualTo(TABLE_ROWS);
- }
-
- @Test
- void testFullPath() {
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(
- String.format(
- "select * from %s.%s.`%s`",
- TEST_CATALOG_NAME,
- catalog.getDefaultDatabase(),
- TEST_TABLE_ALL_TYPES))
- .execute()
- .collect());
- assertThat(results).isEqualTo(TABLE_ROWS);
- }
-
- @Test
- void testSelectToInsert() throws Exception {
-
- String sql =
- String.format(
- "insert into `%s` select * from `%s`",
- TEST_SINK_TABLE_ALL_TYPES, TEST_TABLE_ALL_TYPES);
- tEnv.executeSql(sql).await();
-
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(String.format("select * from %s", TEST_SINK_TABLE_ALL_TYPES))
- .execute()
- .collect());
- assertThat(results).isEqualTo(TABLE_ROWS);
- }
-
- @Test
- void testGroupByInsert() throws Exception {
- // Changes primary key for the next record.
- tEnv.executeSql(
- String.format(
- "insert into `%s` select max(`pid`) `pid`, `col_bigint` from `%s` "
- + "group by `col_bigint` ",
- TEST_TABLE_SINK_FROM_GROUPED_BY, TEST_TABLE_ALL_TYPES))
- .await();
-
- List<Row> results =
- CollectionUtil.iteratorToList(
- tEnv.sqlQuery(
- String.format(
- "select * from `%s`",
- TEST_TABLE_SINK_FROM_GROUPED_BY))
- .execute()
- .collect());
- assertThat(results)
- .isEqualTo(Collections.singletonList(Row.ofKind(RowKind.INSERT, 2L, -1L)));
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyDatabase.java
deleted file mode 100644
index 2db1ebe..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyDatabase.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.apache.flink.connector.jdbc.databases.derby;
-
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import java.io.OutputStream;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-
-/** Derby database for testing. */
-public interface DerbyDatabase extends DatabaseTest {
-
- @SuppressWarnings("unused") // used in string constant in prepareDatabase
- OutputStream DEV_NULL =
- new OutputStream() {
- @Override
- public void write(int b) {}
- };
-
- DatabaseMetadata METADATA = startDatabase();
-
- @Override
- default DatabaseMetadata getMetadata() {
- return METADATA;
- }
-
- static DatabaseMetadata startDatabase() {
- DatabaseMetadata metadata = new DerbyMetadata("test");
- try {
- System.setProperty(
- "derby.stream.error.field",
- DerbyDatabase.class.getCanonicalName() + ".DEV_NULL");
- Class.forName(metadata.getDriverClass());
- DriverManager.getConnection(String.format("%s;create=true", metadata.getJdbcUrl()))
- .close();
- } catch (Exception e) {
- throw new FlinkRuntimeException(e);
- }
- return metadata;
- }
-
- default void stopDatabase() throws Exception {
- DatabaseMetadata metadata = getMetadata();
- try {
- DriverManager.getConnection(String.format("%s;shutdown=true", metadata.getJdbcUrl()))
- .close();
- } catch (SQLException ignored) {
- }
- }
-}
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyTestBase.java
similarity index 60%
copy from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyTestBase.java
index 3e71166..b8f32fd 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyTestBase.java
@@ -16,22 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.derby;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.derby.DerbyDatabase;
-/** Factory for {@link SqlServerDialect}. */
-@Internal
-public class SqlServerDialectFactory implements JdbcDialectFactory {
- @Override
- public boolean acceptsURL(String url) {
- return url.startsWith("jdbc:sqlserver:");
- }
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for Derby testing. */
+@ExtendWith(DerbyDatabase.class)
+public interface DerbyTestBase extends DatabaseTest {
@Override
- public JdbcDialect create() {
- return new SqlServerDialect();
+ default DatabaseMetadata getMetadata() {
+ return DerbyDatabase.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSinkITCase.java
similarity index 63%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSinkITCase.java
index 61314af..acd2ad9 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSinkITCase.java
@@ -16,17 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.derby.table;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
+import org.apache.flink.connector.jdbc.databases.derby.dialect.DerbyDialect;
+import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSinkITCase;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/** The Table Sink ITCase for {@link DerbyDialect}. */
+public class DerbyDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
+ implements DerbyTestBase {}
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
new file mode 100644
index 0000000..fa1d673
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/table/DerbyDynamicTableSourceITCase.java
@@ -0,0 +1,74 @@
+/*
+ * 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.databases.derby.table;
+
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
+import org.apache.flink.connector.jdbc.databases.derby.dialect.DerbyDialect;
+import org.apache.flink.connector.jdbc.table.JdbcDynamicTableSourceITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.List;
+
+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}. */
+public class DerbyDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase
+ implements DerbyTestBase {
+
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ field("id", DataTypes.BIGINT().notNull()),
+ field("decimal_col", DataTypes.DECIMAL(10, 4)),
+ field("timestamp6_col", dbType("TIMESTAMP"), DataTypes.TIMESTAMP(6)),
+ // other fields
+ field("real_col", dbType("REAL"), DataTypes.FLOAT()),
+ field("double_col", DataTypes.DOUBLE()),
+ field("time_col", dbType("TIME"), DataTypes.TIME()),
+ field("timestamp9_col", dbType("TIMESTAMP"), DataTypes.TIMESTAMP(9)));
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ 1.175E-37F,
+ 1.79769E308D,
+ LocalTime.parse("15:35"),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456789")),
+ 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"),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456789")));
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/xa/DerbyExactlyOnceSinkE2eTest.java
similarity index 64%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/xa/DerbyExactlyOnceSinkE2eTest.java
index 61314af..2757d0e 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/xa/DerbyExactlyOnceSinkE2eTest.java
@@ -16,17 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.derby.xa;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
+import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/**
+ * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with errors on
+ * closing connections.
+ */
+public class DerbyExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
+ implements DerbyTestBase {}
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaTestBase.java
similarity index 61%
copy from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaTestBase.java
index 6ec301a..3c588c7 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectFactory.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaTestBase.java
@@ -16,22 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.h2;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.h2.H2XaDatabase;
-/** Factory for {@link MySqlDialect}. */
-@Internal
-public class MySqlDialectFactory implements JdbcDialectFactory {
- @Override
- public boolean acceptsURL(String url) {
- return url.startsWith("jdbc:mysql:");
- }
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for H2 Xa testing. */
+@ExtendWith(H2XaDatabase.class)
+public interface H2XaTestBase extends DatabaseTest {
@Override
- public JdbcDialect create() {
- return new MySqlDialect();
+ default DatabaseMetadata getMetadata() {
+ return H2XaDatabase.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlTestBase.java
similarity index 60%
copy from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlTestBase.java
index 3e71166..73e30bb 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlTestBase.java
@@ -16,22 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.mysql;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.mysql.MySqlDatabase;
-/** Factory for {@link SqlServerDialect}. */
-@Internal
-public class SqlServerDialectFactory implements JdbcDialectFactory {
- @Override
- public boolean acceptsURL(String url) {
- return url.startsWith("jdbc:sqlserver:");
- }
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for MySql testing. */
+@ExtendWith(MySqlDatabase.class)
+public interface MySqlTestBase extends DatabaseTest {
@Override
- public JdbcDialect create() {
- return new SqlServerDialect();
+ default DatabaseMetadata getMetadata() {
+ return MySqlDatabase.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql56CatalogITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql56CatalogITCase.java
similarity index 64%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql56CatalogITCase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql56CatalogITCase.java
index fb553c6..fbfcae2 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql56CatalogITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql56CatalogITCase.java
@@ -16,23 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.mysql.catalog;
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlDatabase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.mysql.MySql56Database;
-import org.testcontainers.containers.MySQLContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
+import org.junit.jupiter.api.extension.RegisterExtension;
/** E2E test for {@link MySqlCatalog} with MySql version 5.6. */
-@Testcontainers
public class MySql56CatalogITCase extends MySqlCatalogTestBase {
- @Container
- private static final MySQLContainer<?> CONTAINER = createContainer(MySqlDatabase.MYSQL_5_6);
+ @RegisterExtension private static final MySql56Database DATABASE = new MySql56Database();
@Override
- protected String getDatabaseUrl() {
- return CONTAINER.getJdbcUrl().substring(0, CONTAINER.getJdbcUrl().lastIndexOf("/"));
+ public DatabaseMetadata getMetadata() {
+ return MySql56Database.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql57CatalogITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql57CatalogITCase.java
similarity index 64%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql57CatalogITCase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql57CatalogITCase.java
index 9d0edcf..c48bf5f 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/MySql57CatalogITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySql57CatalogITCase.java
@@ -16,23 +16,19 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.mysql.catalog;
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlDatabase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.mysql.MySql57Database;
-import org.testcontainers.containers.MySQLContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
+import org.junit.jupiter.api.extension.RegisterExtension;
/** E2E test for {@link MySqlCatalog} with MySql version 5.7. */
-@Testcontainers
public class MySql57CatalogITCase extends MySqlCatalogTestBase {
-
- @Container
- private static final MySQLContainer<?> CONTAINER = createContainer(MySqlDatabase.MYSQL_5_7);
+ @RegisterExtension private static final MySql57Database DATABASE = new MySql57Database();
@Override
- protected String getDatabaseUrl() {
- return CONTAINER.getJdbcUrl().substring(0, CONTAINER.getJdbcUrl().lastIndexOf("/"));
+ public DatabaseMetadata getMetadata() {
+ return MySql57Database.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogITCase.java
similarity index 65%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogITCase.java
index 61314af..daf1a8a 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogITCase.java
@@ -16,17 +16,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.mysql.catalog;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.mysql.MySqlTestBase;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/** E2E test for {@link MySqlCatalog}. */
+public class MySqlCatalogITCase extends MySqlCatalogTestBase implements MySqlTestBase {}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogTestBase.java
new file mode 100644
index 0000000..2bf29b3
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/catalog/MySqlCatalogTestBase.java
@@ -0,0 +1,535 @@
+/*
+ * 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.databases.mysql.catalog;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.catalog.CatalogBaseTable;
+import org.apache.flink.table.catalog.ObjectPath;
+import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
+import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.apache.flink.types.Row;
+import org.apache.flink.types.RowKind;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
+import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches;
+import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test base for {@link MySqlCatalog}. */
+public abstract class MySqlCatalogTestBase implements JdbcITCaseBase, DatabaseTest {
+
+ private static final Logger LOG = LoggerFactory.getLogger(MySqlCatalogTestBase.class);
+ private static final String TEST_CATALOG_NAME = "mysql_catalog";
+ private static final String TEST_DB = "test";
+ private static final String TEST_DB2 = "test2";
+
+ private static final TableRow TABLE_ALL_TYPES = createTableAllTypeTable("t_all_types");
+ private static final TableRow TABLE_ALL_TYPES_SINK =
+ createTableAllTypeTable("t_all_types_sink");
+ private static final TableRow TABLE_GROUPED_BY_SINK = createGroupedTable("t_grouped_by_sink");
+ private static final TableRow TABLE_PK = createGroupedTable("t_pk");
+ private static final TableRow TABLE_PK2 =
+ tableRow(
+ "t_pk",
+ pkField(
+ "pid",
+ dbType("int(11) NOT NULL AUTO_INCREMENT"),
+ DataTypes.BIGINT().notNull()),
+ field("col_varchar", dbType("varchar(255)"), DataTypes.BIGINT()));
+
+ private static final List<Row> TABLE_ALL_TYPES_ROWS =
+ Arrays.asList(
+ Row.ofKind(
+ RowKind.INSERT,
+ 1L,
+ -1L,
+ new BigDecimal(1),
+ null,
+ true,
+ null,
+ "hello",
+ Date.valueOf("2021-08-04").toLocalDate(),
+ Timestamp.valueOf("2021-08-04 01:54:16").toLocalDateTime(),
+ new BigDecimal(-1),
+ new BigDecimal(1),
+ -1.0d,
+ 1.0d,
+ "enum2",
+ -9.1f,
+ 9.1f,
+ -1,
+ 1L,
+ -1,
+ 1L,
+ null,
+ "col_longtext",
+ null,
+ -1,
+ 1,
+ "col_mediumtext",
+ new BigDecimal(-99),
+ new BigDecimal(99),
+ -1.0d,
+ 1.0d,
+ "set_ele1",
+ Short.parseShort("-1"),
+ 1,
+ "col_text",
+ Time.valueOf("10:32:34").toLocalTime(),
+ Timestamp.valueOf("2021-08-04 01:54:16").toLocalDateTime(),
+ "col_tinytext",
+ Byte.parseByte("-1"),
+ Short.parseShort("1"),
+ null,
+ "col_varchar",
+ Timestamp.valueOf("2021-08-04 01:54:16.463").toLocalDateTime(),
+ Time.valueOf("09:33:43").toLocalTime(),
+ Timestamp.valueOf("2021-08-04 01:54:16.463").toLocalDateTime(),
+ null),
+ Row.ofKind(
+ RowKind.INSERT,
+ 2L,
+ -1L,
+ new BigDecimal(1),
+ null,
+ true,
+ null,
+ "hello",
+ Date.valueOf("2021-08-04").toLocalDate(),
+ Timestamp.valueOf("2021-08-04 01:53:19").toLocalDateTime(),
+ new BigDecimal(-1),
+ new BigDecimal(1),
+ -1.0d,
+ 1.0d,
+ "enum2",
+ -9.1f,
+ 9.1f,
+ -1,
+ 1L,
+ -1,
+ 1L,
+ null,
+ "col_longtext",
+ null,
+ -1,
+ 1,
+ "col_mediumtext",
+ new BigDecimal(-99),
+ new BigDecimal(99),
+ -1.0d,
+ 1.0d,
+ "set_ele1,set_ele12",
+ Short.parseShort("-1"),
+ 1,
+ "col_text",
+ Time.valueOf("10:32:34").toLocalTime(),
+ Timestamp.valueOf("2021-08-04 01:53:19").toLocalDateTime(),
+ "col_tinytext",
+ Byte.parseByte("-1"),
+ Short.parseShort("1"),
+ null,
+ "col_varchar",
+ Timestamp.valueOf("2021-08-04 01:53:19.098").toLocalDateTime(),
+ Time.valueOf("09:33:43").toLocalTime(),
+ Timestamp.valueOf("2021-08-04 01:53:19.098").toLocalDateTime(),
+ null));
+ private MySqlCatalog catalog;
+ private TableEnvironment tEnv;
+
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Arrays.asList(
+ TABLE_ALL_TYPES, TABLE_ALL_TYPES_SINK, TABLE_GROUPED_BY_SINK, TABLE_PK);
+ }
+
+ private static TableRow createTableAllTypeTable(String tableName) {
+ return tableRow(
+ tableName,
+ pkField(
+ "pid",
+ dbType("bigint(20) NOT NULL AUTO_INCREMENT"),
+ DataTypes.BIGINT().notNull()),
+ field("col_bigint", dbType("bigint(20)"), DataTypes.BIGINT()),
+ field(
+ "col_bigint_unsigned",
+ dbType("bigint(20) unsigned"),
+ DataTypes.DECIMAL(20, 0)),
+ field("col_binary", dbType("binary(100)"), DataTypes.BYTES()),
+ field("col_bit", dbType("bit(1)"), DataTypes.BOOLEAN()),
+ field("col_blob", dbType("blob"), DataTypes.BYTES()),
+ field("col_char", dbType("char(10)"), DataTypes.CHAR(10)),
+ field("col_date", dbType("date"), DataTypes.DATE()),
+ field(
+ "col_datetime",
+ dbType("datetime DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP"),
+ DataTypes.TIMESTAMP(0)),
+ field("col_decimal", dbType("decimal(10,0)"), DataTypes.DECIMAL(10, 0)),
+ field(
+ "col_decimal_unsigned",
+ dbType("decimal(10,0) unsigned"),
+ DataTypes.DECIMAL(11, 0)),
+ field("col_double", dbType("double"), DataTypes.DOUBLE()),
+ field("col_double_unsigned", dbType("double unsigned"), DataTypes.DOUBLE()),
+ field("col_enum", dbType("enum('enum1','enum2','enum11')"), DataTypes.CHAR(6)),
+ field("col_float", dbType("float"), DataTypes.FLOAT()),
+ field("col_float_unsigned", dbType("float unsigned"), DataTypes.FLOAT()),
+ field("col_int", dbType("int(11)"), DataTypes.INT()),
+ field("col_int_unsigned", dbType("int(10) unsigned"), DataTypes.BIGINT()),
+ field("col_integer", dbType("int(11)"), DataTypes.INT()),
+ field("col_integer_unsigned", dbType("int(10) unsigned"), DataTypes.BIGINT()),
+ field("col_longblob", dbType("longblob"), DataTypes.BYTES()),
+ field(
+ "col_longtext",
+ dbType("longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin"),
+ DataTypes.STRING()),
+ field("col_mediumblob", dbType("mediumblob"), DataTypes.BYTES()),
+ field("col_mediumint", dbType("mediumint(9)"), DataTypes.INT()),
+ field("col_mediumint_unsigned", dbType("mediumint(8) unsigned"), DataTypes.INT()),
+ field("col_mediumtext", dbType("mediumtext"), DataTypes.VARCHAR(5592405)),
+ field("col_numeric", dbType("decimal(10,0)"), DataTypes.DECIMAL(10, 0)),
+ field(
+ "col_numeric_unsigned",
+ dbType("decimal(10,0) unsigned"),
+ DataTypes.DECIMAL(11, 0)),
+ field("col_real", dbType("double"), DataTypes.DOUBLE()),
+ field("col_real_unsigned", dbType("double unsigned"), DataTypes.DOUBLE()),
+ field("col_set", dbType("set('set_ele1','set_ele12')"), DataTypes.CHAR(18)),
+ field("col_smallint", dbType("smallint(6)"), DataTypes.SMALLINT()),
+ field("col_smallint_unsigned", dbType("smallint(5) unsigned"), DataTypes.INT()),
+ field("col_text", dbType("text"), DataTypes.VARCHAR(21845)),
+ field("col_time", dbType("time"), DataTypes.TIME(0)),
+ field(
+ "col_timestamp",
+ dbType(
+ "timestamp NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP"),
+ DataTypes.TIMESTAMP(0)),
+ field("col_tinytext", dbType("tinytext"), DataTypes.VARCHAR(85)),
+ field("col_tinyint", dbType("tinyint"), DataTypes.TINYINT()),
+ field(
+ "col_tinyint_unsinged",
+ dbType("tinyint(255) unsigned"),
+ DataTypes.SMALLINT()),
+ field("col_tinyblob", dbType("tinyblob"), DataTypes.BYTES()),
+ field("col_varchar", dbType("varchar(255)"), DataTypes.VARCHAR(255)),
+ field(
+ "col_datetime_p3",
+ dbType(
+ "datetime(3) NOT NULL DEFAULT CURRENT_TIMESTAMP(3) ON UPDATE CURRENT_TIMESTAMP(3)"),
+ DataTypes.TIMESTAMP(3).notNull()),
+ field("col_time_p3", dbType("time(3)"), DataTypes.TIME(3)),
+ field(
+ "col_timestamp_p3",
+ dbType(
+ "timestamp(3) NULL DEFAULT CURRENT_TIMESTAMP(3) ON UPDATE CURRENT_TIMESTAMP(3)"),
+ DataTypes.TIMESTAMP(3)),
+ field("col_varbinary", dbType("varbinary(255)"), DataTypes.BYTES()));
+ }
+
+ private static TableRow createGroupedTable(String tableName) {
+ return tableRow(
+ tableName,
+ pkField(
+ "pid",
+ dbType("bigint(20) NOT NULL AUTO_INCREMENT"),
+ DataTypes.BIGINT().notNull()),
+ field("col_bigint", dbType("bigint(20)"), DataTypes.BIGINT()));
+ }
+
+ @BeforeEach
+ void setup() {
+ try (Connection conn = getMetadata().getConnection();
+ Statement st = conn.createStatement()) {
+ TABLE_ALL_TYPES.insertIntoTableValues(
+ conn,
+ "1, -1, 1, null, b'1', null, 'hello', '2021-08-04', '2021-08-04 01:54:16', -1, 1, -1, 1, 'enum2', -9.1, 9.1, -1, 1, -1, 1, null, 'col_longtext', null, -1, 1, 'col_mediumtext', -99, 99, -1, 1, 'set_ele1', -1, 1, 'col_text', '10:32:34', '2021-08-04 01:54:16', 'col_tinytext', -1, 1, null, 'col_varchar', '2021-08-04 01:54:16.463', '09:33:43.000', '2021-08-04 01:54:16.463', null",
+ "2, -1, 1, null, b'1', null, 'hello', '2021-08-04', '2021-08-04 01:53:19', -1, 1, -1, 1, 'enum2', -9.1, 9.1, -1, 1, -1, 1, null, 'col_longtext', null, -1, 1, 'col_mediumtext', -99, 99, -1, 1, 'set_ele1,set_ele12', -1, 1, 'col_text', '10:32:34', '2021-08-04 01:53:19', 'col_tinytext', -1, 1, null, 'col_varchar', '2021-08-04 01:53:19.098', '09:33:43.000', '2021-08-04 01:53:19.098', null");
+
+ st.execute(String.format("CREATE DATABASE `%s` CHARSET=utf8", TEST_DB2));
+ st.execute(String.format("use `%s`", TEST_DB2));
+ st.execute(TABLE_PK2.getCreateQuery());
+ } catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+
+ catalog =
+ new MySqlCatalog(
+ Thread.currentThread().getContextClassLoader(),
+ TEST_CATALOG_NAME,
+ TEST_DB,
+ getMetadata().getUsername(),
+ getMetadata().getPassword(),
+ getMetadata()
+ .getJdbcUrl()
+ .substring(0, getMetadata().getJdbcUrl().lastIndexOf("/")));
+
+ this.tEnv = TableEnvironment.create(EnvironmentSettings.inStreamingMode());
+ tEnv.getConfig().set(TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM, 1);
+
+ // Use mysql catalog.
+ tEnv.registerCatalog(TEST_CATALOG_NAME, catalog);
+ tEnv.useCatalog(TEST_CATALOG_NAME);
+ }
+
+ @AfterEach
+ void afterEach() {
+ try (Connection conn = getMetadata().getConnection();
+ Statement st = conn.createStatement()) {
+ st.execute(String.format("DROP DATABASE IF EXISTS `%s`", TEST_DB2));
+ } catch (SQLException e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ @Test
+ void testGetDb_DatabaseNotExistException() {
+ String databaseNotExist = "nonexistent";
+ assertThatThrownBy(() -> catalog.getDatabase(databaseNotExist))
+ .satisfies(
+ anyCauseMatches(
+ DatabaseNotExistException.class,
+ String.format(
+ "Database %s does not exist in Catalog",
+ databaseNotExist)));
+ }
+
+ @Test
+ void testListDatabases() {
+ List<String> actual = catalog.listDatabases();
+ assertThat(actual).containsExactly(TEST_DB, TEST_DB2);
+ }
+
+ @Test
+ void testDbExists() {
+ String databaseNotExist = "nonexistent";
+ assertThat(catalog.databaseExists(databaseNotExist)).isFalse();
+ assertThat(catalog.databaseExists(TEST_DB)).isTrue();
+ }
+
+ // ------ tables ------
+
+ @Test
+ void testListTables() throws DatabaseNotExistException {
+ List<String> actual = catalog.listTables(TEST_DB);
+ assertThat(actual)
+ .isEqualTo(
+ getManagedTables().stream()
+ .map(TableManaged::getTableName)
+ .collect(Collectors.toList()));
+ }
+
+ @Test
+ void testListTables_DatabaseNotExistException() {
+ String anyDatabase = "anyDatabase";
+ assertThatThrownBy(() -> catalog.listTables(anyDatabase))
+ .satisfies(
+ anyCauseMatches(
+ DatabaseNotExistException.class,
+ String.format(
+ "Database %s does not exist in Catalog", anyDatabase)));
+ }
+
+ @Test
+ void testTableExists() {
+ String tableNotExist = "nonexist";
+ assertThat(catalog.tableExists(new ObjectPath(TEST_DB, tableNotExist))).isFalse();
+ assertThat(catalog.tableExists(new ObjectPath(TEST_DB, TABLE_ALL_TYPES.getTableName())))
+ .isTrue();
+ }
+
+ @Test
+ void testGetTables_TableNotExistException() {
+ String anyTableNotExist = "anyTable";
+ assertThatThrownBy(() -> catalog.getTable(new ObjectPath(TEST_DB, anyTableNotExist)))
+ .satisfies(
+ anyCauseMatches(
+ TableNotExistException.class,
+ String.format(
+ "Table (or view) %s.%s does not exist in Catalog",
+ TEST_DB, anyTableNotExist)));
+ }
+
+ @Test
+ void testGetTables_TableNotExistException_NoDb() {
+ String databaseNotExist = "nonexistdb";
+ String tableNotExist = "anyTable";
+ assertThatThrownBy(() -> catalog.getTable(new ObjectPath(databaseNotExist, tableNotExist)))
+ .satisfies(
+ anyCauseMatches(
+ TableNotExistException.class,
+ String.format(
+ "Table (or view) %s.%s does not exist in Catalog",
+ databaseNotExist, tableNotExist)));
+ }
+
+ @Test
+ void testGetTable() throws TableNotExistException {
+ CatalogBaseTable table =
+ catalog.getTable(new ObjectPath(TEST_DB, TABLE_ALL_TYPES.getTableName()));
+ assertThat(table.getUnresolvedSchema()).isEqualTo(TABLE_ALL_TYPES.getTableSchema());
+ }
+
+ @Test
+ void testGetTablePrimaryKey() throws TableNotExistException {
+ // test the PK of test.t_user
+ Schema tableSchemaTestPK1 = TABLE_PK.getTableSchema();
+ CatalogBaseTable tablePK1 =
+ catalog.getTable(new ObjectPath(TEST_DB, TABLE_PK.getTableName()));
+ assertThat(tableSchemaTestPK1.getPrimaryKey())
+ .isEqualTo(tablePK1.getUnresolvedSchema().getPrimaryKey());
+
+ // test the PK of TEST_DB2.t_user
+ Schema tableSchemaTestPK2 = TABLE_PK2.getTableSchema();
+ CatalogBaseTable tablePK2 =
+ catalog.getTable(new ObjectPath(TEST_DB2, TABLE_PK2.getTableName()));
+ assertThat(tableSchemaTestPK2.getPrimaryKey())
+ .isEqualTo(tablePK2.getUnresolvedSchema().getPrimaryKey());
+ }
+
+ // ------ test select query. ------
+
+ @Test
+ void testSelectField() {
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select pid from %s",
+ TABLE_ALL_TYPES.getTableName()))
+ .execute()
+ .collect());
+ assertThat(results)
+ .isEqualTo(
+ Arrays.asList(
+ Row.ofKind(RowKind.INSERT, 1L), Row.ofKind(RowKind.INSERT, 2L)));
+ }
+
+ @Test
+ void testWithoutCatalogDB() {
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select * from %s", TABLE_ALL_TYPES.getTableName()))
+ .execute()
+ .collect());
+
+ assertThat(results).isEqualTo(TABLE_ALL_TYPES_ROWS);
+ }
+
+ @Test
+ void testWithoutCatalog() {
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select * from `%s`.`%s`",
+ TEST_DB, TABLE_ALL_TYPES.getTableName()))
+ .execute()
+ .collect());
+ assertThat(results).isEqualTo(TABLE_ALL_TYPES_ROWS);
+ }
+
+ @Test
+ void testFullPath() {
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select * from %s.%s.`%s`",
+ TEST_CATALOG_NAME,
+ catalog.getDefaultDatabase(),
+ TABLE_ALL_TYPES.getTableName()))
+ .execute()
+ .collect());
+ assertThat(results).isEqualTo(TABLE_ALL_TYPES_ROWS);
+ }
+
+ @Test
+ void testSelectToInsert() throws Exception {
+
+ String sql =
+ String.format(
+ "insert into `%s` select * from `%s`",
+ TABLE_ALL_TYPES_SINK.getTableName(), TABLE_ALL_TYPES.getTableName());
+ tEnv.executeSql(sql).await();
+
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select * from %s",
+ TABLE_ALL_TYPES_SINK.getTableName()))
+ .execute()
+ .collect());
+ assertThat(results).isEqualTo(TABLE_ALL_TYPES_ROWS);
+ }
+
+ @Test
+ void testGroupByInsert() throws Exception {
+ // Changes primary key for the next record.
+ tEnv.executeSql(
+ String.format(
+ "insert into `%s` select max(`pid`) `pid`, `col_bigint` from `%s` "
+ + "group by `col_bigint` ",
+ TABLE_GROUPED_BY_SINK.getTableName(),
+ TABLE_ALL_TYPES.getTableName()))
+ .await();
+
+ List<Row> results =
+ CollectionUtil.iteratorToList(
+ tEnv.sqlQuery(
+ String.format(
+ "select * from `%s`",
+ TABLE_GROUPED_BY_SINK.getTableName()))
+ .execute()
+ .collect());
+ assertThat(results)
+ .isEqualTo(Collections.singletonList(Row.ofKind(RowKind.INSERT, 2L, -1L)));
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTest.java
similarity index 96%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTest.java
index 614020f..a91d9b4 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.mysql.dialect;
import org.junit.jupiter.api.Test;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTypeTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTypeTest.java
similarity index 97%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTypeTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTypeTest.java
index 816d0b4..f636c36 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlDialectTypeTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/dialect/MySqlDialectTypeTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.mysql;
+package org.apache.flink.connector.jdbc.databases.mysql.dialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeTest;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSinkITCase.java
new file mode 100644
index 0000000..2efe39d
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSinkITCase.java
@@ -0,0 +1,54 @@
+/*
+ * 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.databases.mysql.table;
+
+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.JdbcDynamicTableSinkITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
+
+/** The Table Sink ITCase for {@link MySqlDialect}. */
+public class MySqlDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
+ implements MySqlTestBase {
+
+ @Override
+ protected TableRow createUpsertOutputTable() {
+ return tableRow(
+ "dynamicSinkForUpsert",
+ pkField("cnt", DataTypes.BIGINT().notNull()),
+ field("lencnt", DataTypes.BIGINT().notNull()),
+ pkField("cTag", DataTypes.INT().notNull()),
+ field("ts", dbType("DATETIME(3)"), DataTypes.TIMESTAMP()));
+ }
+
+ @Override
+ protected TableRow createAppendOutputTable() {
+ return tableRow(
+ "dynamicSinkForAppend",
+ field("id", DataTypes.INT().notNull()),
+ field("num", DataTypes.BIGINT().notNull()),
+ field("ts", dbType("DATETIME(3)"), DataTypes.TIMESTAMP()));
+ }
+}
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
new file mode 100644
index 0000000..967c7e8
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/MySqlDynamicTableSourceITCase.java
@@ -0,0 +1,75 @@
+/*
+ * 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.databases.mysql.table;
+
+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.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.List;
+
+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 MySqlDialect}. */
+public class MySqlDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase
+ implements MySqlTestBase {
+
+ @Override
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ 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.DOUBLE()),
+ field("double_col", DataTypes.DOUBLE()),
+ field("time_col", dbType("TIME"), DataTypes.TIME()),
+ field("timestamp9_col", DataTypes.TIMESTAMP(6)));
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ 1.175E-37D,
+ 1.79769E308D,
+ LocalTime.parse("15:35"),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456")),
+ Row.of(
+ 2L,
+ BigDecimal.valueOf(101.1234),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456"),
+ -1.175E-37D,
+ -1.79769E308,
+ LocalTime.parse("15:36:01"),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456")));
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/UnsignedTypeConversionITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/UnsignedTypeConversionITCase.java
new file mode 100644
index 0000000..6364cd7
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/table/UnsignedTypeConversionITCase.java
@@ -0,0 +1,128 @@
+/*
+ * 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.databases.mysql.table;
+
+import org.apache.flink.connector.jdbc.databases.mysql.MySqlTestBase;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Table;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.jupiter.api.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.util.Collections;
+import java.util.List;
+
+import static java.lang.String.format;
+import static java.lang.String.join;
+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;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test unsigned type conversion between Flink and JDBC driver mysql, the test underlying use MySQL
+ * to mock a DB.
+ */
+class UnsignedTypeConversionITCase extends AbstractTestBase implements MySqlTestBase {
+
+ private static final Logger LOGGER =
+ LoggerFactory.getLogger(UnsignedTypeConversionITCase.class);
+ private static final String TABLE_SOURCE = "jdbc_source";
+ private static final String TABLE_SINK = "jdbc_sink";
+ private static final String TABLE_DATA = "data";
+ private static final TableRow TABLE =
+ tableRow(
+ "unsigned_test",
+ field("tiny_c", dbType("TINYINT"), DataTypes.TINYINT().notNull()),
+ field("tiny_un_c", dbType("TINYINT UNSIGNED"), DataTypes.SMALLINT().notNull()),
+ field("small_c", dbType("SMALLINT"), DataTypes.SMALLINT().notNull()),
+ field("small_un_c", dbType("SMALLINT UNSIGNED"), DataTypes.INT().notNull()),
+ field("int_c", dbType("INTEGER"), DataTypes.INT().notNull()),
+ field("int_un_c", dbType("INTEGER UNSIGNED"), DataTypes.BIGINT().notNull()),
+ field("big_c", dbType("BIGINT"), DataTypes.BIGINT().notNull()),
+ field(
+ "big_un_c",
+ dbType("BIGINT UNSIGNED"),
+ DataTypes.DECIMAL(20, 0).notNull()));
+
+ public List<TableManaged> getManagedTables() {
+ return Collections.singletonList(TABLE);
+ }
+
+ private static final Row ROW =
+ Row.of(
+ (byte) 127,
+ (short) 255,
+ (short) 32767,
+ 65535,
+ 2147483647,
+ 4294967295L,
+ 9223372036854775807L,
+ new BigDecimal("18446744073709551615"));
+
+ @Test
+ void testUnsignedType() throws Exception {
+ try (Connection con = getMetadata().getConnection()) {
+ StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment();
+ TableEnvironment tableEnv = StreamTableEnvironment.create(sEnv);
+ createFlinkTable(tableEnv);
+ prepareData(tableEnv);
+
+ // write data to db
+ String columns = join(",", TABLE.getTableFields());
+ tableEnv.executeSql(
+ format(
+ "insert into %s select %s from %s",
+ TABLE_SINK, columns, TABLE_DATA))
+ .await();
+
+ // read data from db using jdbc connection and compare
+ List<Row> selectAll = TABLE.selectAllTable(con);
+ assertThat(selectAll).containsOnly(ROW);
+
+ // read data from db using flink and compare
+ String sql = format("select %s from %s", columns, TABLE_SOURCE);
+ CloseableIterator<Row> collected = tableEnv.executeSql(sql).collect();
+ List<Row> result = CollectionUtil.iteratorToList(collected);
+ assertThat(result).containsOnly(ROW);
+ }
+ }
+
+ private void createFlinkTable(TableEnvironment tableEnv) {
+ tableEnv.executeSql(TABLE.getCreateQueryForFlink(getMetadata(), TABLE_SOURCE));
+ tableEnv.executeSql(TABLE.getCreateQueryForFlink(getMetadata(), TABLE_SINK));
+ }
+
+ private void prepareData(TableEnvironment tableEnv) {
+ Table dataTable = tableEnv.fromValues(DataTypes.ROW(TABLE.getTableDataFields()), ROW);
+ tableEnv.createTemporaryView(TABLE_DATA, dataTable);
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/xa/MySqlExactlyOnceSinkE2eTest.java
similarity index 64%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/xa/MySqlExactlyOnceSinkE2eTest.java
index 61314af..01a15f1 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/xa/MySqlExactlyOnceSinkE2eTest.java
@@ -16,17 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.mysql.xa;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.mysql.MySqlTestBase;
+import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/**
+ * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with errors on
+ * closing connections.
+ */
+public class MySqlExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
+ implements MySqlTestBase {}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleDatabase.java
deleted file mode 100644
index 3dd4554..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleDatabase.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.databases.oracle;
-
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
-
-import org.testcontainers.containers.OracleContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-/** A Oracle database for testing. */
-@Testcontainers
-public interface OracleDatabase extends DatabaseTest, OracleImages {
-
- @Container
- OracleContainer CONTAINER =
- new OracleContainer(ORACLE_21)
- .withStartupTimeoutSeconds(240)
- .withConnectTimeoutSeconds(120);
-
- @Override
- default DatabaseMetadata getMetadata() {
- return new OracleMetadata(CONTAINER);
- }
-}
diff --git a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleTestBase.java
similarity index 60%
copy from flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleTestBase.java
index 3e71166..db773ed 100644
--- a/flink-connector-jdbc/src/main/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerDialectFactory.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleTestBase.java
@@ -16,22 +16,20 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.oracle;
-import org.apache.flink.annotation.Internal;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
-import org.apache.flink.connector.jdbc.dialect.JdbcDialectFactory;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.oracle.OracleDatabase;
-/** Factory for {@link SqlServerDialect}. */
-@Internal
-public class SqlServerDialectFactory implements JdbcDialectFactory {
- @Override
- public boolean acceptsURL(String url) {
- return url.startsWith("jdbc:sqlserver:");
- }
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for Oracle testing. */
+@ExtendWith(OracleDatabase.class)
+public interface OracleTestBase extends DatabaseTest {
@Override
- public JdbcDialect create() {
- return new SqlServerDialect();
+ default DatabaseMetadata getMetadata() {
+ return OracleDatabase.getMetadata();
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleXaDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleXaDatabase.java
deleted file mode 100644
index 4886dc5..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleXaDatabase.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.databases.oracle;
-
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
-
-import org.testcontainers.containers.OracleContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-/** A Oracle database for testing. */
-@Testcontainers
-public interface OracleXaDatabase extends DatabaseTest, OracleImages {
-
- @Container
- OracleContainer CONTAINER =
- new OracleContainer(ORACLE_21)
- .withStartupTimeoutSeconds(240)
- .withConnectTimeoutSeconds(120)
- .usingSid();
-
- @Override
- default DatabaseMetadata getMetadata() {
- return new OracleMetadata(CONTAINER, true);
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectTypeTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectTypeTest.java
similarity index 96%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectTypeTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectTypeTest.java
index 3354212..93ba4d4 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleDialectTypeTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OracleDialectTypeTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.oracle;
+package org.apache.flink.connector.jdbc.databases.oracle.dialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeTest;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OraclePreparedStatementTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OraclePreparedStatementTest.java
similarity index 98%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OraclePreparedStatementTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OraclePreparedStatementTest.java
index 79dee14..f69f6a3 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OraclePreparedStatementTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/dialect/OraclePreparedStatementTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.oracle;
+package org.apache.flink.connector.jdbc.databases.oracle.dialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectLoader;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSinkITCase.java
new file mode 100644
index 0000000..2bb3cf1
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSinkITCase.java
@@ -0,0 +1,101 @@
+/*
+ * 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.databases.oracle.table;
+
+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.JdbcDynamicTableSinkITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.util.Arrays;
+import java.util.List;
+
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
+
+/** The Table Sink ITCase for {@link OracleDialect}. */
+public class OracleDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
+ implements OracleTestBase {
+
+ @Override
+ protected TableRow createUpsertOutputTable() {
+ return tableRow(
+ "dynamicSinkForUpsert",
+ pkField("cnt", dbType("NUMBER"), DataTypes.BIGINT().notNull()),
+ field("lencnt", dbType("NUMBER"), DataTypes.BIGINT().notNull()),
+ pkField("cTag", DataTypes.INT().notNull()),
+ field("ts", dbType("TIMESTAMP"), DataTypes.TIMESTAMP()));
+ }
+
+ @Override
+ protected TableRow createAppendOutputTable() {
+ return tableRow(
+ "dynamicSinkForAppend",
+ field("id", DataTypes.INT().notNull()),
+ field("num", dbType("NUMBER"), DataTypes.BIGINT().notNull()),
+ field("ts", dbType("TIMESTAMP"), DataTypes.TIMESTAMP()));
+ }
+
+ @Override
+ protected TableRow createBatchOutputTable() {
+ return tableRow(
+ "dynamicSinkForBatch",
+ field("NAME", DataTypes.VARCHAR(20).notNull()),
+ field("SCORE", dbType("NUMBER"), DataTypes.BIGINT().notNull()));
+ }
+
+ @Override
+ protected TableRow createRealOutputTable() {
+ return tableRow("REAL_TABLE", field("real_data", dbType("REAL"), DataTypes.FLOAT()));
+ }
+
+ @Override
+ protected TableRow createCheckpointOutputTable() {
+ return tableRow(
+ "checkpointTable", field("id", dbType("NUMBER"), DataTypes.BIGINT().notNull()));
+ }
+
+ @Override
+ protected List<Row> testUserData() {
+ return Arrays.asList(
+ Row.of(
+ "user1",
+ "Tom",
+ "tom123@gmail.com",
+ new BigDecimal("8.1"),
+ new BigDecimal("16.2")),
+ Row.of(
+ "user3",
+ "Bailey",
+ "bailey@qq.com",
+ new BigDecimal("9.99"),
+ new BigDecimal("19.98")),
+ Row.of(
+ "user4",
+ "Tina",
+ "tina@gmail.com",
+ new BigDecimal("11.3"),
+ new BigDecimal("22.6")));
+ }
+}
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
new file mode 100644
index 0000000..9e758ee
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleDynamicTableSourceITCase.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.jdbc.databases.oracle.table;
+
+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.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.util.Arrays;
+import java.util.List;
+
+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 OracleDialect}. */
+public class OracleDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase
+ implements OracleTestBase {
+
+ @Override
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ field("id", dbType("INTEGER"), DataTypes.BIGINT().notNull()),
+ field("decimal_col", DataTypes.DECIMAL(10, 4)),
+ field("timestamp6_col", dbType("TIMESTAMP"), DataTypes.TIMESTAMP(6)),
+ // other fields
+ field("float_col", dbType("FLOAT"), DataTypes.FLOAT()),
+ field("double_col", dbType("DOUBLE PRECISION"), DataTypes.DOUBLE()),
+ field("binary_float_col", dbType("BINARY_FLOAT"), DataTypes.FLOAT()),
+ 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("date_col", dbType("DATE"), DataTypes.DATE()),
+ field("timestamp9_col", dbType("TIMESTAMP(9)"), DataTypes.TIMESTAMP(9)),
+ field("clob_col", dbType("CLOB"), DataTypes.STRING()));
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ 1.12345F,
+ 2.12345678790D,
+ 1.175E-10F,
+ 1.79769E+40D,
+ "a",
+ "abc",
+ "abcdef",
+ LocalDate.parse("1997-01-01"),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456789"),
+ "Hello World"),
+ Row.of(
+ 2L,
+ BigDecimal.valueOf(101.1234),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456"),
+ 1.12345F,
+ 2.12345678790D,
+ 1.175E-10F,
+ 1.79769E+40D,
+ "a",
+ "abc",
+ "abcdef",
+ LocalDate.parse("1997-01-02"),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456789"),
+ "Hey Leonard"));
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleTableSourceITCase.java
new file mode 100644
index 0000000..77192bf
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/table/OracleTableSourceITCase.java
@@ -0,0 +1,197 @@
+/*
+ * 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.databases.oracle.table;
+
+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.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.TableEnvironment;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CollectionUtil;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+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;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** The Table Source ITCase for {@link OracleDialect}. */
+class OracleTableSourceITCase extends AbstractTestBase implements OracleTestBase {
+
+ private static final TableRow INPUT_TABLE =
+ tableRow(
+ "oracle_test_table",
+ field("id", dbType("INTEGER"), DataTypes.INT().notNull()),
+ field("float_col", dbType("FLOAT"), DataTypes.FLOAT()),
+ field("double_col", dbType("DOUBLE PRECISION"), DataTypes.DOUBLE()),
+ field(
+ "decimal_col",
+ dbType("NUMBER(10,4) NOT NULL"),
+ DataTypes.DECIMAL(10, 4).notNull()),
+ field("binary_float_col", dbType("BINARY_FLOAT NOT NULL"), DataTypes.FLOAT()),
+ field(
+ "binary_double_col",
+ dbType("BINARY_DOUBLE NOT NULL"),
+ DataTypes.DOUBLE()),
+ field("char_col", dbType("CHAR NOT NULL"), DataTypes.CHAR(1)),
+ field("nchar_col", dbType("NCHAR(3) NOT NULL"), DataTypes.VARCHAR(3)),
+ field("varchar2_col", dbType("VARCHAR2(30) NOT NULL"), DataTypes.VARCHAR(30)),
+ field("date_col", dbType("DATE NOT NULL"), DataTypes.DATE()),
+ field("timestamp6_col", dbType("TIMESTAMP(6)"), DataTypes.TIMESTAMP(6)),
+ field("timestamp9_col", dbType("TIMESTAMP(9)"), DataTypes.TIMESTAMP(9)),
+ field("clob_col", dbType("CLOB"), DataTypes.STRING()),
+ field("blob_col", dbType("BLOB"), DataTypes.BYTES()));
+
+ private static final String INPUT_TABLE_NAME = INPUT_TABLE.getTableName();
+
+ private static StreamExecutionEnvironment env;
+ private static TableEnvironment tEnv;
+
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Collections.singletonList(INPUT_TABLE);
+ }
+
+ @BeforeEach
+ void before() throws Exception {
+ try (Connection conn = getMetadata().getConnection()) {
+ INPUT_TABLE.insertIntoTableValues(
+ conn,
+ "1, 1.12345, 2.12345678790, 100.1234, 1.175E-10, 1.79769E+40, 'a', 'abc', 'abcdef',"
+ + "TO_DATE('1997-01-01','yyyy-mm-dd'), TIMESTAMP '2020-01-01 15:35:00.123456',"
+ + "TIMESTAMP '2020-01-01 15:35:00.123456789', 'Hello World', hextoraw('453d7a34')");
+ INPUT_TABLE.insertIntoTableValues(
+ conn,
+ "2, 1.12345, 2.12345678790, 101.1234, -1.175E-10, -1.79769E+40, 'a', 'abc', 'abcdef',"
+ + "TO_DATE('1997-01-02','yyyy-mm-dd'), TIMESTAMP '2020-01-01 15:36:01.123456',"
+ + "TIMESTAMP '2020-01-01 15:36:01.123456789', 'Hey Leonard', hextoraw('453d7a34')");
+ }
+
+ env = StreamExecutionEnvironment.getExecutionEnvironment();
+ tEnv = StreamTableEnvironment.create(env);
+ }
+
+ @Test
+ void testJdbcSource() throws Exception {
+ tEnv.executeSql(INPUT_TABLE.getCreateQueryForFlink(getMetadata(), INPUT_TABLE_NAME));
+ Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE_NAME).collect();
+ List<String> result =
+ CollectionUtil.iteratorToList(collected).stream()
+ .map(Row::toString)
+ .sorted()
+ .collect(Collectors.toList());
+ List<String> expected =
+ Stream.of(
+ "+I[1, 1.12345, 2.1234567879, 100.1234, 1.175E-10, 1.79769E40, a, abc, abcdef, 1997-01-01, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, Hello World, [69, 61, 122, 52]]",
+ "+I[2, 1.12345, 2.1234567879, 101.1234, -1.175E-10, -1.79769E40, a, abc, abcdef, 1997-01-02, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, Hey Leonard, [69, 61, 122, 52]]")
+ .sorted()
+ .collect(Collectors.toList());
+ assertThat(result).isEqualTo(expected);
+ }
+
+ @Test
+ void testProject() throws Exception {
+ tEnv.executeSql(
+ INPUT_TABLE.getCreateQueryForFlink(
+ getMetadata(),
+ INPUT_TABLE_NAME,
+ Arrays.asList(
+ "id",
+ "timestamp6_col",
+ "timestamp9_col",
+ "binary_float_col",
+ "binary_double_col",
+ "decimal_col"),
+ Arrays.asList(
+ "'scan.partition.column'='id'",
+ "'scan.partition.num'='2'",
+ "'scan.partition.lower-bound'='0'",
+ "'scan.partition.upper-bound'='100'")));
+
+ Iterator<Row> collected =
+ tEnv.executeSql("SELECT id,timestamp6_col,decimal_col FROM " + INPUT_TABLE_NAME)
+ .collect();
+ List<String> result =
+ CollectionUtil.iteratorToList(collected).stream()
+ .map(Row::toString)
+ .sorted()
+ .collect(Collectors.toList());
+ List<String> expected =
+ Stream.of(
+ "+I[1, 2020-01-01T15:35:00.123456, 100.1234]",
+ "+I[2, 2020-01-01T15:36:01.123456, 101.1234]")
+ .collect(Collectors.toList());
+ assertThat(result).containsExactlyInAnyOrderElementsOf(expected);
+ }
+
+ @Test
+ void testLimit() throws Exception {
+ tEnv.executeSql(
+ INPUT_TABLE.getCreateQueryForFlink(
+ getMetadata(),
+ INPUT_TABLE_NAME,
+ Arrays.asList(
+ "id",
+ "timestamp6_col",
+ "timestamp9_col",
+ "binary_float_col",
+ "binary_double_col",
+ "decimal_col"),
+ Arrays.asList(
+ "'scan.partition.column'='id'",
+ "'scan.partition.num'='2'",
+ "'scan.partition.lower-bound'='1'",
+ "'scan.partition.upper-bound'='2'")));
+
+ Iterator<Row> collected =
+ tEnv.executeSql("SELECT * FROM " + INPUT_TABLE_NAME + " LIMIT 1").collect();
+ List<String> result =
+ CollectionUtil.iteratorToList(collected).stream()
+ .map(Row::toString)
+ .sorted()
+ .collect(Collectors.toList());
+
+ Set<String> expected = new HashSet<>();
+ expected.add(
+ "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 1.175E-10, 1.79769E40, 100.1234]");
+ expected.add(
+ "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, -1.175E-10, -1.79769E40, 101.1234]");
+ assertThat(result).hasSize(1);
+ assertThat(expected)
+ .as("The actual output is not a subset of the expected set.")
+ .containsAll(result);
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/xa/OracleExactlyOnceSinkE2eTest.java
similarity index 65%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/xa/OracleExactlyOnceSinkE2eTest.java
index 61314af..08ab8cd 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/xa/OracleExactlyOnceSinkE2eTest.java
@@ -16,17 +16,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.oracle.xa;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.oracle.OracleTestBase;
+import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/** A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. */
+public class OracleExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
+ implements OracleTestBase {}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresTestBase.java
new file mode 100644
index 0000000..8f29ecd
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresTestBase.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.jdbc.databases.postgres;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.postgres.PostgresDatabase;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for Postgres testing. */
+@ExtendWith(PostgresDatabase.class)
+public interface PostgresTestBase extends DatabaseTest {
+
+ @Override
+ default DatabaseMetadata getMetadata() {
+ return PostgresDatabase.getMetadata();
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogITCase.java
similarity index 97%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogITCase.java
index 925f75d..29c4c88 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogITCase.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
import org.apache.flink.table.api.EnvironmentSettings;
import org.apache.flink.table.api.TableEnvironment;
@@ -28,7 +28,7 @@
import java.util.List;
-import static org.apache.flink.connector.jdbc.catalog.PostgresCatalog.DEFAULT_DATABASE;
+import static org.apache.flink.connector.jdbc.databases.postgres.catalog.PostgresCatalog.DEFAULT_DATABASE;
import static org.apache.flink.table.api.config.ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM;
import static org.assertj.core.api.Assertions.assertThat;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTest.java
similarity index 89%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTest.java
index 4286a49..46ba695 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.catalog.CatalogBaseTable;
@@ -24,6 +24,7 @@
import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException;
import org.apache.flink.table.catalog.exceptions.TableNotExistException;
+import org.assertj.core.api.Assertions;
import org.junit.jupiter.api.Test;
import java.util.Arrays;
@@ -53,9 +54,9 @@
@Test
void testDbExists() {
- assertThat(catalog.databaseExists("nonexistent")).isFalse();
+ Assertions.assertThat(catalog.databaseExists("nonexistent")).isFalse();
- assertThat(catalog.databaseExists(PostgresCatalog.DEFAULT_DATABASE)).isTrue();
+ Assertions.assertThat(catalog.databaseExists(PostgresCatalog.DEFAULT_DATABASE)).isTrue();
}
// ------ tables ------
@@ -88,12 +89,15 @@
@Test
void testTableExists() {
- assertThat(catalog.tableExists(new ObjectPath(TEST_DB, "nonexist"))).isFalse();
+ Assertions.assertThat(catalog.tableExists(new ObjectPath(TEST_DB, "nonexist"))).isFalse();
- assertThat(catalog.tableExists(new ObjectPath(PostgresCatalog.DEFAULT_DATABASE, TABLE1)))
+ Assertions.assertThat(
+ catalog.tableExists(
+ new ObjectPath(PostgresCatalog.DEFAULT_DATABASE, TABLE1)))
.isTrue();
- assertThat(catalog.tableExists(new ObjectPath(TEST_DB, TABLE2))).isTrue();
- assertThat(catalog.tableExists(new ObjectPath(TEST_DB, "test_schema.t3"))).isTrue();
+ Assertions.assertThat(catalog.tableExists(new ObjectPath(TEST_DB, TABLE2))).isTrue();
+ Assertions.assertThat(catalog.tableExists(new ObjectPath(TEST_DB, "test_schema.t3")))
+ .isTrue();
}
@Test
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTestBase.java
similarity index 87%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTestBase.java
index f5a5e34..6e6c2d5 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresCatalogTestBase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresCatalogTestBase.java
@@ -16,14 +16,17 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
-import org.apache.flink.connector.jdbc.databases.postgres.PostgresDatabase;
+import org.apache.flink.connector.jdbc.databases.postgres.PostgresTestBase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import org.apache.flink.connector.jdbc.testutils.JdbcITCaseBase;
+import org.apache.flink.connector.jdbc.testutils.databases.postgres.PostgresDatabase;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.Schema;
import org.apache.flink.table.types.logical.DecimalType;
+import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.BeforeAll;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -34,13 +37,17 @@
import java.sql.Statement;
/** Test base for {@link PostgresCatalog}. */
-class PostgresCatalogTestBase implements PostgresDatabase, JdbcITCaseBase {
+class PostgresCatalogTestBase implements JdbcITCaseBase, PostgresTestBase {
public static final Logger LOG = LoggerFactory.getLogger(PostgresCatalogTestBase.class);
+ private static DatabaseMetadata getStaticMetadata() {
+ return PostgresDatabase.getMetadata();
+ }
+
protected static final String TEST_CATALOG_NAME = "mypg";
- protected static final String TEST_USERNAME = CONTAINER.getUsername();
- protected static final String TEST_PWD = CONTAINER.getPassword();
+ protected static final String TEST_USERNAME = getStaticMetadata().getUsername();
+ protected static final String TEST_PWD = getStaticMetadata().getPassword();
protected static final String TEST_DB = "test";
protected static final String TEST_SCHEMA = "test_schema";
protected static final String TABLE1 = "t1";
@@ -59,7 +66,7 @@
@BeforeAll
static void init() throws SQLException {
// jdbc:postgresql://localhost:50807/postgres?user=postgres
- String jdbcUrl = CONTAINER.getJdbcUrl();
+ String jdbcUrl = getStaticMetadata().getJdbcUrl();
// jdbc:postgresql://localhost:50807/
baseUrl = jdbcUrl.substring(0, jdbcUrl.lastIndexOf("/"));
@@ -125,6 +132,42 @@
"insert into %s values (%s);", TABLE_SERIAL_TYPE, getSerialTable().values));
}
+ @AfterAll
+ static void afterAll() throws SQLException {
+ executeSQL(TEST_DB, String.format("DROP SCHEMA %s CASCADE", TEST_SCHEMA));
+ executeSQL(
+ TEST_DB,
+ String.format("DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE2)));
+
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format("DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE1)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format("DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE4)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format("DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE5)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format(
+ "DROP TABLE %s ",
+ PostgresTablePath.fromFlinkTableName(TABLE_PRIMITIVE_TYPE)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format(
+ "DROP TABLE %s ",
+ PostgresTablePath.fromFlinkTableName(TABLE_PRIMITIVE_TYPE2)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format(
+ "DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE_ARRAY_TYPE)));
+ executeSQL(
+ PostgresCatalog.DEFAULT_DATABASE,
+ String.format(
+ "DROP TABLE %s ", PostgresTablePath.fromFlinkTableName(TABLE_SERIAL_TYPE)));
+ }
+
public static void createTable(PostgresTablePath tablePath, String tableSchemaSql)
throws SQLException {
executeSQL(
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePathTest.java
similarity index 94%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePathTest.java
index 61314af..6798834 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/PostgresTablePathTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog;
import org.junit.jupiter.api.Test;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/factory/JdbcCatalogFactoryTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/factory/JdbcCatalogFactoryTest.java
similarity index 79%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/factory/JdbcCatalogFactoryTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/factory/JdbcCatalogFactoryTest.java
index 705e494..5c66bda 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/factory/JdbcCatalogFactoryTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/catalog/factory/JdbcCatalogFactoryTest.java
@@ -16,16 +16,18 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog.factory;
+package org.apache.flink.connector.jdbc.databases.postgres.catalog.factory;
import org.apache.flink.connector.jdbc.catalog.JdbcCatalog;
-import org.apache.flink.connector.jdbc.catalog.PostgresCatalog;
-import org.apache.flink.connector.jdbc.databases.postgres.PostgresDatabase;
+import org.apache.flink.connector.jdbc.catalog.factory.JdbcCatalogFactory;
+import org.apache.flink.connector.jdbc.catalog.factory.JdbcCatalogFactoryOptions;
+import org.apache.flink.connector.jdbc.databases.postgres.PostgresTestBase;
+import org.apache.flink.connector.jdbc.databases.postgres.catalog.PostgresCatalog;
import org.apache.flink.table.catalog.Catalog;
import org.apache.flink.table.catalog.CommonCatalogOptions;
import org.apache.flink.table.factories.FactoryUtil;
-import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -37,7 +39,7 @@
import static org.assertj.core.api.Assertions.assertThat;
/** Test for {@link JdbcCatalogFactory}. */
-class JdbcCatalogFactoryTest implements PostgresDatabase {
+class JdbcCatalogFactoryTest implements PostgresTestBase {
public static final Logger LOG = LoggerFactory.getLogger(JdbcCatalogFactoryTest.class);
@@ -45,13 +47,11 @@
protected static JdbcCatalog catalog;
protected static final String TEST_CATALOG_NAME = "mypg";
- protected static final String TEST_USERNAME = CONTAINER.getUsername();
- protected static final String TEST_PWD = CONTAINER.getPassword();
- @BeforeAll
- static void setup() throws SQLException {
+ @BeforeEach
+ void setup() throws SQLException {
// jdbc:postgresql://localhost:50807/postgres?user=postgres
- String jdbcUrl = CONTAINER.getJdbcUrl();
+ String jdbcUrl = getMetadata().getJdbcUrl();
// jdbc:postgresql://localhost:50807/
baseUrl = jdbcUrl.substring(0, jdbcUrl.lastIndexOf("/"));
@@ -60,8 +60,8 @@
Thread.currentThread().getContextClassLoader(),
TEST_CATALOG_NAME,
PostgresCatalog.DEFAULT_DATABASE,
- TEST_USERNAME,
- TEST_PWD,
+ getMetadata().getUsername(),
+ getMetadata().getPassword(),
baseUrl);
}
@@ -71,8 +71,8 @@
options.put(CommonCatalogOptions.CATALOG_TYPE.key(), JdbcCatalogFactoryOptions.IDENTIFIER);
options.put(
JdbcCatalogFactoryOptions.DEFAULT_DATABASE.key(), PostgresCatalog.DEFAULT_DATABASE);
- options.put(JdbcCatalogFactoryOptions.USERNAME.key(), TEST_USERNAME);
- options.put(JdbcCatalogFactoryOptions.PASSWORD.key(), TEST_PWD);
+ options.put(JdbcCatalogFactoryOptions.USERNAME.key(), getMetadata().getUsername());
+ options.put(JdbcCatalogFactoryOptions.PASSWORD.key(), getMetadata().getPassword());
options.put(JdbcCatalogFactoryOptions.BASE_URL.key(), baseUrl);
final Catalog actualCatalog =
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresDialectTypeTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectTypeTest.java
similarity index 97%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresDialectTypeTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectTypeTest.java
index 91d968d..ce1c4e8 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresDialectTypeTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/dialect/PostgresDialectTypeTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.postgres;
+package org.apache.flink.connector.jdbc.databases.postgres.dialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectTypeTest;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSinkITCase.java
similarity index 62%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSinkITCase.java
index 61314af..d0ba3c6 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSinkITCase.java
@@ -16,17 +16,12 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.table;
-import org.junit.jupiter.api.Test;
+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.JdbcDynamicTableSinkITCase;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/** The Table Sink ITCase for {@link PostgresDialect}. */
+public class PostgresDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
+ implements PostgresTestBase {}
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
new file mode 100644
index 0000000..174be59
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/table/PostgresDynamicTableSourceITCase.java
@@ -0,0 +1,72 @@
+/*
+ * 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.databases.postgres.table;
+
+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.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import java.math.BigDecimal;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.List;
+
+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 PostgresDialect}. */
+public class PostgresDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase
+ implements PostgresTestBase {
+
+ @Override
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ 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()));
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ 1.175E-37F,
+ 1.79769E308D,
+ LocalTime.parse("15:35")),
+ 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")));
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/xa/PostgresExactlyOnceSinkE2eTest.java
similarity index 60%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/xa/PostgresExactlyOnceSinkE2eTest.java
index 61314af..38d165d 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/xa/PostgresExactlyOnceSinkE2eTest.java
@@ -16,17 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.postgres.xa;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.postgres.PostgresTestBase;
+import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/**
+ * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with suspending
+ * connections (requires pooling) and honoring limits (properly closing connections).
+ */
+public class PostgresExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
+ implements PostgresTestBase {}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerTestBase.java
new file mode 100644
index 0000000..d59f9eb
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerTestBase.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.jdbc.databases.sqlserver;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.databases.sqlserver.SqlServerDatabase;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/** Base class for SqlServer testing. */
+@ExtendWith(SqlServerDatabase.class)
+public interface SqlServerTestBase extends DatabaseTest {
+
+ @Override
+ default DatabaseMetadata getMetadata() {
+ return SqlServerDatabase.getMetadata();
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerPreparedStatementTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerPreparedStatementTest.java
similarity index 98%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerPreparedStatementTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerPreparedStatementTest.java
index 48bf24b..33fc3ed 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerPreparedStatementTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/dialect/SqlServerPreparedStatementTest.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.sqlserver.dialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectLoader;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSinkITCase.java
new file mode 100644
index 0000000..c2e56a6
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSinkITCase.java
@@ -0,0 +1,54 @@
+/*
+ * 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.databases.sqlserver.table;
+
+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.JdbcDynamicTableSinkITCase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
+
+/** The Table Sink ITCase for {@link SqlServerDialect}. */
+public class SqlServerDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase
+ implements SqlServerTestBase {
+
+ @Override
+ protected TableRow createUpsertOutputTable() {
+ return tableRow(
+ "dynamicSinkForUpsert",
+ pkField("cnt", DataTypes.BIGINT().notNull()),
+ field("lencnt", DataTypes.BIGINT().notNull()),
+ pkField("cTag", DataTypes.INT().notNull()),
+ field("ts", dbType("DATETIME2"), DataTypes.TIMESTAMP()));
+ }
+
+ @Override
+ protected TableRow createAppendOutputTable() {
+ return tableRow(
+ "dynamicSinkForAppend",
+ field("id", DataTypes.INT().notNull()),
+ field("num", DataTypes.BIGINT().notNull()),
+ field("ts", dbType("DATETIME2"), DataTypes.TIMESTAMP()));
+ }
+}
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
new file mode 100644
index 0000000..f9444f4
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerDynamicTableSourceITCase.java
@@ -0,0 +1,123 @@
+/*
+ * 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.databases.sqlserver.table;
+
+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.tables.TableRow;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.types.Row;
+
+import org.junit.jupiter.api.Test;
+
+import java.math.BigDecimal;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.List;
+
+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;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** The Table Source ITCase for {@link SqlServerDialect}. */
+public class SqlServerDynamicTableSourceITCase extends JdbcDynamicTableSourceITCase
+ implements SqlServerTestBase {
+
+ @Override
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ field("id", DataTypes.BIGINT().notNull()),
+ field("decimal_col", DataTypes.DECIMAL(10, 4)),
+ field("timestamp6_col", dbType("DATETIME2"), DataTypes.TIMESTAMP(6)),
+ // other fields
+ field("tiny_int", dbType("TINYINT"), DataTypes.TINYINT()),
+ field("small_int", dbType("SMALLINT"), DataTypes.SMALLINT()),
+ field("big_int", dbType("BIGINT"), DataTypes.BIGINT().notNull()),
+ field("float_col", dbType("REAL"), DataTypes.FLOAT()),
+ field("double_col", dbType("FLOAT"), DataTypes.DOUBLE()),
+ field("bool", dbType("BIT"), DataTypes.BOOLEAN()),
+ field("date_col", dbType("DATE"), DataTypes.DATE()),
+ field("time_col", dbType("TIME(3)"), DataTypes.TIME()),
+ field("datetime_col", dbType("DATETIME"), DataTypes.TIMESTAMP()),
+ 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("nvarchar2_col", dbType("NVARCHAR(30)"), DataTypes.STRING()),
+ field("text_col", dbType("TEXT"), DataTypes.STRING()),
+ field("ntext_col", dbType("NTEXT"), DataTypes.STRING()));
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ Byte.decode("2"),
+ Short.decode("4"),
+ 10000000000L,
+ 1.12345F,
+ 2.12345678791D,
+ false,
+ LocalDate.parse("1997-01-01"),
+ LocalTime.parse("05:20:20"),
+ LocalDateTime.parse("2020-01-01T15:35:00.123"),
+ LocalDateTime.parse("2020-01-01T15:35:00.1234567"),
+ "a",
+ "abc",
+ "abcdef",
+ "xyz",
+ "Hello World",
+ "World Hello"),
+ Row.of(
+ 2L,
+ BigDecimal.valueOf(101.1234),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456"),
+ Byte.decode("2"),
+ Short.decode("4"),
+ 10000000000L,
+ 1.12345F,
+ 2.12345678791D,
+ true,
+ LocalDate.parse("1997-01-02"),
+ LocalTime.parse("05:20:20"),
+ LocalDateTime.parse("2020-01-01T15:36:01.123"),
+ LocalDateTime.parse("2020-01-01T15:36:01.1234567"),
+ "a",
+ "abc",
+ "abcdef",
+ "xyz",
+ "Hey Leonard",
+ "World Hello"));
+ }
+
+ @Test
+ @Override
+ public void testLimit() {
+ assertThatThrownBy(super::testLimit)
+ .isInstanceOf(IllegalArgumentException.class)
+ .hasMessage("SqlServerDialect does not support limit clause");
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerTableSourceITCase.java
similarity index 68%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSourceITCase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerTableSourceITCase.java
index 91da3ce..02805be 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSourceITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/table/SqlServerTableSourceITCase.java
@@ -16,107 +16,92 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
+package org.apache.flink.connector.jdbc.databases.sqlserver.table;
-import org.apache.flink.connector.jdbc.databases.sqlserver.SqlServerDatabase;
+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.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.types.Row;
import org.apache.flink.util.CollectionUtil;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import java.sql.Connection;
-import java.sql.DriverManager;
import java.sql.SQLException;
-import java.sql.Statement;
+import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+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;
import static org.assertj.core.api.Assertions.assertThat;
/** The Table Source ITCase for {@link SqlServerDialect}. */
-class SqlServerTableSourceITCase extends AbstractTestBase implements SqlServerDatabase {
+class SqlServerTableSourceITCase extends AbstractTestBase implements SqlServerTestBase {
- private static final String INPUT_TABLE = "sql_test_table";
+ private static final TableRow INPUT_TABLE =
+ tableRow(
+ "sql_test_table",
+ field("id", dbType("INT"), DataTypes.INT().notNull()),
+ field("tiny_int", dbType("TINYINT"), DataTypes.TINYINT()),
+ field("small_int", dbType("SMALLINT"), DataTypes.SMALLINT()),
+ field("big_int", dbType("BIGINT"), DataTypes.BIGINT().notNull()),
+ field("float_col", dbType("REAL"), DataTypes.FLOAT()),
+ field("double_col", dbType("FLOAT"), DataTypes.DOUBLE()),
+ field("decimal_col", dbType("DECIMAL(10, 4)"), DataTypes.DECIMAL(10, 4)),
+ field("bool", dbType("BIT"), DataTypes.BOOLEAN()),
+ field("date_col", dbType("DATE"), DataTypes.DATE()),
+ field("time_col", dbType("TIME(5)"), DataTypes.TIME(0)),
+ field("datetime_col", dbType("DATETIME"), DataTypes.TIMESTAMP()),
+ field(
+ "datetime2_col",
+ dbType("DATETIME2"),
+ DataTypes.TIMESTAMP_WITH_TIME_ZONE()),
+ field("char_col", dbType("CHAR"), DataTypes.STRING()),
+ field("nchar_col", dbType("NCHAR(3)"), DataTypes.STRING()),
+ field("varchar2_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()),
+ field("binary_col", dbType("BINARY(10)"), DataTypes.BYTES()));
+
+ private static final String INPUT_TABLE_NAME = INPUT_TABLE.getTableName();
private static StreamExecutionEnvironment env;
private static TableEnvironment tEnv;
- @BeforeAll
- static void beforeAll() throws ClassNotFoundException, SQLException {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement statement = conn.createStatement()) {
- statement.executeUpdate(
- "CREATE TABLE "
- + INPUT_TABLE
- + " ("
- + "id INT NOT NULL,"
- + "tiny_int TINYINT,"
- + "small_int SMALLINT,"
- + "big_int BIGINT,"
- + "float_col REAL,"
- + "double_col FLOAT ,"
- + "decimal_col DECIMAL(10, 4) NOT NULL,"
- + "bool BIT NOT NULL,"
- + "date_col DATE NOT NULL,"
- + "time_col TIME(5) NOT NULL,"
- + "datetime_col DATETIME,"
- + "datetime2_col DATETIME2,"
- + "char_col CHAR NOT NULL,"
- + "nchar_col NCHAR(3) NOT NULL,"
- + "varchar2_col VARCHAR(30) NOT NULL,"
- + "nvarchar2_col NVARCHAR(30) NOT NULL,"
- + "text_col TEXT,"
- + "ntext_col NTEXT,"
- + "binary_col BINARY(10)"
- + ")");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "1, 2, 4, 10000000000, 1.12345, 2.12345678791, 100.1234, 0, "
- + "'1997-01-01', '05:20:20.222','2020-01-01 15:35:00.123',"
- + "'2020-01-01 15:35:00.1234567', 'a', 'abc', 'abcdef', 'xyz',"
- + "'Hello World', 'World Hello', 1024)");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "2, 2, 4, 10000000000, 1.12345, 2.12345678791, 101.1234, 1, "
- + "'1997-01-02', '05:20:20.222','2020-01-01 15:36:01.123',"
- + "'2020-01-01 15:36:01.1234567', 'a', 'abc', 'abcdef', 'xyz',"
- + "'Hey Leonard', 'World Hello', 1024)");
- }
- }
-
- @AfterAll
- static void afterAll() throws Exception {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement statement = conn.createStatement()) {
- statement.executeUpdate("DROP TABLE " + INPUT_TABLE);
- }
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Collections.singletonList(INPUT_TABLE);
}
@BeforeEach
- void before() throws Exception {
+ void before() throws SQLException {
+
+ try (Connection conn = getMetadata().getConnection()) {
+ INPUT_TABLE.insertIntoTableValues(
+ conn,
+ "1, 2, 4, 10000000000, 1.12345, 2.12345678791, 100.1234, 0, "
+ + "'1997-01-01', '05:20:20.222','2020-01-01 15:35:00.123',"
+ + "'2020-01-01 15:35:00.1234567', 'a', 'abc', 'abcdef', 'xyz',"
+ + "'Hello World', 'World Hello', 1024");
+ INPUT_TABLE.insertIntoTableValues(
+ conn,
+ "2, 2, 4, 10000000000, 1.12345, 2.12345678791, 101.1234, 1, "
+ + "'1997-01-02', '05:20:20.222','2020-01-01 15:36:01.123',"
+ + "'2020-01-01 15:36:01.1234567', 'a', 'abc', 'abcdef', 'xyz',"
+ + "'Hey Leonard', 'World Hello', 1024");
+ }
env = StreamExecutionEnvironment.getExecutionEnvironment();
tEnv = StreamTableEnvironment.create(env);
}
@@ -124,7 +109,7 @@
@Test
void testJdbcSource() throws Exception {
createFlinkTable();
- Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE).collect();
+ Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE_NAME).collect();
List<String> result =
CollectionUtil.iteratorToList(collected).stream()
.map(Row::toString)
@@ -149,7 +134,8 @@
void testProject() throws Exception {
createFlinkTable();
Iterator<Row> collected =
- tEnv.executeSql("SELECT id,datetime_col,decimal_col FROM " + INPUT_TABLE).collect();
+ tEnv.executeSql("SELECT id,datetime_col,decimal_col FROM " + INPUT_TABLE_NAME)
+ .collect();
List<String> result =
CollectionUtil.iteratorToList(collected).stream()
.map(Row::toString)
@@ -170,7 +156,7 @@
Iterator<Row> collected =
tEnv.executeSql(
"SELECT id,datetime_col,decimal_col FROM "
- + INPUT_TABLE
+ + INPUT_TABLE_NAME
+ " WHERE id = 1")
.collect();
List<String> result =
@@ -186,7 +172,7 @@
private void createFlinkTable() {
tEnv.executeSql(
"CREATE TABLE "
- + INPUT_TABLE
+ + INPUT_TABLE_NAME
+ " ("
+ "id INT NOT NULL,"
+ "tiny_int TINYINT,"
@@ -213,7 +199,7 @@
+ getMetadata().getJdbcUrl()
+ "',"
+ " 'table-name'='"
- + INPUT_TABLE
+ + INPUT_TABLE_NAME
+ "',"
+ " 'username'='"
+ getMetadata().getUsername()
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/xa/SqlServerExactlyOnceSinkE2eTest.java
similarity index 63%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/xa/SqlServerExactlyOnceSinkE2eTest.java
index 61314af..cfc570c 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/xa/SqlServerExactlyOnceSinkE2eTest.java
@@ -16,17 +16,14 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.databases.sqlserver.xa;
-import org.junit.jupiter.api.Test;
+import org.apache.flink.connector.jdbc.databases.sqlserver.SqlServerTestBase;
+import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
-}
+/**
+ * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with errors on
+ * closing connections.
+ */
+public class SqlServerExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
+ implements SqlServerTestBase {}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlExactlyOnceSinkE2eTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlExactlyOnceSinkE2eTest.java
deleted file mode 100644
index 24fa8df..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/mysql/MySqlExactlyOnceSinkE2eTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package org.apache.flink.connector.jdbc.dialect.mysql;
-
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlDatabase;
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlMetadata;
-import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import org.apache.flink.util.function.SerializableSupplier;
-
-import com.mysql.cj.jdbc.MysqlXADataSource;
-
-import javax.sql.XADataSource;
-
-/**
- * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with errors on
- * closing connections.
- */
-public class MySqlExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
- implements MySqlDatabase {
-
- @Override
- public DatabaseMetadata getMetadata() {
- return new MySqlMetadata(CONTAINER, true);
- }
-
- @Override
- public SerializableSupplier<XADataSource> getDataSourceSupplier() {
- return () -> {
- MysqlXADataSource xaDataSource = new MysqlXADataSource();
- xaDataSource.setUrl(CONTAINER.getJdbcUrl());
- xaDataSource.setUser(CONTAINER.getUsername());
- xaDataSource.setPassword(CONTAINER.getPassword());
- return xaDataSource;
- };
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleExactlyOnceSinkE2eTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleExactlyOnceSinkE2eTest.java
deleted file mode 100644
index 072131b..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleExactlyOnceSinkE2eTest.java
+++ /dev/null
@@ -1,31 +0,0 @@
-package org.apache.flink.connector.jdbc.dialect.oracle;
-
-import org.apache.flink.connector.jdbc.databases.oracle.OracleXaDatabase;
-import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import org.apache.flink.util.function.SerializableSupplier;
-
-import oracle.jdbc.xa.client.OracleXADataSource;
-
-import javax.sql.XADataSource;
-
-import java.sql.SQLException;
-
-/** A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. */
-public class OracleExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
- implements OracleXaDatabase {
-
- @Override
- public SerializableSupplier<XADataSource> getDataSourceSupplier() {
- return () -> {
- try {
- OracleXADataSource xaDataSource = new OracleXADataSource();
- xaDataSource.setURL(CONTAINER.getJdbcUrl());
- xaDataSource.setUser(CONTAINER.getUsername());
- xaDataSource.setPassword(CONTAINER.getPassword());
- return xaDataSource;
- } catch (SQLException ex) {
- throw new RuntimeException(ex);
- }
- };
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSinkITCase.java
deleted file mode 100644
index 605a42f..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSinkITCase.java
+++ /dev/null
@@ -1,465 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.dialect.oracle;
-
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.jdbc.databases.oracle.OracleDatabase;
-import org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction;
-import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
-import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.table.planner.runtime.utils.TestData;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
-
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.connector.jdbc.internal.JdbcTableOutputFormatTest.check;
-import static org.apache.flink.table.api.Expressions.$;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-
-/** The Table Sink ITCase for {@link OracleDialect}. */
-class OracleTableSinkITCase extends AbstractTestBase implements OracleDatabase {
-
- public static final String OUTPUT_TABLE1 = "dynamicSinkForUpsert";
- public static final String OUTPUT_TABLE2 = "dynamicSinkForAppend";
- public static final String OUTPUT_TABLE3 = "dynamicSinkForBatch";
- public static final String OUTPUT_TABLE4 = "REAL_TABLE";
- public static final String OUTPUT_TABLE5 = "checkpointTable";
- public static final String USER_TABLE = "USER_TABLE";
-
- @BeforeAll
- static void beforeAll() throws ClassNotFoundException, SQLException {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement stat = conn.createStatement()) {
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE1
- + " ("
- + "cnt NUMBER(38,2) DEFAULT 0 NOT NULL,"
- + "lencnt NUMBER(38,2) DEFAULT 0 NOT NULL,"
- + "cTag INT DEFAULT 0 NOT NULL,"
- + "ts TIMESTAMP,"
- + "PRIMARY KEY (cnt, cTag))");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE2
- + " ("
- + "id INT DEFAULT 0 NOT NULL,"
- + "num NUMBER DEFAULT 0 NOT NULL,"
- + "ts TIMESTAMP)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE3
- + " ("
- + "NAME VARCHAR(20) NOT NULL,"
- + "SCORE NUMBER DEFAULT 0 NOT NULL)");
-
- stat.executeUpdate("CREATE TABLE " + OUTPUT_TABLE4 + " (real_data REAL)");
-
- stat.executeUpdate(
- "CREATE TABLE " + OUTPUT_TABLE5 + " (" + "id NUMBER DEFAULT 0 NOT NULL)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + USER_TABLE
- + " ("
- + "user_id VARCHAR(20) NOT NULL,"
- + "user_name VARCHAR(20) NOT NULL,"
- + "email VARCHAR(255),"
- + "balance DECIMAL(18,2),"
- + "balance2 DECIMAL(18,2),"
- + "PRIMARY KEY (user_id))");
- }
- }
-
- @AfterAll
- static void afterAll() throws Exception {
- TestValuesTableFactory.clearAllData();
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement stat = conn.createStatement()) {
- stat.execute("DROP TABLE " + OUTPUT_TABLE1);
- stat.execute("DROP TABLE " + OUTPUT_TABLE2);
- stat.execute("DROP TABLE " + OUTPUT_TABLE3);
- stat.execute("DROP TABLE " + OUTPUT_TABLE4);
- stat.execute("DROP TABLE " + OUTPUT_TABLE5);
- stat.execute("DROP TABLE " + USER_TABLE);
- }
- }
-
- public static DataStream<Tuple4<Integer, Long, String, Timestamp>> get4TupleDataStream(
- StreamExecutionEnvironment env) {
- List<Tuple4<Integer, Long, String, Timestamp>> data = new ArrayList<>();
- data.add(new Tuple4<>(1, 1L, "Hi", Timestamp.valueOf("1970-01-01 00:00:00.001")));
- data.add(new Tuple4<>(2, 2L, "Hello", Timestamp.valueOf("1970-01-01 00:00:00.002")));
- data.add(new Tuple4<>(3, 2L, "Hello world", Timestamp.valueOf("1970-01-01 00:00:00.003")));
- data.add(
- new Tuple4<>(
- 4,
- 3L,
- "Hello world, how are you?",
- Timestamp.valueOf("1970-01-01 00:00:00.004")));
- data.add(new Tuple4<>(5, 3L, "I am fine.", Timestamp.valueOf("1970-01-01 00:00:00.005")));
- data.add(
- new Tuple4<>(
- 6, 3L, "Luke Skywalker", Timestamp.valueOf("1970-01-01 00:00:00.006")));
- data.add(new Tuple4<>(7, 4L, "Comment#1", Timestamp.valueOf("1970-01-01 00:00:00.007")));
- data.add(new Tuple4<>(8, 4L, "Comment#2", Timestamp.valueOf("1970-01-01 00:00:00.008")));
- data.add(new Tuple4<>(9, 4L, "Comment#3", Timestamp.valueOf("1970-01-01 00:00:00.009")));
- data.add(new Tuple4<>(10, 4L, "Comment#4", Timestamp.valueOf("1970-01-01 00:00:00.010")));
- data.add(new Tuple4<>(11, 5L, "Comment#5", Timestamp.valueOf("1970-01-01 00:00:00.011")));
- data.add(new Tuple4<>(12, 5L, "Comment#6", Timestamp.valueOf("1970-01-01 00:00:00.012")));
- data.add(new Tuple4<>(13, 5L, "Comment#7", Timestamp.valueOf("1970-01-01 00:00:00.013")));
- data.add(new Tuple4<>(14, 5L, "Comment#8", Timestamp.valueOf("1970-01-01 00:00:00.014")));
- data.add(new Tuple4<>(15, 5L, "Comment#9", Timestamp.valueOf("1970-01-01 00:00:00.015")));
- data.add(new Tuple4<>(16, 6L, "Comment#10", Timestamp.valueOf("1970-01-01 00:00:00.016")));
- data.add(new Tuple4<>(17, 6L, "Comment#11", Timestamp.valueOf("1970-01-01 00:00:00.017")));
- data.add(new Tuple4<>(18, 6L, "Comment#12", Timestamp.valueOf("1970-01-01 00:00:00.018")));
- data.add(new Tuple4<>(19, 6L, "Comment#13", Timestamp.valueOf("1970-01-01 00:00:00.019")));
- data.add(new Tuple4<>(20, 6L, "Comment#14", Timestamp.valueOf("1970-01-01 00:00:00.020")));
- data.add(new Tuple4<>(21, 6L, "Comment#15", Timestamp.valueOf("1970-01-01 00:00:00.021")));
-
- Collections.shuffle(data);
- return env.fromCollection(data);
- }
-
- @Test
- void testReal() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- StreamTableEnvironment tEnv =
- StreamTableEnvironment.create(env, EnvironmentSettings.inStreamingMode());
-
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " real_data float"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE4
- + "'"
- + ")");
-
- tEnv.executeSql("INSERT INTO upsertSink SELECT CAST(1.1 as FLOAT)").await();
- check(
- new Row[] {Row.of(1.1f)},
- getMetadata().getJdbcUrlWithCredentials(),
- "REAL_TABLE",
- new String[] {"real_data"});
- }
-
- @Test
- void testUpsert() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
-
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env)
- .assignTimestampsAndWatermarks(
- new AscendingTimestampExtractor<
- Tuple4<Integer, Long, String, Timestamp>>() {
- @Override
- public long extractAscendingTimestamp(
- Tuple4<Integer, Long, String, Timestamp>
- element) {
- return element.f0;
- }
- }),
- $("id"),
- $("num"),
- $("text"),
- $("ts"));
-
- tEnv.createTemporaryView("T", t);
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " cnt DECIMAL(18,2),"
- + " lencnt DECIMAL(18,2),"
- + " cTag INT,"
- + " ts TIMESTAMP(3),"
- + " PRIMARY KEY (cnt, cTag) NOT ENFORCED"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE1
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0',"
- + " 'sink.max-retries' = '0'"
- + ")");
-
- tEnv.executeSql(
- "INSERT INTO upsertSink \n"
- + "SELECT cnt, COUNT(len) AS lencnt, cTag, MAX(ts) AS ts\n"
- + "FROM (\n"
- + " SELECT len, COUNT(id) as cnt, cTag, MAX(ts) AS ts\n"
- + " FROM (SELECT id, CHAR_LENGTH(text) AS len, (CASE WHEN id > 0 THEN 1 ELSE 0 END) cTag, ts FROM T)\n"
- + " GROUP BY len, cTag\n"
- + ")\n"
- + "GROUP BY cnt, cTag")
- .await();
- check(
- new Row[] {
- Row.of(1, 5, 1, Timestamp.valueOf("1970-01-01 00:00:00.006")),
- Row.of(7, 1, 1, Timestamp.valueOf("1970-01-01 00:00:00.021")),
- Row.of(9, 1, 1, Timestamp.valueOf("1970-01-01 00:00:00.015"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE1,
- new String[] {"cnt", "lencnt", "cTag", "ts"});
- }
-
- @Test
- void testAppend() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- env.getConfig().setParallelism(1);
- StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
-
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env), $("id"), $("num"), $("text"), $("ts"));
-
- tEnv.registerTable("T", t);
-
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " id INT,"
- + " num BIGINT,"
- + " ts TIMESTAMP(3)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE2
- + "'"
- + ")");
-
- tEnv.executeSql("INSERT INTO upsertSink SELECT id, num, ts FROM T WHERE id IN (2, 10, 20)")
- .await();
- check(
- new Row[] {
- Row.of(2, 2, Timestamp.valueOf("1970-01-01 00:00:00.002")),
- Row.of(10, 4, Timestamp.valueOf("1970-01-01 00:00:00.01")),
- Row.of(20, 6, Timestamp.valueOf("1970-01-01 00:00:00.02"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE2,
- new String[] {"id", "num", "ts"});
- }
-
- @Test
- void testBatchSink() throws Exception {
- TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
-
- tEnv.executeSql(
- "CREATE TABLE USER_RESULT("
- + "NAME VARCHAR,"
- + "SCORE BIGINT"
- + ") WITH ( "
- + "'connector' = 'jdbc',"
- + "'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + "'table-name' = '"
- + OUTPUT_TABLE3
- + "',"
- + "'sink.buffer-flush.max-rows' = '2',"
- + "'sink.buffer-flush.interval' = '300ms',"
- + "'sink.max-retries' = '4'"
- + ")");
-
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO USER_RESULT\n"
- + "SELECT user_name, score "
- + "FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), "
- + "(42, 'Kim'), (1, 'Bob')) "
- + "AS UserCountTable(score, user_name)");
- tableResult.await();
-
- check(
- new Row[] {
- Row.of("Bob", 1),
- Row.of("Tom", 22),
- Row.of("Kim", 42),
- Row.of("Kim", 42),
- Row.of("Bob", 1)
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE3,
- new String[] {"NAME", "SCORE"});
- }
-
- @Test
- void testReadingFromChangelogSource() throws Exception {
- TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
- String dataId = TestValuesTableFactory.registerData(TestData.userChangelog());
- tEnv.executeSql(
- "CREATE TABLE user_logs (\n"
- + " user_id STRING,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,2),\n"
- + " balance2 AS balance * 2\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'data-id' = '"
- + dataId
- + "',\n"
- + " 'changelog-mode' = 'I,UA,UB,D'\n"
- + ")");
- tEnv.executeSql(
- "CREATE TABLE user_sink (\n"
- + " user_id STRING PRIMARY KEY NOT ENFORCED,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,2),\n"
- + " balance2 DECIMAL(18,2)\n"
- + ") WITH (\n"
- + " 'connector' = 'jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name' = '"
- + USER_TABLE
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0'"
- + // disable async flush
- ")");
- tEnv.executeSql("INSERT INTO user_sink SELECT * FROM user_logs").await();
-
- check(
- new Row[] {
- Row.of(
- "user1",
- "Tom",
- "tom123@gmail.com",
- new BigDecimal("8.1"),
- new BigDecimal("16.2")),
- Row.of(
- "user3",
- "Bailey",
- "bailey@qq.com",
- new BigDecimal("9.99"),
- new BigDecimal("19.98")),
- Row.of(
- "user4",
- "Tina",
- "tina@gmail.com",
- new BigDecimal("11.3"),
- new BigDecimal("22.6"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- USER_TABLE,
- new String[] {"user_id", "user_name", "email", "balance", "balance2"});
- }
-
- @Test
- void testFlushBufferWhenCheckpoint() throws Exception {
- Map<String, String> options = new HashMap<>();
- options.put("connector", "jdbc");
- options.put("url", getMetadata().getJdbcUrlWithCredentials());
- options.put("table-name", OUTPUT_TABLE5);
- options.put("sink.buffer-flush.interval", "0");
-
- ResolvedSchema schema =
- ResolvedSchema.of(Column.physical("id", DataTypes.BIGINT().notNull()));
-
- DynamicTableSink tableSink = createTableSink(schema, options);
-
- SinkRuntimeProviderContext context = new SinkRuntimeProviderContext(false);
- SinkFunctionProvider sinkProvider =
- (SinkFunctionProvider) tableSink.getSinkRuntimeProvider(context);
- GenericJdbcSinkFunction<RowData> sinkFunction =
- (GenericJdbcSinkFunction<RowData>) sinkProvider.createSinkFunction();
- sinkFunction.setRuntimeContext(new MockStreamingRuntimeContext(true, 1, 0));
- sinkFunction.open(new Configuration());
- sinkFunction.invoke(GenericRowData.of(1L), SinkContextUtil.forTimestamp(1));
- sinkFunction.invoke(GenericRowData.of(2L), SinkContextUtil.forTimestamp(1));
-
- check(
- new Row[] {},
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE5,
- new String[] {"id"});
- sinkFunction.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
- check(
- new Row[] {Row.of(1L), Row.of(2L)},
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE5,
- new String[] {"id"});
- sinkFunction.close();
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSourceITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSourceITCase.java
deleted file mode 100644
index a891b64..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/oracle/OracleTableSourceITCase.java
+++ /dev/null
@@ -1,251 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.dialect.oracle;
-
-import org.apache.flink.connector.jdbc.databases.oracle.OracleDatabase;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.util.CollectionUtil;
-
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** The Table Source ITCase for {@link OracleDialect}. */
-class OracleTableSourceITCase extends AbstractTestBase implements OracleDatabase {
-
- private static final String INPUT_TABLE = "oracle_test_table";
-
- private static StreamExecutionEnvironment env;
- private static TableEnvironment tEnv;
-
- @BeforeAll
- static void beforeAll() throws ClassNotFoundException, SQLException {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement statement = conn.createStatement()) {
- statement.executeUpdate(
- "CREATE TABLE "
- + INPUT_TABLE
- + " ("
- + "id INTEGER NOT NULL,"
- + "float_col FLOAT,"
- + "double_col DOUBLE PRECISION ,"
- + "decimal_col NUMBER(10, 4) NOT NULL,"
- + "binary_float_col BINARY_FLOAT NOT NULL,"
- + "binary_double_col BINARY_DOUBLE NOT NULL,"
- + "char_col CHAR NOT NULL,"
- + "nchar_col NCHAR(3) NOT NULL,"
- + "varchar2_col VARCHAR2(30) NOT NULL,"
- + "date_col DATE NOT NULL,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "clob_col CLOB,"
- + "blob_col BLOB"
- + ")");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "1, 1.12345, 2.12345678790, 100.1234, 1.175E-10, 1.79769E+40, 'a', 'abc', 'abcdef', "
- + "TO_DATE('1997-01-01','yyyy-mm-dd'),TIMESTAMP '2020-01-01 15:35:00.123456',"
- + " TIMESTAMP '2020-01-01 15:35:00.123456789', 'Hello World', hextoraw('453d7a34'))");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "2, 1.12345, 2.12345678790, 101.1234, -1.175E-10, -1.79769E+40, 'a', 'abc', 'abcdef', "
- + "TO_DATE('1997-01-02','yyyy-mm-dd'), TIMESTAMP '2020-01-01 15:36:01.123456', "
- + "TIMESTAMP '2020-01-01 15:36:01.123456789', 'Hey Leonard', hextoraw('453d7a34'))");
- }
- }
-
- @AfterAll
- static void afterAll() throws Exception {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement statement = conn.createStatement()) {
- statement.executeUpdate("DROP TABLE " + INPUT_TABLE);
- }
- }
-
- @BeforeEach
- void before() throws Exception {
- env = StreamExecutionEnvironment.getExecutionEnvironment();
- tEnv = StreamTableEnvironment.create(env);
- }
-
- @Test
- void testJdbcSource() throws Exception {
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "("
- + "id BIGINT,"
- + "float_col DECIMAL(6, 5),"
- + "double_col DECIMAL(11, 10),"
- + "decimal_col DECIMAL(10, 4),"
- + "binary_float_col FLOAT,"
- + "binary_double_col DOUBLE,"
- + "char_col CHAR(1),"
- + "nchar_col VARCHAR(3),"
- + "varchar2_col VARCHAR(30),"
- + "date_col DATE,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "clob_col STRING,"
- + "blob_col BYTES"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "'"
- + ")");
-
- Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE).collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- List<String> expected =
- Stream.of(
- "+I[1, 1.12345, 2.1234567879, 100.1234, 1.175E-10, 1.79769E40, a, abc, abcdef, 1997-01-01, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, Hello World, [69, 61, 122, 52]]",
- "+I[2, 1.12345, 2.1234567879, 101.1234, -1.175E-10, -1.79769E40, a, abc, abcdef, 1997-01-02, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, Hey Leonard, [69, 61, 122, 52]]")
- .sorted()
- .collect(Collectors.toList());
- assertThat(result).isEqualTo(expected);
- }
-
- @Test
- void testProject() throws Exception {
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "binary_float_col FLOAT,"
- + "binary_double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "',"
- + " 'scan.partition.column'='id',"
- + " 'scan.partition.num'='2',"
- + " 'scan.partition.lower-bound'='0',"
- + " 'scan.partition.upper-bound'='100'"
- + ")");
-
- Iterator<Row> collected =
- tEnv.executeSql("SELECT id,timestamp6_col,decimal_col FROM " + INPUT_TABLE)
- .collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- List<String> expected =
- Stream.of(
- "+I[1, 2020-01-01T15:35:00.123456, 100.1234]",
- "+I[2, 2020-01-01T15:36:01.123456, 101.1234]")
- .sorted()
- .collect(Collectors.toList());
- assertThat(result).isEqualTo(expected);
- }
-
- @Test
- void testLimit() throws Exception {
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "(\n"
- + "id BIGINT,\n"
- + "timestamp6_col TIMESTAMP(6),\n"
- + "timestamp9_col TIMESTAMP(9),\n"
- + "binary_float_col FLOAT,\n"
- + "binary_double_col DOUBLE,\n"
- + "decimal_col DECIMAL(10, 4)\n"
- + ") WITH (\n"
- + " 'connector'='jdbc',\n"
- + " 'url'='"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',\n"
- + " 'table-name'='"
- + INPUT_TABLE
- + "',\n"
- + " 'scan.partition.column'='id',\n"
- + " 'scan.partition.num'='2',\n"
- + " 'scan.partition.lower-bound'='1',\n"
- + " 'scan.partition.upper-bound'='2'\n"
- + ")");
-
- Iterator<Row> collected =
- tEnv.executeSql("SELECT * FROM " + INPUT_TABLE + " LIMIT 1").collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
-
- Set<String> expected = new HashSet<>();
- expected.add(
- "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 1.175E-10, 1.79769E40, 100.1234]");
- expected.add(
- "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, -1.175E-10, -1.79769E40, 101.1234]");
- assertThat(result).hasSize(1);
- assertThat(expected)
- .as("The actual output is not a subset of the expected set.")
- .containsAll(result);
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresExactlyOnceSinkE2eTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresExactlyOnceSinkE2eTest.java
deleted file mode 100644
index 609523a..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/postgres/PostgresExactlyOnceSinkE2eTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-package org.apache.flink.connector.jdbc.dialect.postgres;
-
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.postgres.PostgresDatabase;
-import org.apache.flink.connector.jdbc.databases.postgres.PostgresMetadata;
-import org.apache.flink.connector.jdbc.xa.JdbcExactlyOnceSinkE2eTest;
-import org.apache.flink.util.function.SerializableSupplier;
-
-import org.postgresql.xa.PGXADataSource;
-
-import javax.sql.XADataSource;
-
-/**
- * A simple end-to-end test for {@link JdbcExactlyOnceSinkE2eTest}. Check for issues with suspending
- * connections (requires pooling) and honoring limits (properly closing connections).
- */
-public class PostgresExactlyOnceSinkE2eTest extends JdbcExactlyOnceSinkE2eTest
- implements PostgresDatabase {
-
- @Override
- public DatabaseMetadata getMetadata() {
- return new PostgresMetadata(CONTAINER, true);
- }
-
- @Override
- public SerializableSupplier<XADataSource> getDataSourceSupplier() {
- return () -> {
- PGXADataSource xaDataSource = new PGXADataSource();
- xaDataSource.setUrl(CONTAINER.getJdbcUrl());
- xaDataSource.setUser(CONTAINER.getUsername());
- xaDataSource.setPassword(CONTAINER.getPassword());
- return xaDataSource;
- };
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSinkITCase.java
deleted file mode 100644
index 18b8ca0..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/dialect/sqlserver/SqlServerTableSinkITCase.java
+++ /dev/null
@@ -1,497 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.dialect.sqlserver;
-
-import org.apache.flink.api.java.tuple.Tuple4;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.jdbc.databases.sqlserver.SqlServerDatabase;
-import org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction;
-import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.streaming.api.datastream.DataStream;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
-import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.sink.DynamicTableSink;
-import org.apache.flink.table.connector.sink.SinkFunctionProvider;
-import org.apache.flink.table.data.GenericRowData;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.table.planner.runtime.utils.TestData;
-import org.apache.flink.table.runtime.connector.sink.SinkRuntimeProviderContext;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
-
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Test;
-
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.apache.flink.connector.jdbc.internal.JdbcTableOutputFormatTest.check;
-import static org.apache.flink.table.api.Expressions.$;
-import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
-
-/** The Table Sink ITCase for {@link SqlServerDialect}. */
-class SqlServerTableSinkITCase extends AbstractTestBase implements SqlServerDatabase {
-
- public static final String OUTPUT_TABLE1 = "dynamicSinkForUpsert";
- public static final String OUTPUT_TABLE2 = "dynamicSinkForAppend";
- public static final String OUTPUT_TABLE3 = "dynamicSinkForBatch";
- public static final String OUTPUT_TABLE4 = "REAL_TABLE";
- public static final String OUTPUT_TABLE5 = "checkpointTable";
- public static final String USER_TABLE = "USER_TABLE";
-
- @BeforeAll
- static void beforeAll() throws ClassNotFoundException, SQLException {
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement stat = conn.createStatement()) {
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE1
- + " ("
- + "cnt FLOAT DEFAULT 0 NOT NULL,"
- + "lencnt FLOAT DEFAULT 0 NOT NULL,"
- + "cTag INT DEFAULT 0 NOT NULL,"
- + "ts DATETIME2,"
- + "CONSTRAINT PK1 PRIMARY KEY CLUSTERED (cnt, cTag))");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE2
- + " ("
- + "id INT DEFAULT 0 NOT NULL,"
- + "num INT DEFAULT 0 NOT NULL,"
- + "ts DATETIME2)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE3
- + " ("
- + "NAME VARCHAR(20) NOT NULL,"
- + "SCORE INT DEFAULT 0 NOT NULL)");
-
- stat.executeUpdate("CREATE TABLE " + OUTPUT_TABLE4 + " (real_data REAL)");
-
- stat.executeUpdate(
- "CREATE TABLE " + OUTPUT_TABLE5 + " (" + "id BIGINT DEFAULT 0 NOT NULL)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + USER_TABLE
- + " ("
- + "user_id VARCHAR(20) NOT NULL,"
- + "user_name VARCHAR(20) NOT NULL,"
- + "email VARCHAR(255),"
- + "balance DECIMAL(18,2),"
- + "balance2 DECIMAL(18,2),"
- + "CONSTRAINT PK2 PRIMARY KEY CLUSTERED (user_id))");
- }
- }
-
- @AfterAll
- static void afterAll() throws Exception {
- TestValuesTableFactory.clearAllData();
- Class.forName(CONTAINER.getDriverClassName());
- try (Connection conn =
- DriverManager.getConnection(
- CONTAINER.getJdbcUrl(),
- CONTAINER.getUsername(),
- CONTAINER.getPassword());
- Statement stat = conn.createStatement()) {
- stat.execute("DROP TABLE " + OUTPUT_TABLE1);
- stat.execute("DROP TABLE " + OUTPUT_TABLE2);
- stat.execute("DROP TABLE " + OUTPUT_TABLE3);
- stat.execute("DROP TABLE " + OUTPUT_TABLE4);
- stat.execute("DROP TABLE " + OUTPUT_TABLE5);
- stat.execute("DROP TABLE " + USER_TABLE);
- }
- }
-
- public static DataStream<Tuple4<Integer, Long, String, Timestamp>> get4TupleDataStream(
- StreamExecutionEnvironment env) {
- List<Tuple4<Integer, Long, String, Timestamp>> data = new ArrayList<>();
- data.add(new Tuple4<>(1, 1L, "Hi", Timestamp.valueOf("1970-01-01 00:00:00.001")));
- data.add(new Tuple4<>(2, 2L, "Hello", Timestamp.valueOf("1970-01-01 00:00:00.002")));
- data.add(new Tuple4<>(3, 2L, "Hello world", Timestamp.valueOf("1970-01-01 00:00:00.003")));
- data.add(
- new Tuple4<>(
- 4,
- 3L,
- "Hello world, how are you?",
- Timestamp.valueOf("1970-01-01 00:00:00.004")));
- data.add(new Tuple4<>(5, 3L, "I am fine.", Timestamp.valueOf("1970-01-01 00:00:00.005")));
- data.add(
- new Tuple4<>(
- 6, 3L, "Luke Skywalker", Timestamp.valueOf("1970-01-01 00:00:00.006")));
- data.add(new Tuple4<>(7, 4L, "Comment#1", Timestamp.valueOf("1970-01-01 00:00:00.007")));
- data.add(new Tuple4<>(8, 4L, "Comment#2", Timestamp.valueOf("1970-01-01 00:00:00.008")));
- data.add(new Tuple4<>(9, 4L, "Comment#3", Timestamp.valueOf("1970-01-01 00:00:00.009")));
- data.add(new Tuple4<>(10, 4L, "Comment#4", Timestamp.valueOf("1970-01-01 00:00:00.010")));
- data.add(new Tuple4<>(11, 5L, "Comment#5", Timestamp.valueOf("1970-01-01 00:00:00.011")));
- data.add(new Tuple4<>(12, 5L, "Comment#6", Timestamp.valueOf("1970-01-01 00:00:00.012")));
- data.add(new Tuple4<>(13, 5L, "Comment#7", Timestamp.valueOf("1970-01-01 00:00:00.013")));
- data.add(new Tuple4<>(14, 5L, "Comment#8", Timestamp.valueOf("1970-01-01 00:00:00.014")));
- data.add(new Tuple4<>(15, 5L, "Comment#9", Timestamp.valueOf("1970-01-01 00:00:00.015")));
- data.add(new Tuple4<>(16, 6L, "Comment#10", Timestamp.valueOf("1970-01-01 00:00:00.016")));
- data.add(new Tuple4<>(17, 6L, "Comment#11", Timestamp.valueOf("1970-01-01 00:00:00.017")));
- data.add(new Tuple4<>(18, 6L, "Comment#12", Timestamp.valueOf("1970-01-01 00:00:00.018")));
- data.add(new Tuple4<>(19, 6L, "Comment#13", Timestamp.valueOf("1970-01-01 00:00:00.019")));
- data.add(new Tuple4<>(20, 6L, "Comment#14", Timestamp.valueOf("1970-01-01 00:00:00.020")));
- data.add(new Tuple4<>(21, 6L, "Comment#15", Timestamp.valueOf("1970-01-01 00:00:00.021")));
-
- Collections.shuffle(data);
- return env.fromCollection(data);
- }
-
- @Test
- void testReal() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- StreamTableEnvironment tEnv =
- StreamTableEnvironment.create(env, EnvironmentSettings.inStreamingMode());
-
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " real_data float"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrl()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE4
- + "',"
- + " 'username'='"
- + getMetadata().getUsername()
- + "',"
- + " 'password'='"
- + getMetadata().getPassword()
- + "'"
- + ")");
-
- tEnv.executeSql("INSERT INTO upsertSink SELECT CAST(1.1 as FLOAT)").await();
- check(
- new Row[] {Row.of(1.1f)},
- getMetadata().getJdbcUrlWithCredentials(),
- "REAL_TABLE",
- new String[] {"real_data"});
- }
-
- @Test
- void testUpsert() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
-
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env)
- .assignTimestampsAndWatermarks(
- new AscendingTimestampExtractor<
- Tuple4<Integer, Long, String, Timestamp>>() {
- @Override
- public long extractAscendingTimestamp(
- Tuple4<Integer, Long, String, Timestamp>
- element) {
- return element.f0;
- }
- }),
- $("id"),
- $("num"),
- $("text"),
- $("ts"));
-
- tEnv.createTemporaryView("T", t);
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " cnt DECIMAL(18,2),"
- + " lencnt DECIMAL(18,2),"
- + " cTag INT,"
- + " ts TIMESTAMP(3),"
- + " PRIMARY KEY (cnt, cTag) NOT ENFORCED"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrl()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE1
- + "',"
- + " 'username'='"
- + getMetadata().getUsername()
- + "',"
- + " 'password'='"
- + getMetadata().getPassword()
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0',"
- + " 'sink.max-retries' = '0'"
- + ")");
-
- tEnv.executeSql(
- "INSERT INTO upsertSink \n"
- + "SELECT cnt, COUNT(len) AS lencnt, cTag, MAX(ts) AS ts\n"
- + "FROM (\n"
- + " SELECT len, COUNT(id) as cnt, cTag, MAX(ts) AS ts\n"
- + " FROM (SELECT id, CHAR_LENGTH(text) AS len, (CASE WHEN id > 0 THEN 1 ELSE 0 END) cTag, ts FROM T)\n"
- + " GROUP BY len, cTag\n"
- + ")\n"
- + "GROUP BY cnt, cTag")
- .await();
- check(
- new Row[] {
- Row.of(1.0, 5.0, 1, Timestamp.valueOf("1970-01-01 00:00:00.006")),
- Row.of(7.0, 1.0, 1, Timestamp.valueOf("1970-01-01 00:00:00.021")),
- Row.of(9.0, 1.0, 1, Timestamp.valueOf("1970-01-01 00:00:00.015"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE1,
- new String[] {"cnt", "lencnt", "cTag", "ts"});
- }
-
- @Test
- void testAppend() throws Exception {
- StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
- env.getConfig().enableObjectReuse();
- env.getConfig().setParallelism(1);
- StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
-
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env), $("id"), $("num"), $("text"), $("ts"));
-
- tEnv.registerTable("T", t);
-
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " id INT,"
- + " num BIGINT,"
- + " ts TIMESTAMP(3)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrl()
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE2
- + "',"
- + " 'username'='"
- + getMetadata().getUsername()
- + "',"
- + " 'password'='"
- + getMetadata().getPassword()
- + "'"
- + ")");
-
- tEnv.executeSql("INSERT INTO upsertSink SELECT id, num, ts FROM T WHERE id IN (2, 10, 20)")
- .await();
- check(
- new Row[] {
- Row.of(2, 2, Timestamp.valueOf("1970-01-01 00:00:00.002")),
- Row.of(10, 4, Timestamp.valueOf("1970-01-01 00:00:00.01")),
- Row.of(20, 6, Timestamp.valueOf("1970-01-01 00:00:00.02"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE2,
- new String[] {"id", "num", "ts"});
- }
-
- @Test
- void testBatchSink() throws Exception {
- TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
-
- tEnv.executeSql(
- "CREATE TABLE USER_RESULT("
- + "NAME VARCHAR,"
- + "SCORE INT"
- + ") WITH ( "
- + "'connector' = 'jdbc',"
- + "'url'='"
- + getMetadata().getJdbcUrl()
- + "',"
- + "'table-name' = '"
- + OUTPUT_TABLE3
- + "',"
- + " 'username'='"
- + getMetadata().getUsername()
- + "',"
- + " 'password'='"
- + getMetadata().getPassword()
- + "',"
- + "'sink.buffer-flush.max-rows' = '2',"
- + "'sink.buffer-flush.interval' = '300ms',"
- + "'sink.max-retries' = '4'"
- + ")");
-
- TableResult tableResult =
- tEnv.executeSql(
- "INSERT INTO USER_RESULT\n"
- + "SELECT user_name, score "
- + "FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), "
- + "(42, 'Kim'), (1, 'Bob')) "
- + "AS UserCountTable(score, user_name)");
- tableResult.await();
-
- check(
- new Row[] {
- Row.of("Bob", 1),
- Row.of("Tom", 22),
- Row.of("Kim", 42),
- Row.of("Kim", 42),
- Row.of("Bob", 1)
- },
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE3,
- new String[] {"NAME", "SCORE"});
- }
-
- @Test
- void testReadingFromChangelogSource() throws Exception {
- TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
- String dataId = TestValuesTableFactory.registerData(TestData.userChangelog());
- tEnv.executeSql(
- "CREATE TABLE user_logs (\n"
- + " user_id STRING,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,2),\n"
- + " balance2 AS balance * 2\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'data-id' = '"
- + dataId
- + "',\n"
- + " 'changelog-mode' = 'I,UA,UB,D'\n"
- + ")");
- tEnv.executeSql(
- "CREATE TABLE user_sink (\n"
- + " user_id STRING PRIMARY KEY NOT ENFORCED,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,3),\n"
- + " balance2 DECIMAL(18,3)\n"
- + ") WITH (\n"
- + " 'connector' = 'jdbc',"
- + " 'url'='"
- + getMetadata().getJdbcUrl()
- + "',"
- + " 'table-name' = '"
- + USER_TABLE
- + "',"
- + " 'username'='"
- + getMetadata().getUsername()
- + "',"
- + " 'password'='"
- + getMetadata().getPassword()
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0'"
- + // disable async flush
- ")");
- tEnv.executeSql("INSERT INTO user_sink SELECT * FROM user_logs").await();
-
- check(
- new Row[] {
- Row.of(
- "user1",
- "Tom",
- "tom123@gmail.com",
- new BigDecimal("8.10"),
- new BigDecimal("16.20")),
- Row.of(
- "user3",
- "Bailey",
- "bailey@qq.com",
- new BigDecimal("9.99"),
- new BigDecimal("19.98")),
- Row.of(
- "user4",
- "Tina",
- "tina@gmail.com",
- new BigDecimal("11.30"),
- new BigDecimal("22.60"))
- },
- getMetadata().getJdbcUrlWithCredentials(),
- USER_TABLE,
- new String[] {"user_id", "user_name", "email", "balance", "balance2"});
- }
-
- @Test
- void testFlushBufferWhenCheckpoint() throws Exception {
- Map<String, String> options = new HashMap<>();
- options.put("connector", "jdbc");
- options.put("url", getMetadata().getJdbcUrl());
- options.put("table-name", OUTPUT_TABLE5);
- options.put("sink.buffer-flush.interval", "0");
- options.put("username", getMetadata().getUsername());
- options.put("password", getMetadata().getPassword());
-
- ResolvedSchema schema =
- ResolvedSchema.of(Column.physical("id", DataTypes.BIGINT().notNull()));
-
- DynamicTableSink tableSink = createTableSink(schema, options);
-
- SinkRuntimeProviderContext context = new SinkRuntimeProviderContext(false);
- SinkFunctionProvider sinkProvider =
- (SinkFunctionProvider) tableSink.getSinkRuntimeProvider(context);
- GenericJdbcSinkFunction<RowData> sinkFunction =
- (GenericJdbcSinkFunction<RowData>) sinkProvider.createSinkFunction();
- sinkFunction.setRuntimeContext(new MockStreamingRuntimeContext(true, 1, 0));
- sinkFunction.open(new Configuration());
- sinkFunction.invoke(GenericRowData.of(1L), SinkContextUtil.forTimestamp(1));
- sinkFunction.invoke(GenericRowData.of(2L), SinkContextUtil.forTimestamp(1));
-
- check(
- new Row[] {},
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE5,
- new String[] {"id"});
- sinkFunction.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
- check(
- new Row[] {Row.of(1L), Row.of(2L)},
- getMetadata().getJdbcUrlWithCredentials(),
- OUTPUT_TABLE5,
- new String[] {"id"});
- sinkFunction.close();
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcAppendOnlyWriterTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcAppendOnlyWriterTest.java
index 5e93b01..3bd08ea 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcAppendOnlyWriterTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcAppendOnlyWriterTest.java
@@ -22,7 +22,6 @@
import org.apache.flink.api.common.functions.RuntimeContext;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.jdbc.JdbcTestBase;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
import org.apache.flink.connector.jdbc.dialect.JdbcDialectLoader;
import org.apache.flink.connector.jdbc.internal.JdbcOutputFormat;
import org.apache.flink.connector.jdbc.internal.options.InternalJdbcConnectionOptions;
@@ -37,7 +36,6 @@
import java.sql.Statement;
import static org.apache.flink.connector.jdbc.JdbcDataTestBase.toRow;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.OUTPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.TEST_DATA;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.TestEntry;
@@ -116,9 +114,4 @@
stat.execute("DELETE FROM " + OUTPUT_TABLE);
}
}
-
- @Override
- public DatabaseMetadata getMetadata() {
- return DERBY_EBOOKSHOP_DB;
- }
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSinkITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSinkITCase.java
index 7b4d36d..1973a18 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSinkITCase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcDynamicTableSinkITCase.java
@@ -18,14 +18,14 @@
package org.apache.flink.connector.jdbc.table;
-import org.apache.flink.api.java.tuple.Tuple4;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.jdbc.internal.GenericJdbcSinkFunction;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
import org.apache.flink.runtime.state.StateSnapshotContextSynchronousImpl;
-import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.sink.SinkContextUtil;
-import org.apache.flink.streaming.api.functions.timestamps.AscendingTimestampExtractor;
import org.apache.flink.streaming.util.MockStreamingRuntimeContext;
import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.EnvironmentSettings;
@@ -33,7 +33,6 @@
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableResult;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.connector.sink.SinkFunctionProvider;
@@ -45,132 +44,155 @@
import org.apache.flink.test.util.AbstractTestBase;
import org.apache.flink.types.Row;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
import java.sql.Timestamp;
-import java.util.ArrayList;
-import java.util.Collections;
+import java.util.Arrays;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
-import static org.apache.flink.connector.jdbc.internal.JdbcTableOutputFormatTest.check;
-import static org.apache.flink.table.api.Expressions.$;
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
import static org.apache.flink.table.factories.utils.FactoryMocks.createTableSink;
+import static org.assertj.core.api.Assertions.assertThat;
/** The ITCase for {@link JdbcDynamicTableSink}. */
-class JdbcDynamicTableSinkITCase extends AbstractTestBase {
+public abstract class JdbcDynamicTableSinkITCase extends AbstractTestBase implements DatabaseTest {
- public static final String DB_URL = DERBY_EBOOKSHOP_DB.getJdbcUrl();
- public static final String OUTPUT_TABLE1 = "dynamicSinkForUpsert";
- public static final String OUTPUT_TABLE2 = "dynamicSinkForAppend";
- public static final String OUTPUT_TABLE3 = "dynamicSinkForBatch";
- public static final String OUTPUT_TABLE4 = "REAL_TABLE";
- public static final String OUTPUT_TABLE5 = "checkpointTable";
- public static final String USER_TABLE = "USER_TABLE";
+ private final TableRow upsertOutputTable = createUpsertOutputTable();
+ private final TableRow appendOutputTable = createAppendOutputTable();
+ private final TableRow batchOutputTable = createBatchOutputTable();
+ private final TableRow realOutputTable = createRealOutputTable();
+ private final TableRow checkpointOutputTable = createCheckpointOutputTable();
+ private final TableRow userOutputTable = createUserOutputTable();
- @BeforeAll
- static void beforeAll() throws SQLException {
- try (Connection conn = DERBY_EBOOKSHOP_DB.getConnection();
- Statement stat = conn.createStatement()) {
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE1
- + " ("
- + "cnt BIGINT NOT NULL DEFAULT 0,"
- + "lencnt BIGINT NOT NULL DEFAULT 0,"
- + "cTag INT NOT NULL DEFAULT 0,"
- + "ts TIMESTAMP,"
- + "PRIMARY KEY (cnt, cTag))");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE2
- + " ("
- + "id INT NOT NULL DEFAULT 0,"
- + "num BIGINT NOT NULL DEFAULT 0,"
- + "ts TIMESTAMP)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + OUTPUT_TABLE3
- + " ("
- + "NAME VARCHAR(20) NOT NULL,"
- + "SCORE BIGINT NOT NULL DEFAULT 0)");
-
- stat.executeUpdate("CREATE TABLE " + OUTPUT_TABLE4 + " (real_data REAL)");
-
- stat.executeUpdate(
- "CREATE TABLE " + OUTPUT_TABLE5 + " (" + "id BIGINT NOT NULL DEFAULT 0)");
-
- stat.executeUpdate(
- "CREATE TABLE "
- + USER_TABLE
- + " ("
- + "user_id VARCHAR(20) NOT NULL,"
- + "user_name VARCHAR(20) NOT NULL,"
- + "email VARCHAR(255),"
- + "balance DECIMAL(18,2),"
- + "balance2 DECIMAL(18,2),"
- + "PRIMARY KEY (user_id))");
- }
+ protected TableRow createUpsertOutputTable() {
+ return tableRow(
+ "dynamicSinkForUpsert",
+ pkField("cnt", DataTypes.BIGINT().notNull()),
+ field("lencnt", DataTypes.BIGINT().notNull()),
+ pkField("cTag", DataTypes.INT().notNull()),
+ field("ts", dbType("TIMESTAMP"), DataTypes.TIMESTAMP()));
}
- @AfterAll
- static void afterAll() throws Exception {
+ protected TableRow createAppendOutputTable() {
+ return tableRow(
+ "dynamicSinkForAppend",
+ field("id", DataTypes.INT().notNull()),
+ field("num", DataTypes.BIGINT().notNull()),
+ field("ts", dbType("TIMESTAMP"), DataTypes.TIMESTAMP()));
+ }
+
+ protected TableRow createBatchOutputTable() {
+ return tableRow(
+ "dynamicSinkForBatch",
+ field("NAME", DataTypes.VARCHAR(20).notNull()),
+ field("SCORE", DataTypes.BIGINT().notNull()));
+ }
+
+ protected TableRow createRealOutputTable() {
+ return tableRow("REAL_TABLE", field("real_data", dbType("REAL"), DataTypes.FLOAT()));
+ }
+
+ protected TableRow createCheckpointOutputTable() {
+ return tableRow("checkpointTable", field("id", DataTypes.BIGINT().notNull()));
+ }
+
+ protected TableRow createUserOutputTable() {
+ return tableRow(
+ "USER_TABLE",
+ pkField("user_id", DataTypes.VARCHAR(20).notNull()),
+ field("user_name", DataTypes.VARCHAR(20).notNull()),
+ field("email", DataTypes.VARCHAR(255)),
+ field("balance", DataTypes.DECIMAL(18, 2)),
+ field("balance2", DataTypes.DECIMAL(18, 2)));
+ }
+
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Arrays.asList(
+ upsertOutputTable,
+ appendOutputTable,
+ batchOutputTable,
+ realOutputTable,
+ checkpointOutputTable,
+ userOutputTable);
+ }
+
+ @AfterEach
+ void afterEach() {
TestValuesTableFactory.clearAllData();
- try (Connection conn = DERBY_EBOOKSHOP_DB.getConnection();
- Statement stat = conn.createStatement()) {
- stat.execute("DROP TABLE " + OUTPUT_TABLE1);
- stat.execute("DROP TABLE " + OUTPUT_TABLE2);
- stat.execute("DROP TABLE " + OUTPUT_TABLE3);
- stat.execute("DROP TABLE " + OUTPUT_TABLE4);
- stat.execute("DROP TABLE " + OUTPUT_TABLE5);
- stat.execute("DROP TABLE " + USER_TABLE);
- }
}
- public static DataStream<Tuple4<Integer, Long, String, Timestamp>> get4TupleDataStream(
- StreamExecutionEnvironment env) {
- List<Tuple4<Integer, Long, String, Timestamp>> data = new ArrayList<>();
- data.add(new Tuple4<>(1, 1L, "Hi", Timestamp.valueOf("1970-01-01 00:00:00.001")));
- data.add(new Tuple4<>(2, 2L, "Hello", Timestamp.valueOf("1970-01-01 00:00:00.002")));
- data.add(new Tuple4<>(3, 2L, "Hello world", Timestamp.valueOf("1970-01-01 00:00:00.003")));
- data.add(
- new Tuple4<>(
+ protected List<Row> testUserData() {
+ return Arrays.asList(
+ Row.of(
+ "user1",
+ "Tom",
+ "tom123@gmail.com",
+ new BigDecimal("8.10"),
+ new BigDecimal("16.20")),
+ Row.of(
+ "user3",
+ "Bailey",
+ "bailey@qq.com",
+ new BigDecimal("9.99"),
+ new BigDecimal("19.98")),
+ Row.of(
+ "user4",
+ "Tina",
+ "tina@gmail.com",
+ new BigDecimal("11.30"),
+ new BigDecimal("22.60")));
+ }
+
+ protected List<Row> testData() {
+ return Arrays.asList(
+ Row.of(1, 1L, "Hi", Timestamp.valueOf("1970-01-01 00:00:00.001")),
+ Row.of(2, 2L, "Hello", Timestamp.valueOf("1970-01-01 00:00:00.002")),
+ Row.of(3, 2L, "Hello world", Timestamp.valueOf("1970-01-01 00:00:00.003")),
+ Row.of(
4,
3L,
"Hello world, how are you?",
- Timestamp.valueOf("1970-01-01 00:00:00.004")));
- data.add(new Tuple4<>(5, 3L, "I am fine.", Timestamp.valueOf("1970-01-01 00:00:00.005")));
- data.add(
- new Tuple4<>(
- 6, 3L, "Luke Skywalker", Timestamp.valueOf("1970-01-01 00:00:00.006")));
- data.add(new Tuple4<>(7, 4L, "Comment#1", Timestamp.valueOf("1970-01-01 00:00:00.007")));
- data.add(new Tuple4<>(8, 4L, "Comment#2", Timestamp.valueOf("1970-01-01 00:00:00.008")));
- data.add(new Tuple4<>(9, 4L, "Comment#3", Timestamp.valueOf("1970-01-01 00:00:00.009")));
- data.add(new Tuple4<>(10, 4L, "Comment#4", Timestamp.valueOf("1970-01-01 00:00:00.010")));
- data.add(new Tuple4<>(11, 5L, "Comment#5", Timestamp.valueOf("1970-01-01 00:00:00.011")));
- data.add(new Tuple4<>(12, 5L, "Comment#6", Timestamp.valueOf("1970-01-01 00:00:00.012")));
- data.add(new Tuple4<>(13, 5L, "Comment#7", Timestamp.valueOf("1970-01-01 00:00:00.013")));
- data.add(new Tuple4<>(14, 5L, "Comment#8", Timestamp.valueOf("1970-01-01 00:00:00.014")));
- data.add(new Tuple4<>(15, 5L, "Comment#9", Timestamp.valueOf("1970-01-01 00:00:00.015")));
- data.add(new Tuple4<>(16, 6L, "Comment#10", Timestamp.valueOf("1970-01-01 00:00:00.016")));
- data.add(new Tuple4<>(17, 6L, "Comment#11", Timestamp.valueOf("1970-01-01 00:00:00.017")));
- data.add(new Tuple4<>(18, 6L, "Comment#12", Timestamp.valueOf("1970-01-01 00:00:00.018")));
- data.add(new Tuple4<>(19, 6L, "Comment#13", Timestamp.valueOf("1970-01-01 00:00:00.019")));
- data.add(new Tuple4<>(20, 6L, "Comment#14", Timestamp.valueOf("1970-01-01 00:00:00.020")));
- data.add(new Tuple4<>(21, 6L, "Comment#15", Timestamp.valueOf("1970-01-01 00:00:00.021")));
+ Timestamp.valueOf("1970-01-01 00:00:00.004")),
+ Row.of(5, 3L, "I am fine.", Timestamp.valueOf("1970-01-01 00:00:00.005")),
+ Row.of(6, 3L, "Luke Skywalker", Timestamp.valueOf("1970-01-01 00:00:00.006")),
+ Row.of(7, 4L, "Comment#1", Timestamp.valueOf("1970-01-01 00:00:00.007")),
+ Row.of(8, 4L, "Comment#2", Timestamp.valueOf("1970-01-01 00:00:00.008")),
+ Row.of(9, 4L, "Comment#3", Timestamp.valueOf("1970-01-01 00:00:00.009")),
+ Row.of(10, 4L, "Comment#4", Timestamp.valueOf("1970-01-01 00:00:00.010")),
+ Row.of(11, 5L, "Comment#5", Timestamp.valueOf("1970-01-01 00:00:00.011")),
+ Row.of(12, 5L, "Comment#6", Timestamp.valueOf("1970-01-01 00:00:00.012")),
+ Row.of(13, 5L, "Comment#7", Timestamp.valueOf("1970-01-01 00:00:00.013")),
+ Row.of(14, 5L, "Comment#8", Timestamp.valueOf("1970-01-01 00:00:00.014")),
+ Row.of(15, 5L, "Comment#9", Timestamp.valueOf("1970-01-01 00:00:00.015")),
+ Row.of(16, 6L, "Comment#10", Timestamp.valueOf("1970-01-01 00:00:00.016")),
+ Row.of(17, 6L, "Comment#11", Timestamp.valueOf("1970-01-01 00:00:00.017")),
+ Row.of(18, 6L, "Comment#12", Timestamp.valueOf("1970-01-01 00:00:00.018")),
+ Row.of(19, 6L, "Comment#13", Timestamp.valueOf("1970-01-01 00:00:00.019")),
+ Row.of(20, 6L, "Comment#14", Timestamp.valueOf("1970-01-01 00:00:00.020")),
+ Row.of(21, 6L, "Comment#15", Timestamp.valueOf("1970-01-01 00:00:00.021")));
+ }
- Collections.shuffle(data);
- return env.fromCollection(data);
+ protected Map<Integer, Row> testDataMap() {
+ return testData().stream()
+ .collect(Collectors.toMap(r -> r.getFieldAs(0), Function.identity()));
+ }
+
+ private void createTestDataTempView(StreamTableEnvironment tEnv, String viewName) {
+ Table table = tEnv.fromValues(testData()).as("id", "num", "text", "ts");
+
+ tEnv.createTemporaryView(viewName, table);
}
@Test
@@ -180,21 +202,13 @@
StreamTableEnvironment tEnv =
StreamTableEnvironment.create(env, EnvironmentSettings.inStreamingMode());
- tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " real_data float"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE4
- + "'"
- + ")");
+ String tableName = "realSink";
+ tEnv.executeSql(realOutputTable.getCreateQueryForFlink(getMetadata(), tableName));
- tEnv.executeSql("INSERT INTO upsertSink SELECT CAST(1.0 as FLOAT)").await();
- check(new Row[] {Row.of(1.0f)}, DB_URL, "REAL_TABLE", new String[] {"real_data"});
+ tEnv.executeSql(String.format("INSERT INTO %s SELECT CAST(1.0 as FLOAT)", tableName))
+ .await();
+
+ assertThat(realOutputTable.selectAllTable(getMetadata())).containsExactly(Row.of(1.0f));
}
@Test
@@ -203,64 +217,38 @@
env.getConfig().enableObjectReuse();
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env)
- .assignTimestampsAndWatermarks(
- new AscendingTimestampExtractor<
- Tuple4<Integer, Long, String, Timestamp>>() {
- @Override
- public long extractAscendingTimestamp(
- Tuple4<Integer, Long, String, Timestamp>
- element) {
- return element.f0;
- }
- }),
- $("id"),
- $("num"),
- $("text"),
- $("ts"));
+ String viewName = "testData";
+ createTestDataTempView(tEnv, viewName);
- tEnv.createTemporaryView("T", t);
+ String tableName = "upsertSink";
tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " cnt BIGINT,"
- + " lencnt BIGINT,"
- + " cTag INT,"
- + " ts TIMESTAMP(3),"
- + " PRIMARY KEY (cnt, cTag) NOT ENFORCED"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE1
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0',"
- + " 'sink.max-retries' = '0'"
- + ")");
+ upsertOutputTable.getCreateQueryForFlink(
+ getMetadata(),
+ tableName,
+ Arrays.asList(
+ "'sink.buffer-flush.max-rows' = '2'",
+ "'sink.buffer-flush.interval' = '0'",
+ "'sink.max-retries' = '0'")));
tEnv.executeSql(
- "INSERT INTO upsertSink \n"
- + "SELECT cnt, COUNT(len) AS lencnt, cTag, MAX(ts) AS ts\n"
- + "FROM (\n"
- + " SELECT len, COUNT(id) as cnt, cTag, MAX(ts) AS ts\n"
- + " FROM (SELECT id, CHAR_LENGTH(text) AS len, (CASE WHEN id > 0 THEN 1 ELSE 0 END) cTag, ts FROM T)\n"
- + " GROUP BY len, cTag\n"
- + ")\n"
- + "GROUP BY cnt, cTag")
+ String.format(
+ "INSERT INTO %s "
+ + " SELECT cnt, COUNT(len) AS lencnt, cTag, MAX(ts) AS ts "
+ + " FROM ( "
+ + " SELECT len, COUNT(id) as cnt, cTag, MAX(ts) AS ts "
+ + " FROM (SELECT id, CHAR_LENGTH(text) AS len, (CASE WHEN id > 0 THEN 1 ELSE 0 END) cTag, ts FROM %s) "
+ + " GROUP BY len, cTag "
+ + " ) "
+ + " GROUP BY cnt, cTag",
+ tableName, viewName))
.await();
- check(
- new Row[] {
- Row.of(1, 5, 1, Timestamp.valueOf("1970-01-01 00:00:00.006")),
- Row.of(7, 1, 1, Timestamp.valueOf("1970-01-01 00:00:00.021")),
- Row.of(9, 1, 1, Timestamp.valueOf("1970-01-01 00:00:00.015"))
- },
- DB_URL,
- OUTPUT_TABLE1,
- new String[] {"cnt", "lencnt", "cTag", "ts"});
+
+ Map<Integer, Row> mapTestData = testDataMap();
+ assertThat(upsertOutputTable.selectAllTable(getMetadata()))
+ .containsExactlyInAnyOrder(
+ Row.of(1L, 5L, 1, mapTestData.get(6).getField(3)),
+ Row.of(7L, 1L, 1, mapTestData.get(21).getField(3)),
+ Row.of(9L, 1L, 1, mapTestData.get(15).getField(3)));
}
@Test
@@ -270,158 +258,118 @@
env.getConfig().setParallelism(1);
StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
- Table t =
- tEnv.fromDataStream(
- get4TupleDataStream(env), $("id"), $("num"), $("text"), $("ts"));
+ String viewName = "testData";
+ createTestDataTempView(tEnv, viewName);
- tEnv.registerTable("T", t);
+ String tableName = "appendSink";
+ tEnv.executeSql(appendOutputTable.getCreateQueryForFlink(getMetadata(), tableName));
+ Set<Integer> searchIds = new HashSet<>(Arrays.asList(2, 10, 20));
tEnv.executeSql(
- "CREATE TABLE upsertSink ("
- + " id INT,"
- + " num BIGINT,"
- + " ts TIMESTAMP(3)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + OUTPUT_TABLE2
- + "'"
- + ")");
-
- tEnv.executeSql("INSERT INTO upsertSink SELECT id, num, ts FROM T WHERE id IN (2, 10, 20)")
+ String.format(
+ "INSERT INTO %s SELECT id, num, ts FROM %s WHERE id IN (%s)",
+ tableName,
+ viewName,
+ searchIds.stream()
+ .map(Object::toString)
+ .collect(Collectors.joining(","))))
.await();
- check(
- new Row[] {
- Row.of(2, 2, Timestamp.valueOf("1970-01-01 00:00:00.002")),
- Row.of(10, 4, Timestamp.valueOf("1970-01-01 00:00:00.01")),
- Row.of(20, 6, Timestamp.valueOf("1970-01-01 00:00:00.02"))
- },
- DB_URL,
- OUTPUT_TABLE2,
- new String[] {"id", "num", "ts"});
+
+ List<Row> tableRows = appendOutputTable.selectAllTable(getMetadata());
+ assertThat(tableRows.size()).isEqualTo(3);
+
+ Map<Integer, Row> mapTestData = testDataMap();
+ assertThat(tableRows)
+ .containsExactlyInAnyOrderElementsOf(
+ searchIds.stream()
+ .map(mapTestData::get)
+ .map(d -> Row.of(d.getField(0), d.getField(1), d.getField(3)))
+ .collect(Collectors.toList()));
}
@Test
void testBatchSink() throws Exception {
TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode());
+ String tableName = "batchSink";
tEnv.executeSql(
- "CREATE TABLE USER_RESULT("
- + "NAME VARCHAR,"
- + "SCORE BIGINT"
- + ") WITH ( "
- + "'connector' = 'jdbc',"
- + "'url'='"
- + DB_URL
- + "',"
- + "'table-name' = '"
- + OUTPUT_TABLE3
- + "',"
- + "'sink.buffer-flush.max-rows' = '2',"
- + "'sink.buffer-flush.interval' = '300ms',"
- + "'sink.max-retries' = '4'"
- + ")");
+ batchOutputTable.getCreateQueryForFlink(
+ getMetadata(),
+ tableName,
+ Arrays.asList(
+ "'sink.buffer-flush.max-rows' = '2'",
+ "'sink.buffer-flush.interval' = '300ms'",
+ "'sink.max-retries' = '4'")));
TableResult tableResult =
tEnv.executeSql(
- "INSERT INTO USER_RESULT\n"
- + "SELECT user_name, score "
- + "FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), "
- + "(42, 'Kim'), (1, 'Bob')) "
- + "AS UserCountTable(score, user_name)");
+ String.format(
+ "INSERT INTO %s "
+ + " SELECT user_name, score "
+ + " FROM (VALUES (1, 'Bob'), (22, 'Tom'), (42, 'Kim'), (42, 'Kim'), (1, 'Bob')) "
+ + " AS UserCountTable(score, user_name) ",
+ tableName));
tableResult.await();
- check(
- new Row[] {
- Row.of("Bob", 1),
- Row.of("Tom", 22),
- Row.of("Kim", 42),
- Row.of("Kim", 42),
- Row.of("Bob", 1)
- },
- DB_URL,
- OUTPUT_TABLE3,
- new String[] {"NAME", "SCORE"});
+ assertThat(batchOutputTable.selectAllTable(getMetadata()))
+ .containsExactlyInAnyOrder(
+ Row.of("Bob", 1L),
+ Row.of("Tom", 22L),
+ Row.of("Kim", 42L),
+ Row.of("Kim", 42L),
+ Row.of("Bob", 1L));
}
@Test
void testReadingFromChangelogSource() throws Exception {
TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().build());
String dataId = TestValuesTableFactory.registerData(TestData.userChangelog());
- tEnv.executeSql(
- "CREATE TABLE user_logs (\n"
- + " user_id STRING,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,2),\n"
- + " balance2 AS balance * 2\n"
- + ") WITH (\n"
- + " 'connector' = 'values',\n"
- + " 'data-id' = '"
- + dataId
- + "',\n"
- + " 'changelog-mode' = 'I,UA,UB,D'\n"
- + ")");
- tEnv.executeSql(
- "CREATE TABLE user_sink (\n"
- + " user_id STRING PRIMARY KEY NOT ENFORCED,\n"
- + " user_name STRING,\n"
- + " email STRING,\n"
- + " balance DECIMAL(18,2),\n"
- + " balance2 DECIMAL(18,2)\n"
- + ") WITH (\n"
- + " 'connector' = 'jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name' = '"
- + USER_TABLE
- + "',"
- + " 'sink.buffer-flush.max-rows' = '2',"
- + " 'sink.buffer-flush.interval' = '0'"
- + // disable async flush
- ")");
- tEnv.executeSql("INSERT INTO user_sink SELECT * FROM user_logs").await();
- check(
- new Row[] {
- Row.of(
- "user1",
- "Tom",
- "tom123@gmail.com",
- new BigDecimal("8.10"),
- new BigDecimal("16.20")),
- Row.of(
- "user3",
- "Bailey",
- "bailey@qq.com",
- new BigDecimal("9.99"),
- new BigDecimal("19.98")),
- Row.of(
- "user4",
- "Tina",
- "tina@gmail.com",
- new BigDecimal("11.30"),
- new BigDecimal("22.60"))
- },
- DB_URL,
- USER_TABLE,
- new String[] {"user_id", "user_name", "email", "balance", "balance2"});
+ String userTableLogs = "user_logs";
+ tEnv.executeSql(
+ String.format(
+ "CREATE TABLE %s ( "
+ + " user_id STRING, "
+ + " user_name STRING, "
+ + " email STRING, "
+ + " balance DECIMAL(18,2), "
+ + " balance2 AS balance * 2 "
+ + ") WITH ( "
+ + " 'connector' = 'values', "
+ + " 'data-id' = '%s', "
+ + " 'changelog-mode' = 'I,UA,UB,D' "
+ + ")",
+ userTableLogs, dataId));
+
+ String userTableSink = "user_sink";
+ tEnv.executeSql(
+ userOutputTable.getCreateQueryForFlink(
+ getMetadata(),
+ userTableSink,
+ Arrays.asList(
+ "'sink.buffer-flush.max-rows' = '2'",
+ "'sink.buffer-flush.interval' = '0'")));
+
+ tEnv.executeSql(
+ String.format(
+ "INSERT INTO %s SELECT * FROM %s", userTableSink, userTableLogs))
+ .await();
+
+ assertThat(userOutputTable.selectAllTable(getMetadata()))
+ .containsExactlyInAnyOrderElementsOf(testUserData());
}
@Test
void testFlushBufferWhenCheckpoint() throws Exception {
Map<String, String> options = new HashMap<>();
options.put("connector", "jdbc");
- options.put("url", DB_URL);
- options.put("table-name", OUTPUT_TABLE5);
+ options.put("url", getMetadata().getJdbcUrl());
+ options.put("username", getMetadata().getUsername());
+ options.put("password", getMetadata().getPassword());
+ options.put("table-name", checkpointOutputTable.getTableName());
options.put("sink.buffer-flush.interval", "0");
- ResolvedSchema schema =
- ResolvedSchema.of(Column.physical("id", DataTypes.BIGINT().notNull()));
+ ResolvedSchema schema = checkpointOutputTable.getTableResolvedSchema();
DynamicTableSink tableSink = createTableSink(schema, options);
@@ -435,9 +383,13 @@
sinkFunction.invoke(GenericRowData.of(1L), SinkContextUtil.forTimestamp(1));
sinkFunction.invoke(GenericRowData.of(2L), SinkContextUtil.forTimestamp(1));
- check(new Row[] {}, DB_URL, OUTPUT_TABLE5, new String[] {"id"});
+ assertThat(checkpointOutputTable.selectAllTable(getMetadata())).isEmpty();
+
sinkFunction.snapshotState(new StateSnapshotContextSynchronousImpl(1, 1));
- check(new Row[] {Row.of(1L), Row.of(2L)}, DB_URL, OUTPUT_TABLE5, new String[] {"id"});
+
+ assertThat(checkpointOutputTable.selectAllTable(getMetadata()))
+ .containsExactlyInAnyOrder(Row.of(1L), Row.of(2L));
+
sinkFunction.close();
}
}
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 17e1be0..3b1855e 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
@@ -19,12 +19,16 @@
package org.apache.flink.connector.jdbc.table;
import org.apache.flink.configuration.Configuration;
-import org.apache.flink.connector.jdbc.JdbcTestBase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.TableRow;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.DataTypes;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.table.connector.source.lookup.cache.LookupCache;
+import org.apache.flink.table.data.DecimalData;
import org.apache.flink.table.data.GenericRowData;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.TimestampData;
@@ -34,39 +38,33 @@
import org.apache.flink.table.test.lookup.cache.LookupCacheAssert;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.types.Row;
-import org.apache.flink.util.CloseableIterator;
import org.apache.flink.util.CollectionUtil;
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.RegisterExtension;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
+import java.math.BigDecimal;
import java.sql.Connection;
-import java.sql.DriverManager;
import java.sql.SQLException;
-import java.sql.Statement;
import java.time.LocalDateTime;
-import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
import java.util.List;
import java.util.Map;
-import java.util.Set;
import java.util.stream.Collectors;
-import java.util.stream.Stream;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.field;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
import static org.assertj.core.api.Assertions.assertThat;
/** ITCase for {@link JdbcDynamicTableSource}. */
-class JdbcDynamicTableSourceITCase {
+public abstract class JdbcDynamicTableSourceITCase implements DatabaseTest {
@RegisterExtension
static final MiniClusterExtension MINI_CLUSTER_RESOURCE =
@@ -76,360 +74,213 @@
.setConfiguration(new Configuration())
.build());
- public static final String DRIVER_CLASS = "org.apache.derby.jdbc.EmbeddedDriver";
- public static final String DB_URL = "jdbc:derby:memory:test";
- public static final String INPUT_TABLE = "jdbDynamicTableSource";
+ private final TableRow inputTable = createInputTable();
public static StreamExecutionEnvironment env;
public static TableEnvironment tEnv;
- @BeforeAll
- static void beforeAll() throws ClassNotFoundException, SQLException {
- System.setProperty(
- "derby.stream.error.field", JdbcTestBase.class.getCanonicalName() + ".DEV_NULL");
- Class.forName(DRIVER_CLASS);
-
- try (Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
- Statement statement = conn.createStatement()) {
- statement.executeUpdate(
- "CREATE TABLE "
- + INPUT_TABLE
- + " ("
- + "id BIGINT NOT NULL,"
- + "timestamp6_col TIMESTAMP, "
- + "timestamp9_col TIMESTAMP, "
- + "time_col TIME, "
- + "real_col FLOAT(23), "
- + // A precision of 23 or less makes FLOAT equivalent to REAL.
- "double_col FLOAT(24),"
- + // A precision of 24 or greater makes FLOAT equivalent to DOUBLE
- // PRECISION.
- "decimal_col DECIMAL(10, 4))");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "1, TIMESTAMP('2020-01-01 15:35:00.123456'), TIMESTAMP('2020-01-01 15:35:00.123456789'), "
- + "TIME('15:35:00'), 1.175E-37, 1.79769E+308, 100.1234)");
- statement.executeUpdate(
- "INSERT INTO "
- + INPUT_TABLE
- + " VALUES ("
- + "2, TIMESTAMP('2020-01-01 15:36:01.123456'), TIMESTAMP('2020-01-01 15:36:01.123456789'), "
- + "TIME('15:36:01'), -1.175E-37, -1.79769E+308, 101.1234)");
- }
+ protected TableRow createInputTable() {
+ return tableRow(
+ "jdbDynamicTableSource",
+ field("id", DataTypes.BIGINT().notNull()),
+ field("decimal_col", DataTypes.DECIMAL(10, 4)),
+ field("timestamp6_col", DataTypes.TIMESTAMP(6)));
}
- @AfterAll
- static void afterAll() throws Exception {
- Class.forName(DRIVER_CLASS);
- try (Connection conn = DriverManager.getConnection(DB_URL);
- Statement stat = conn.createStatement()) {
- stat.executeUpdate("DROP TABLE " + INPUT_TABLE);
- }
- StreamTestSink.clear();
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Collections.singletonList(inputTable);
+ }
+
+ protected List<Row> getTestData() {
+ return Arrays.asList(
+ Row.of(
+ 1L,
+ BigDecimal.valueOf(100.1234),
+ LocalDateTime.parse("2020-01-01T15:35:00.123456")),
+ Row.of(
+ 2L,
+ BigDecimal.valueOf(101.1234),
+ LocalDateTime.parse("2020-01-01T15:36:01.123456")));
}
@BeforeEach
- void before() throws Exception {
+ void beforeAll() throws SQLException {
+ try (Connection conn = getMetadata().getConnection()) {
+ inputTable.insertIntoTableValues(conn, getTestData());
+ }
env = StreamExecutionEnvironment.getExecutionEnvironment();
tEnv = StreamTableEnvironment.create(env);
}
- @Test
- void testJdbcSource() throws Exception {
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "time_col TIME,"
- + "real_col FLOAT,"
- + "double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "'"
- + ")");
-
- Iterator<Row> collected = tEnv.executeSql("SELECT * FROM " + INPUT_TABLE).collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- List<String> expected =
- Stream.of(
- "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 15:35, 1.175E-37, 1.79769E308, 100.1234]",
- "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, 15:36:01, -1.175E-37, -1.79769E308, 101.1234]")
- .sorted()
- .collect(Collectors.toList());
- assertThat(result).isEqualTo(expected);
+ @AfterEach
+ void afterEach() {
+ StreamTestSink.clear();
}
@Test
- void testProject() throws Exception {
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "time_col TIME,"
- + "real_col FLOAT,"
- + "double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "',"
- + " 'scan.partition.column'='id',"
- + " 'scan.partition.num'='2',"
- + " 'scan.partition.lower-bound'='0',"
- + " 'scan.partition.upper-bound'='100'"
- + ")");
+ void testJdbcSource() {
+ String testTable = "testTable";
+ tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable));
- Iterator<Row> collected =
- tEnv.executeSql("SELECT id,timestamp6_col,decimal_col FROM " + INPUT_TABLE)
- .collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- List<String> expected =
- Stream.of(
- "+I[1, 2020-01-01T15:35:00.123456, 100.1234]",
- "+I[2, 2020-01-01T15:36:01.123456, 101.1234]")
- .sorted()
- .collect(Collectors.toList());
- assertThat(result).isEqualTo(expected);
+ List<Row> collected = executeQuery("SELECT * FROM " + testTable);
+
+ assertThat(collected).containsExactlyInAnyOrderElementsOf(getTestData());
}
@Test
- void testLimit() throws Exception {
+ void testProject() {
+ String testTable = "testTable";
tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "(\n"
- + "id BIGINT,\n"
- + "timestamp6_col TIMESTAMP(6),\n"
- + "timestamp9_col TIMESTAMP(9),\n"
- + "time_col TIME,\n"
- + "real_col FLOAT,\n"
- + "double_col DOUBLE,\n"
- + "decimal_col DECIMAL(10, 4)\n"
- + ") WITH (\n"
- + " 'connector'='jdbc',\n"
- + " 'url'='"
- + DB_URL
- + "',\n"
- + " 'table-name'='"
- + INPUT_TABLE
- + "',\n"
- + " 'scan.partition.column'='id',\n"
- + " 'scan.partition.num'='2',\n"
- + " 'scan.partition.lower-bound'='1',\n"
- + " 'scan.partition.upper-bound'='2'\n"
- + ")");
+ inputTable.getCreateQueryForFlink(
+ getMetadata(),
+ testTable,
+ Arrays.asList(
+ "'scan.partition.column'='id'",
+ "'scan.partition.num'='2'",
+ "'scan.partition.lower-bound'='0'",
+ "'scan.partition.upper-bound'='100'")));
- Iterator<Row> collected =
- tEnv.executeSql("SELECT * FROM " + INPUT_TABLE + " LIMIT 1").collect();
- List<String> result =
- CollectionUtil.iteratorToList(collected).stream()
- .map(Row::toString)
- .sorted()
+ String fields = String.join(",", Arrays.copyOfRange(inputTable.getTableFields(), 0, 3));
+ List<Row> collected = executeQuery(String.format("SELECT %s FROM %s", fields, testTable));
+
+ List<Row> expected =
+ getTestData().stream()
+ .map(row -> Row.of(row.getField(0), row.getField(1), row.getField(2)))
.collect(Collectors.toList());
- Set<String> expected = new HashSet<>();
- expected.add(
- "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 15:35, 1.175E-37, 1.79769E308, 100.1234]");
- expected.add(
- "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, 15:36:01, -1.175E-37, -1.79769E308, 101.1234]");
- assertThat(result).hasSize(1);
- assertThat(expected)
+ assertThat(collected).containsExactlyInAnyOrderElementsOf(expected);
+ }
+
+ @Test
+ public void testLimit() throws Exception {
+ 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'")));
+
+ List<Row> 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(result);
+ .containsAll(collected);
}
@Test
public void testFilter() throws Exception {
- String partitionedTable = "PARTITIONED_TABLE";
- tEnv.executeSql(
- "CREATE TABLE "
- + INPUT_TABLE
- + "("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "time_col TIME,"
- + "real_col FLOAT,"
- + "double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "'"
- + ")");
+ String testTable = "testTable";
+ tEnv.executeSql(inputTable.getCreateQueryForFlink(getMetadata(), testTable));
// create a partitioned table to ensure no regression
+ String partitionedTable = "PARTITIONED_TABLE";
tEnv.executeSql(
- "CREATE TABLE "
- + partitionedTable
- + "("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "time_col TIME,"
- + "real_col FLOAT,"
- + "double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " 'connector'='jdbc',"
- + " 'url'='"
- + DB_URL
- + "',"
- + " 'table-name'='"
- + INPUT_TABLE
- + "',"
- + " 'scan.partition.column'='id',\n"
- + " 'scan.partition.num'='1',\n"
- + " 'scan.partition.lower-bound'='1',\n"
- + " 'scan.partition.upper-bound'='1'\n"
- + ")");
+ 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, timestamp6_col, timestamp9_col, time_col, real_col, double_col, decimal_col"
- + ") as (SELECT * from %s )",
- INPUT_TABLE));
+ "CREATE VIEW FAKE_TABLE (idx, %s) as (SELECT * from %s )",
+ Arrays.stream(inputTable.getTableFields())
+ .filter(f -> !f.equals("id"))
+ .collect(Collectors.joining(",")),
+ testTable));
- List<String> onlyRow1 =
- Stream.of(
- "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 15:35, 1.175E-37, 1.79769E308, 100.1234]")
- .collect(Collectors.toList());
+ Row onlyRow1 =
+ getTestData().stream()
+ .filter(row -> row.getFieldAs(0).equals(1L))
+ .findAny()
+ .orElseThrow(NullPointerException::new);
- List<String> twoRows =
- Stream.of(
- "+I[1, 2020-01-01T15:35:00.123456, 2020-01-01T15:35:00.123456789, 15:35, 1.175E-37, 1.79769E308, 100.1234]",
- "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, 15:36:01, -1.175E-37, -1.79769E308, 101.1234]")
- .collect(Collectors.toList());
+ Row onlyRow2 =
+ getTestData().stream()
+ .filter(row -> row.getFieldAs(0).equals(2L))
+ .findAny()
+ .orElseThrow(NullPointerException::new);
- List<String> onlyRow2 =
- Stream.of(
- "+I[2, 2020-01-01T15:36:01.123456, 2020-01-01T15:36:01.123456789, 15:36:01, -1.175E-37, -1.79769E308, 101.1234]")
- .collect(Collectors.toList());
- List<String> noRows = new ArrayList<>();
+ List<Row> twoRows = getTestData();
// test simple filter
- assertQueryReturns("SELECT * FROM FAKE_TABLE WHERE idx = 1", onlyRow1);
+ assertThat(executeQuery("SELECT * FROM FAKE_TABLE WHERE idx = 1"))
+ .containsExactly(onlyRow1);
+
// test TIMESTAMP filter
- assertQueryReturns(
- "SELECT * FROM FAKE_TABLE WHERE timestamp6_col = TIMESTAMP '2020-01-01 15:35:00.123456'",
- onlyRow1);
+ assertThat(
+ executeQuery(
+ "SELECT * FROM FAKE_TABLE WHERE timestamp6_col = TIMESTAMP '2020-01-01 15:35:00.123456'"))
+ .containsExactly(onlyRow1);
+
// test the IN operator
- assertQueryReturns(
- "SELECT * FROM "
- + "FAKE_TABLE"
- + " WHERE 1 = idx AND decimal_col IN (100.1234, 101.1234)",
- 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
- assertQueryReturns(
- "SELECT * FROM "
- + "FAKE_TABLE"
- + " WHERE idx = 1 AND decimal_col = 100.1234 OR decimal_col = 101.1234",
- 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
- assertQueryReturns(
- "SELECT * FROM "
- + "FAKE_TABLE"
- + " WHERE (2 = idx AND decimal_col = 100.1234) OR decimal_col = 101.1234",
- 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
- assertQueryReturns(
- "SELECT * FROM "
- + "FAKE_TABLE"
- + " WHERE idx = 2 AND decimal_col > 100 OR decimal_col = 101.123",
- 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
- assertQueryReturns(
- "SELECT * FROM "
- + "FAKE_TABLE"
- + " WHERE 2 = idx AND (decimal_col = 100.1234 OR real_col = 101.1234)",
- noRows);
+ assertThat(
+ executeQuery(
+ "SELECT * FROM FAKE_TABLE WHERE 2 = idx AND (decimal_col = 100.1234 OR decimal_col = 102.1234)"))
+ .isEmpty();
- assertQueryReturns(
- "SELECT * FROM "
- + partitionedTable
- + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123",
- noRows);
+ assertThat(
+ executeQuery(
+ "SELECT * FROM "
+ + partitionedTable
+ + " WHERE id = 2 AND decimal_col > 100 OR decimal_col = 101.123"))
+ .isEmpty();
- assertQueryReturns(
- "SELECT * FROM "
- + partitionedTable
- + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)",
- onlyRow1);
- }
-
- private List<String> rowIterToList(Iterator<Row> rows) {
- return CollectionUtil.iteratorToList(rows).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- }
-
- private void assertQueryReturns(String query, List<String> expected) {
- List<String> actual = rowIterToList(tEnv.executeSql(query).collect());
- assertThat(actual).isEqualTo(expected);
+ assertThat(
+ executeQuery(
+ "SELECT * FROM "
+ + partitionedTable
+ + " WHERE 1 = id AND decimal_col IN (100.1234, 101.1234)"))
+ .containsExactly(onlyRow1);
}
@ParameterizedTest
@EnumSource(Caching.class)
void testLookupJoin(Caching caching) throws Exception {
// Create JDBC lookup table
- String cachingOptions = "";
+ List<String> cachingOptions = Collections.emptyList();
if (caching.equals(Caching.ENABLE_CACHE)) {
cachingOptions =
- "'lookup.cache.max-rows' = '100', \n" + "'lookup.cache.ttl' = '10min',";
+ Arrays.asList(
+ "'lookup.cache.max-rows' = '100'", "'lookup.cache.ttl' = '10min'");
}
tEnv.executeSql(
- String.format(
- "CREATE TABLE jdbc_lookup ("
- + "id BIGINT,"
- + "timestamp6_col TIMESTAMP(6),"
- + "timestamp9_col TIMESTAMP(9),"
- + "time_col TIME,"
- + "real_col FLOAT,"
- + "double_col DOUBLE,"
- + "decimal_col DECIMAL(10, 4)"
- + ") WITH ("
- + " %s"
- + " 'connector' = 'jdbc',"
- + " 'url' = '%s',"
- + " 'table-name' = '%s'"
- + ")",
- cachingOptions, DB_URL, INPUT_TABLE));
+ inputTable.getCreateQueryForFlink(getMetadata(), "jdbc_lookup", cachingOptions));
// Create and prepare a value source
String dataId =
@@ -441,13 +292,14 @@
Row.of(3L, "Charlie")));
tEnv.executeSql(
String.format(
- "CREATE TABLE value_source (\n"
- + "`id` BIGINT,\n"
- + "`name` STRING,\n"
- + "`proctime` AS PROCTIME()\n"
- + ") WITH (\n"
- + "'connector' = 'values', \n"
- + "'data-id' = '%s')",
+ "CREATE TABLE value_source ( "
+ + " `id` BIGINT, "
+ + " `name` STRING, "
+ + " `proctime` AS PROCTIME()"
+ + ") WITH ("
+ + " 'connector' = 'values', "
+ + " 'data-id' = '%s'"
+ + ")",
dataId));
if (caching == Caching.ENABLE_CACHE) {
@@ -455,36 +307,42 @@
}
// Execute lookup join
- try (CloseableIterator<Row> iterator =
- tEnv.executeSql(
- "SELECT S.id, S.name, D.id, D.timestamp6_col, D.double_col FROM value_source"
- + " AS S JOIN jdbc_lookup for system_time as of S.proctime AS D ON S.id = D.id")
- .collect()) {
- List<String> result =
- CollectionUtil.iteratorToList(iterator).stream()
- .map(Row::toString)
- .sorted()
- .collect(Collectors.toList());
- List<String> expected = new ArrayList<>();
- expected.add("+I[1, Alice, 1, 2020-01-01T15:35:00.123456, 100.1234]");
- expected.add("+I[1, Alice, 1, 2020-01-01T15:35:00.123456, 100.1234]");
- expected.add("+I[2, Bob, 2, 2020-01-01T15:36:01.123456, 101.1234]");
- assertThat(result).hasSize(3);
- assertThat(expected)
+ try {
+ List<Row> 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<Row> expected =
+ Arrays.asList(
+ Row.of(
+ 1L,
+ "Alice",
+ 1L,
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ BigDecimal.valueOf(100.1234)),
+ Row.of(
+ 1L,
+ "Alice",
+ 1L,
+ LocalDateTime.parse("2020-01-01T15:35:00.123456"),
+ BigDecimal.valueOf(100.1234)),
+ Row.of(
+ 2L,
+ "Bob",
+ 2L,
+ 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) {
- // Validate cache
- Map<String, LookupCacheManager.RefCountedCache> 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();
- validateCachedValues(cache);
- }
+ if (caching == Caching.ENABLE_CACHE) {
+ validateCachedValues();
+ }
} finally {
if (caching == Caching.ENABLE_CACHE) {
LookupCacheManager.getInstance().checkAllReleased();
@@ -494,23 +352,32 @@
}
}
- private void validateCachedValues(LookupCache cache) {
+ private List<Row> executeQuery(String query) {
+ return CollectionUtil.iteratorToList(tEnv.executeSql(query).collect());
+ }
+
+ private void validateCachedValues() {
+ // Validate cache
+ Map<String, LookupCacheManager.RefCountedCache> 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(
- LocalDateTime.parse("2020-01-01T15:35:00.123456")),
- Double.valueOf("1.79769E308"));
+ 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(
- LocalDateTime.parse("2020-01-01T15:36:01.123456")),
- Double.valueOf("-1.79769E308"));
+ LocalDateTime.parse("2020-01-01T15:36:01.123456")));
RowData key3 = GenericRowData.of(3L);
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitorTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitorTest.java
index 57a9e0d..ec08391 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitorTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcFilterPushdownPreparedStatementVisitorTest.java
@@ -19,8 +19,8 @@
package org.apache.flink.connector.jdbc.table;
import org.apache.flink.connector.jdbc.JdbcTestBase;
+import org.apache.flink.connector.jdbc.databases.derby.dialect.DerbyDialectFactory;
import org.apache.flink.connector.jdbc.dialect.JdbcDialect;
-import org.apache.flink.connector.jdbc.dialect.derby.DerbyDialectFactory;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.TableEnvironment;
import org.apache.flink.table.api.TableException;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcLookupTestBase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcLookupTestBase.java
index 700e196..585ff43 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcLookupTestBase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcLookupTestBase.java
@@ -18,31 +18,22 @@
package org.apache.flink.connector.jdbc.table;
-import org.apache.flink.connector.jdbc.JdbcTestFixture;
+import org.apache.flink.connector.jdbc.databases.derby.DerbyTestBase;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import java.sql.Connection;
-import java.sql.DriverManager;
import java.sql.SQLException;
import java.sql.Statement;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
-
/** Base class for JDBC lookup test. */
-class JdbcLookupTestBase {
-
- public static final String DB_URL = "jdbc:derby:memory:lookup";
+class JdbcLookupTestBase implements DerbyTestBase {
public static final String LOOKUP_TABLE = "lookup_table";
@BeforeEach
- void before() throws ClassNotFoundException, SQLException {
- System.setProperty(
- "derby.stream.error.field", JdbcTestFixture.class.getCanonicalName() + ".DEV_NULL");
-
- Class.forName(DERBY_EBOOKSHOP_DB.getDriverClass());
- try (Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+ void before() throws SQLException {
+ try (Connection conn = getMetadata().getConnection();
Statement stat = conn.createStatement()) {
stat.executeUpdate(
"CREATE TABLE "
@@ -96,7 +87,7 @@
}
public void insert(String insertQuery) throws SQLException {
- try (Connection conn = DriverManager.getConnection(DB_URL + ";create=true");
+ try (Connection conn = getMetadata().getConnection();
Statement stat = conn.createStatement()) {
stat.execute(insertQuery);
}
@@ -104,8 +95,7 @@
@AfterEach
void clearOutputTable() throws Exception {
- Class.forName(DERBY_EBOOKSHOP_DB.getDriverClass());
- try (Connection conn = DriverManager.getConnection(DB_URL);
+ try (Connection conn = getMetadata().getConnection();
Statement stat = conn.createStatement()) {
stat.execute("DROP TABLE " + LOOKUP_TABLE);
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcOutputFormatTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcOutputFormatTest.java
index dacacce..d7d95b6 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcOutputFormatTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcOutputFormatTest.java
@@ -42,7 +42,6 @@
import java.sql.Statement;
import java.util.Arrays;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.OUTPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.OUTPUT_TABLE_2;
@@ -92,7 +91,7 @@
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
.setDriverName("org.apache.derby.jdbc.idontexist")
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(INPUT_TABLE)
.build();
JdbcDmlOptions dmlOptions =
@@ -122,7 +121,7 @@
() -> {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDriverName(getMetadata().getDriverClass())
.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
.setTableName(INPUT_TABLE)
.build();
@@ -152,8 +151,8 @@
() -> {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(INPUT_TABLE)
.build();
JdbcDmlOptions dmlOptions =
@@ -191,8 +190,8 @@
() -> {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE)
.build();
JdbcDmlOptions dmlOptions =
@@ -232,8 +231,8 @@
() -> {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE)
.build();
JdbcDmlOptions dmlOptions =
@@ -278,8 +277,8 @@
void testJdbcOutputFormat() throws IOException, SQLException {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE)
.build();
JdbcDmlOptions dmlOptions =
@@ -310,7 +309,7 @@
outputFormat.close();
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS);
ResultSet resultSet = statement.executeQuery()) {
int recordCount = 0;
@@ -331,8 +330,8 @@
void testFlush() throws SQLException, IOException {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE_2)
.build();
JdbcDmlOptions dmlOptions =
@@ -355,7 +354,7 @@
setRuntimeContext(outputFormat, true);
outputFormat.open(0, 1);
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS_2)) {
outputFormat.open(0, 1);
for (int i = 0; i < 2; ++i) {
@@ -401,8 +400,8 @@
void testFlushWithBatchSizeEqualsZero() throws SQLException, IOException {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE_2)
.build();
JdbcDmlOptions dmlOptions =
@@ -424,7 +423,7 @@
.build();
setRuntimeContext(outputFormat, true);
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS_2)) {
outputFormat.open(0, 1);
for (int i = 0; i < 2; ++i) {
@@ -448,8 +447,8 @@
void testInvalidConnectionInJdbcOutputFormat() throws IOException, SQLException {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(OUTPUT_TABLE_3)
.build();
JdbcDmlOptions dmlOptions =
@@ -489,7 +488,7 @@
outputFormat.close();
- try (Connection dbConn = DriverManager.getConnection(DERBY_EBOOKSHOP_DB.getJdbcUrl());
+ try (Connection dbConn = DriverManager.getConnection(getMetadata().getJdbcUrl());
PreparedStatement statement = dbConn.prepareStatement(SELECT_ALL_NEWBOOKS_3);
ResultSet resultSet = statement.executeQuery()) {
int recordCount = 0;
@@ -508,7 +507,7 @@
@AfterEach
void clearOutputTable() throws Exception {
- try (Connection conn = DERBY_EBOOKSHOP_DB.getConnection();
+ try (Connection conn = getMetadata().getConnection();
Statement stat = conn.createStatement()) {
stat.execute("DELETE FROM " + OUTPUT_TABLE);
}
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 a0040a2..b2e4dee 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
@@ -42,7 +42,6 @@
import java.sql.ResultSet;
import java.util.Arrays;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.INPUT_TABLE;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR;
@@ -67,7 +66,7 @@
};
final JdbcDialect dialect =
InternalJdbcConnectionOptions.builder()
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(INPUT_TABLE)
.build()
.getDialect();
@@ -93,8 +92,8 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.build();
inputFormat.openInputFormat();
@@ -109,7 +108,7 @@
inputFormat =
JdbcRowDataInputFormat.builder()
.setDrivername("org.apache.derby.jdbc.idontexist")
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setRowConverter(dialect.getRowConverter(rowType))
.build();
@@ -124,7 +123,7 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setDBUrl("jdbc:der:iamanerror:mory:ebookshop")
.setQuery(SELECT_ALL_BOOKS)
.setRowConverter(dialect.getRowConverter(rowType))
@@ -140,8 +139,8 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery("iamnotsql")
.setRowConverter(dialect.getRowConverter(rowType))
.build();
@@ -156,8 +155,8 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setRowConverter(dialect.getRowConverter(rowType))
.build();
})
@@ -171,7 +170,7 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
+ .setDrivername(getMetadata().getDriverClass())
.setQuery(SELECT_ALL_BOOKS)
.setRowConverter(dialect.getRowConverter(rowType))
.build();
@@ -186,8 +185,8 @@
() -> {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setFetchSize(-7)
.build();
@@ -199,8 +198,8 @@
void testValidFetchSizeIntegerMin() {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setFetchSize(Integer.MIN_VALUE)
.setRowConverter(dialect.getRowConverter(rowType))
@@ -211,8 +210,8 @@
void testJdbcInputFormatWithoutParallelism() throws IOException {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
.setRowConverter(dialect.getRowConverter(rowType))
@@ -244,8 +243,8 @@
new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize);
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID)
.setParametersProvider(pramProvider)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
@@ -282,8 +281,8 @@
new JdbcNumericBetweenParametersProvider(min, max).ofBatchSize(fetchSize);
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_ID)
.setParametersProvider(pramProvider)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
@@ -320,8 +319,8 @@
new JdbcGenericParameterValuesProvider(queryParameters);
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_ALL_BOOKS_SPLIT_BY_AUTHOR)
.setParametersProvider(paramProvider)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
@@ -362,8 +361,8 @@
void testEmptyResults() throws IOException {
inputFormat =
JdbcRowDataInputFormat.builder()
- .setDrivername(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DERBY_EBOOKSHOP_DB.getJdbcUrl())
+ .setDrivername(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setQuery(SELECT_EMPTY)
.setResultSetType(ResultSet.TYPE_SCROLL_INSENSITIVE)
.setRowConverter(dialect.getRowConverter(rowType))
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunctionTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunctionTest.java
index 5759c52..b4f06ce 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunctionTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/JdbcRowDataLookupFunctionTest.java
@@ -37,7 +37,6 @@
import java.util.List;
import java.util.stream.Collectors;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.DERBY_EBOOKSHOP_DB;
import static org.assertj.core.api.Assertions.assertThat;
/** Test suite for {@link JdbcRowDataLookupFunction}. */
@@ -86,8 +85,8 @@
private JdbcRowDataLookupFunction buildRowDataLookupFunction(boolean withFailure) {
InternalJdbcConnectionOptions jdbcOptions =
InternalJdbcConnectionOptions.builder()
- .setDriverName(DERBY_EBOOKSHOP_DB.getDriverClass())
- .setDBUrl(DB_URL)
+ .setDriverName(getMetadata().getDriverClass())
+ .setDBUrl(getMetadata().getJdbcUrl())
.setTableName(LOOKUP_TABLE)
.build();
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
deleted file mode 100644
index d236d57..0000000
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/table/UnsignedTypeConversionITCase.java
+++ /dev/null
@@ -1,172 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.connector.jdbc.table;
-
-import org.apache.flink.connector.jdbc.databases.mysql.MySqlDatabase;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.Table;
-import org.apache.flink.table.api.TableEnvironment;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.test.util.AbstractTestBase;
-import org.apache.flink.types.Row;
-import org.apache.flink.types.RowKind;
-import org.apache.flink.util.CloseableIterator;
-import org.apache.flink.util.CollectionUtil;
-
-import org.junit.jupiter.api.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.math.BigDecimal;
-import java.sql.Connection;
-import java.sql.PreparedStatement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.Arrays;
-import java.util.List;
-
-import static java.lang.String.format;
-import static java.lang.String.join;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/**
- * Test unsigned type conversion between Flink and JDBC driver mysql, the test underlying use MySQL
- * to mock a DB.
- */
-class UnsignedTypeConversionITCase extends AbstractTestBase implements MySqlDatabase {
-
- private static final Logger LOGGER =
- LoggerFactory.getLogger(UnsignedTypeConversionITCase.class);
-
- private static final String TABLE_NAME = "unsigned_test";
- private static final List<String> COLUMNS =
- Arrays.asList(
- "tiny_c",
- "tiny_un_c",
- "small_c",
- "small_un_c",
- "int_c",
- "int_un_c",
- "big_c",
- "big_un_c");
-
- private static final Object[] ROW =
- new Object[] {
- (byte) 127,
- (short) 255,
- (short) 32767,
- 65535,
- 2147483647,
- 4294967295L,
- 9223372036854775807L,
- new BigDecimal("18446744073709551615")
- };
-
- @Test
- void testUnsignedType() throws Exception {
- try (Connection con = getMetadata().getConnection()) {
- StreamExecutionEnvironment sEnv = StreamExecutionEnvironment.getExecutionEnvironment();
- TableEnvironment tableEnv = StreamTableEnvironment.create(sEnv);
- createMysqlTable(con);
- createFlinkTable(tableEnv);
- prepareData(tableEnv);
-
- // write data to db
- tableEnv.executeSql(
- format("insert into jdbc_sink select %s from data", join(",", COLUMNS)))
- .await();
-
- // read data from db using jdbc connection and compare
- try (PreparedStatement ps =
- con.prepareStatement(
- format("select %s from %s", join(",", COLUMNS), TABLE_NAME))) {
- ResultSet resultSet = ps.executeQuery();
- while (resultSet.next()) {
- for (int i = 0; i < ROW.length; i++) {
- assertThat(resultSet.getObject(i + 1, ROW[i].getClass())).isEqualTo(ROW[i]);
- }
- }
- }
-
- // read data from db using flink and compare
- String sql = format("select %s from jdbc_source", join(",", COLUMNS));
- CloseableIterator<Row> collected = tableEnv.executeSql(sql).collect();
- List<Row> result = CollectionUtil.iteratorToList(collected);
- assertThat(result).containsOnly(Row.ofKind(RowKind.INSERT, ROW));
- }
- }
-
- private void createMysqlTable(Connection con) throws SQLException {
- try (PreparedStatement ps =
- con.prepareStatement(
- "create table "
- + TABLE_NAME
- + " ("
- + " tiny_c TINYINT,"
- + " tiny_un_c TINYINT UNSIGNED,"
- + " small_c SMALLINT,"
- + " small_un_c SMALLINT UNSIGNED,"
- + " int_c INTEGER ,"
- + " int_un_c INTEGER UNSIGNED,"
- + " big_c BIGINT,"
- + " big_un_c BIGINT UNSIGNED);")) {
- ps.execute();
- }
- }
-
- private void createFlinkTable(TableEnvironment tableEnv) {
- String commonDDL =
- "create table %s ("
- + "tiny_c TINYINT,"
- + "tiny_un_c SMALLINT,"
- + "small_c SMALLINT,"
- + "small_un_c INT,"
- + "int_c INT,"
- + "int_un_c BIGINT,"
- + "big_c BIGINT,"
- + "big_un_c DECIMAL(20, 0)) with("
- + " 'connector' = 'jdbc',"
- + " 'url' = '"
- + getMetadata().getJdbcUrlWithCredentials()
- + "',"
- + " 'table-name' = '"
- + TABLE_NAME
- + "'"
- + ")";
- tableEnv.executeSql(format(commonDDL, "jdbc_source"));
- tableEnv.executeSql(format(commonDDL, "jdbc_sink"));
- }
-
- private void prepareData(TableEnvironment tableEnv) {
- Table dataTable =
- tableEnv.fromValues(
- DataTypes.ROW(
- DataTypes.FIELD("tiny_c", DataTypes.TINYINT().notNull()),
- DataTypes.FIELD("tiny_un_c", DataTypes.SMALLINT().notNull()),
- DataTypes.FIELD("small_c", DataTypes.SMALLINT().notNull()),
- DataTypes.FIELD("small_un_c", DataTypes.INT().notNull()),
- DataTypes.FIELD("int_c", DataTypes.INT().notNull()),
- DataTypes.FIELD("int_un_c", DataTypes.BIGINT().notNull()),
- DataTypes.FIELD("big_c", DataTypes.BIGINT().notNull()),
- DataTypes.FIELD("big_un_c", DataTypes.DECIMAL(20, 0).notNull())),
- Row.of(ROW));
- tableEnv.createTemporaryView("data", dataTable);
- }
-}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseExtension.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseExtension.java
new file mode 100644
index 0000000..a9a6453
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseExtension.java
@@ -0,0 +1,182 @@
+/*
+ * 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;
+
+import org.apache.flink.util.function.BiConsumerWithException;
+
+import org.junit.jupiter.api.extension.AfterAllCallback;
+import org.junit.jupiter.api.extension.AfterEachCallback;
+import org.junit.jupiter.api.extension.BeforeAllCallback;
+import org.junit.jupiter.api.extension.BeforeEachCallback;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.api.extension.ExtensionContext;
+import org.junit.jupiter.api.extension.ExtensionContext.Namespace;
+
+import java.lang.reflect.Constructor;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.Set;
+import java.util.function.BiFunction;
+import java.util.stream.Collectors;
+
+import static org.junit.platform.commons.support.AnnotationSupport.findRepeatableAnnotations;
+
+/** Database extension for testing. */
+public abstract class DatabaseExtension
+ implements BeforeAllCallback,
+ AfterAllCallback,
+ BeforeEachCallback,
+ AfterEachCallback,
+ ExtensionContext.Store.CloseableResource {
+
+ /**
+ * Database Lifecycle for testing. The goal it's that all database containers are create only
+ * one time.
+ */
+ public enum Lifecycle {
+ /** Database will be instantiated only one time. */
+ PER_EXECUTION,
+ /** Database will be instantiated by class. */
+ PER_CLASS
+ }
+
+ protected abstract DatabaseMetadata startDatabase() throws Exception;
+
+ protected abstract void stopDatabase() throws Exception;
+
+ private final String uniqueKey = this.getClass().getSimpleName();
+
+ protected Lifecycle getLifecycle() {
+ return Lifecycle.PER_EXECUTION;
+ }
+
+ private ExtensionContext.Store getStore(ExtensionContext context) {
+ return context.getRoot().getStore(Namespace.GLOBAL);
+ }
+
+ private DatabaseTest getDatabaseBaseTest(Class<?> clazz) throws Exception {
+ DatabaseTest dbClazz = null;
+ for (Constructor<?> c : clazz.getDeclaredConstructors()) {
+ c.setAccessible(true);
+ dbClazz = (DatabaseTest) c.newInstance();
+ }
+ return dbClazz;
+ }
+
+ private void getManagedTables(
+ ExtensionContext context,
+ BiConsumerWithException<TableManaged, Connection, SQLException> execute) {
+ context.getTestClass()
+ .filter(DatabaseTest.class::isAssignableFrom)
+ .ifPresent(
+ clazz -> {
+ DatabaseMetadata metadata =
+ getStore(context).get(uniqueKey, DatabaseMetadata.class);
+ if (metadata != null) {
+ try (Connection conn = metadata.getConnection()) {
+ for (TableManaged table :
+ getDatabaseBaseTest(clazz).getManagedTables()) {
+ execute.accept(table, conn);
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+ });
+ }
+
+ private boolean ignoreTestDatabase(ExtensionContext context) {
+ Set<String> dbExtensions = retrieveDatabaseExtensions(context);
+
+ if (dbExtensions.size() > 1) {
+ return uniqueKey.equals("DerbyDatabase") && dbExtensions.contains(uniqueKey);
+ }
+ return false;
+ }
+
+ @Override
+ public final void beforeAll(ExtensionContext context) throws Exception {
+ if (ignoreTestDatabase(context)) {
+ return;
+ }
+
+ if (getStore(context).get(uniqueKey) == null) {
+ getStore(context).put(uniqueKey, startDatabase());
+ }
+
+ getManagedTables(context, TableManaged::createTable);
+ }
+
+ @Override
+ public final void beforeEach(ExtensionContext context) throws Exception {}
+
+ @Override
+ public final void afterEach(ExtensionContext context) throws Exception {
+ if (ignoreTestDatabase(context)) {
+ return;
+ }
+ getManagedTables(context, TableManaged::deleteTable);
+ }
+
+ @Override
+ public final void afterAll(ExtensionContext context) throws Exception {
+ if (ignoreTestDatabase(context)) {
+ return;
+ }
+ getManagedTables(context, TableManaged::dropTable);
+ if (Lifecycle.PER_CLASS == getLifecycle()) {
+ stopDatabase();
+ getStore(context).remove(uniqueKey, DatabaseMetadata.class);
+ }
+ }
+
+ @Override
+ public final void close() throws Throwable {
+ if (Lifecycle.PER_EXECUTION == getLifecycle()) {
+ stopDatabase();
+ }
+ }
+
+ private Set<String> retrieveDatabaseExtensions(final ExtensionContext context) {
+
+ BiFunction<ExtensionContext, Set<String>, Set<String>> retrieveExtensions =
+ new BiFunction<ExtensionContext, Set<String>, Set<String>>() {
+
+ @Override
+ public Set<String> apply(ExtensionContext context, Set<String> acc) {
+ Set<String> current = new HashSet<>(acc);
+ current.addAll(
+ findRepeatableAnnotations(context.getElement(), ExtendWith.class)
+ .stream()
+ .flatMap(extendWith -> Arrays.stream(extendWith.value()))
+ .filter(DatabaseExtension.class::isAssignableFrom)
+ .map(Class::getSimpleName)
+ .collect(Collectors.toSet()));
+
+ return context.getParent()
+ .map(extensionContext -> apply(extensionContext, current))
+ .orElse(current);
+ }
+ };
+
+ return retrieveExtensions.apply(context, new HashSet<>());
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseMetadata.java
similarity index 89%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseMetadata.java
index b0956a8..30ba9cc 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseMetadata.java
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases;
+package org.apache.flink.connector.jdbc.testutils;
import org.apache.flink.connector.jdbc.JdbcConnectionOptions;
import org.apache.flink.util.FlinkRuntimeException;
+import org.apache.flink.util.function.SerializableSupplier;
import javax.sql.XADataSource;
@@ -43,6 +44,10 @@
String getVersion();
+ default SerializableSupplier<XADataSource> getXaSourceSupplier() {
+ return this::buildXaDataSource;
+ }
+
default JdbcConnectionOptions getConnectionOptions() {
return new JdbcConnectionOptions.JdbcConnectionOptionsBuilder()
.withDriverName(getDriverClass())
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseTest.java
similarity index 82%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseTest.java
index 802a468..d0f26fa 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseTest.java
@@ -15,10 +15,17 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases;
+package org.apache.flink.connector.jdbc.testutils;
+
+import java.util.Collections;
+import java.util.List;
/** Base interface for tests that have dependency in a database. */
public interface DatabaseTest {
DatabaseMetadata getMetadata();
+
+ default List<TableManaged> getManagedTables() {
+ return Collections.emptyList();
+ }
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/TableManaged.java
similarity index 66%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/TableManaged.java
index 61314af..dd43a09 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/TableManaged.java
@@ -16,17 +16,19 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.testutils;
-import org.junit.jupiter.api.Test;
+import java.sql.Connection;
+import java.sql.SQLException;
-import static org.assertj.core.api.Assertions.assertThat;
+/** Table that can be manage by {@link DatabaseExtension}. */
+public interface TableManaged {
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
+ String getTableName();
+
+ void createTable(Connection conn) throws SQLException;
+
+ void deleteTable(Connection conn) throws SQLException;
+
+ void dropTable(Connection conn) throws SQLException;
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyDatabase.java
new file mode 100644
index 0000000..61c5541
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyDatabase.java
@@ -0,0 +1,74 @@
+/*
+ * 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.derby;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import java.io.OutputStream;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+
+/** Derby database for testing. */
+public class DerbyDatabase extends DatabaseExtension {
+
+ @SuppressWarnings("unused") // used in string constant in prepareDatabase
+ public static final OutputStream DEV_NULL =
+ new OutputStream() {
+ @Override
+ public void write(int b) {}
+ };
+
+ private static DerbyMetadata metadata;
+
+ public static DerbyMetadata getMetadata() {
+ if (metadata == null) {
+ metadata = new DerbyMetadata("test");
+ }
+ return metadata;
+ }
+
+ @Override
+ public DatabaseMetadata startDatabase() throws Exception {
+ DatabaseMetadata metadata = getMetadata();
+ try {
+ System.setProperty(
+ "derby.stream.error.field",
+ DerbyDatabase.class.getCanonicalName() + ".DEV_NULL");
+ Class.forName(metadata.getDriverClass());
+ DriverManager.getConnection(String.format("%s;create=true", metadata.getJdbcUrl()))
+ .close();
+ } catch (Exception e) {
+ throw new FlinkRuntimeException(e);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ try {
+ DriverManager.getConnection(String.format("%s;shutdown=true", metadata.getJdbcUrl()))
+ .close();
+ } catch (SQLException ignored) {
+ } finally {
+ metadata = null;
+ }
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyMetadata.java
similarity index 93%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyMetadata.java
index 087aa58..b5df6ef 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/derby/DerbyMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/derby/DerbyMetadata.java
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.derby;
+package org.apache.flink.connector.jdbc.testutils.databases.derby;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import org.apache.derby.jdbc.EmbeddedXADataSource;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2Metadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2Metadata.java
similarity index 89%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2Metadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2Metadata.java
index ee41685..cf308ea 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2Metadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2Metadata.java
@@ -15,10 +15,10 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.h2;
+package org.apache.flink.connector.jdbc.testutils.databases.h2;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.h2.xa.H2XaDsWrapper;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.h2.xa.H2XaDsWrapper;
import javax.sql.XADataSource;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2XaDatabase.java
similarity index 69%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaDatabase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2XaDatabase.java
index 747ec0f..731d2b4 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/H2XaDatabase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/H2XaDatabase.java
@@ -15,26 +15,29 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.h2;
+package org.apache.flink.connector.jdbc.testutils.databases.h2;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import org.apache.flink.util.FlinkRuntimeException;
import java.sql.DriverManager;
/** H2 database for testing. */
-public interface H2XaDatabase extends DatabaseTest {
+public class H2XaDatabase extends DatabaseExtension {
- DatabaseMetadata METADATA = startDatabase();
+ private static H2Metadata metadata;
- @Override
- default DatabaseMetadata getMetadata() {
- return METADATA;
+ public static H2Metadata getMetadata() {
+ if (metadata == null) {
+ metadata = new H2Metadata("test");
+ }
+ return metadata;
}
- static DatabaseMetadata startDatabase() {
- DatabaseMetadata metadata = new H2Metadata("test");
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ DatabaseMetadata metadata = getMetadata();
try {
Class.forName(metadata.getDriverClass());
DriverManager.getConnection(
@@ -47,4 +50,7 @@
}
return metadata;
}
+
+ @Override
+ protected void stopDatabase() throws Exception {}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaConnectionWrapper.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaConnectionWrapper.java
similarity index 96%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaConnectionWrapper.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaConnectionWrapper.java
index 50893db..274093b 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaConnectionWrapper.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaConnectionWrapper.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.h2.xa;
+package org.apache.flink.connector.jdbc.testutils.databases.h2.xa;
import javax.sql.ConnectionEventListener;
import javax.sql.StatementEventListener;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaDsWrapper.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaDsWrapper.java
similarity index 97%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaDsWrapper.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaDsWrapper.java
index 25b6c37..e4a3b22 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaDsWrapper.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaDsWrapper.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.h2.xa;
+package org.apache.flink.connector.jdbc.testutils.databases.h2.xa;
import javax.sql.XAConnection;
import javax.sql.XADataSource;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaResourceWrapper.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaResourceWrapper.java
similarity index 98%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaResourceWrapper.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaResourceWrapper.java
index 8cc7074..c168fc3 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/H2XaResourceWrapper.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/H2XaResourceWrapper.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.h2.xa;
+package org.apache.flink.connector.jdbc.testutils.databases.h2.xa;
import org.apache.flink.util.function.ThrowingRunnable;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/package-info.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/package-info.java
similarity index 84%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/package-info.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/package-info.java
index 4e4ad39..8e46a4f 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/h2/xa/package-info.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/h2/xa/package-info.java
@@ -17,6 +17,6 @@
/**
* This package holds some workarounds for the H2 XA client, plus {@link
- * org.apache.flink.connector.jdbc.xa.h2.H2DbMetadata}. Used only for testing.
+ * org.apache.flink.connector.jdbc.testutils.databases.h2.H2Metadata}. Used only for testing.
*/
-package org.apache.flink.connector.jdbc.databases.h2.xa;
+package org.apache.flink.connector.jdbc.testutils.databases.h2.xa;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql56Database.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql56Database.java
new file mode 100644
index 0000000..6a6d123
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql56Database.java
@@ -0,0 +1,57 @@
+/*
+ * 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.mysql;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.testcontainers.containers.MySQLContainer;
+
+/** A MySql database for testing. */
+public class MySql56Database extends DatabaseExtension implements MySqlImages {
+
+ private static final MySQLContainer<?> CONTAINER =
+ new MySqlContainer(MYSQL_5_6)
+ .withCommand("--character-set-server=utf8")
+ .withEnv("MYSQL_ROOT_HOST", "%");
+
+ private static MySqlMetadata metadata;
+
+ public static MySqlMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new MySqlMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql57Database.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql57Database.java
new file mode 100644
index 0000000..c100f34
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySql57Database.java
@@ -0,0 +1,57 @@
+/*
+ * 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.mysql;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.testcontainers.containers.MySQLContainer;
+
+/** A MySql database for testing. */
+public class MySql57Database extends DatabaseExtension implements MySqlImages {
+
+ private static final MySQLContainer<?> CONTAINER =
+ new MySqlContainer(MYSQL_5_7)
+ .withCommand("--character-set-server=utf8")
+ .withEnv("MYSQL_ROOT_HOST", "%");
+
+ private static MySqlMetadata metadata;
+
+ public static MySqlMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new MySqlMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlContainer.java
similarity index 65%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlDatabase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlContainer.java
index 512b43a..5859bdf 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlDatabase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlContainer.java
@@ -1,12 +1,13 @@
/*
- * 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
+ * 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
+ * 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,
@@ -15,17 +16,13 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.mysql;
+package org.apache.flink.connector.jdbc.testutils.databases.mysql;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
import org.apache.flink.util.ExceptionUtils;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.MySQLContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
import org.testcontainers.utility.DockerImageName;
import java.sql.Connection;
@@ -36,81 +33,84 @@
import static org.apache.flink.util.Preconditions.checkArgument;
-/** A MySql database for testing. */
-@Testcontainers
-public interface MySqlDatabase extends DatabaseTest {
+/** {@link MySQLContainer} with Xa and LockDB. */
+public class MySqlContainer extends MySQLContainer<MySqlContainer> {
+ private long lockWaitTimeout = 0;
+ private boolean xaActive = false;
+ private volatile InnoDbStatusLogger innoDbStatusLogger;
- String MYSQL_5_6 = "mysql:5.6.51";
- String MYSQL_5_7 = "mysql:5.7.41";
- String MYSQL_8_0 = "mysql:8.0.32";
-
- @Container
- MySqlXaContainer CONTAINER = new MySqlXaContainer(MYSQL_8_0).withLockWaitTimeout(50_000L);
-
- @Override
- default DatabaseMetadata getMetadata() {
- return new MySqlMetadata(CONTAINER);
+ public MySqlContainer(String dockerImageName) {
+ super(DockerImageName.parse(dockerImageName));
}
- /** {@link MySQLContainer} with XA enabled. */
- class MySqlXaContainer extends MySQLContainer<MySqlXaContainer> {
- private long lockWaitTimeout = 0;
- private volatile InnoDbStatusLogger innoDbStatusLogger;
+ public MySqlContainer withXa() {
+ this.xaActive = true;
+ return this.self();
+ }
- public MySqlXaContainer(String dockerImageName) {
- super(DockerImageName.parse(dockerImageName));
- }
+ public MySqlContainer withLockWaitTimeout(long lockWaitTimeout) {
+ checkArgument(lockWaitTimeout >= 0, "lockWaitTimeout should be greater than 0");
+ this.lockWaitTimeout = lockWaitTimeout;
+ return this.self();
+ }
- public MySqlXaContainer withLockWaitTimeout(long lockWaitTimeout) {
- checkArgument(lockWaitTimeout >= 0, "lockWaitTimeout should be greater than 0");
- this.lockWaitTimeout = lockWaitTimeout;
- return this.self();
- }
+ @Override
+ public void start() {
+ super.start();
+ // prevent XAER_RMERR: Fatal error occurred in the transaction branch - check your
+ // data for consistency works for mysql v8+
+ prepareDb();
- @Override
- public void start() {
- super.start();
- // prevent XAER_RMERR: Fatal error occurred in the transaction branch - check your
- // data for consistency works for mysql v8+
- try (Connection connection =
- DriverManager.getConnection(getJdbcUrl(), "root", getPassword())) {
- prepareDb(connection, lockWaitTimeout);
- } catch (SQLException e) {
- ExceptionUtils.rethrow(e);
- }
-
+ if (lockWaitTimeout > 0) {
this.innoDbStatusLogger =
new InnoDbStatusLogger(
getJdbcUrl(), "root", getPassword(), lockWaitTimeout / 2);
innoDbStatusLogger.start();
}
+ }
- @Override
- public void stop() {
- try {
+ @Override
+ public void stop() {
+ try {
+ if (innoDbStatusLogger != null) {
innoDbStatusLogger.stop();
- } catch (Exception e) {
- ExceptionUtils.rethrow(e);
- } finally {
- super.stop();
}
+ } catch (Exception e) {
+ ExceptionUtils.rethrow(e);
+ } finally {
+ super.stop();
}
+ }
- private void prepareDb(Connection connection, long lockWaitTimeout) throws SQLException {
- try (Statement st = connection.createStatement()) {
- st.execute("GRANT XA_RECOVER_ADMIN ON *.* TO '" + getUsername() + "'@'%'");
+ private void prepareDb() {
+ try (Connection connection =
+ DriverManager.getConnection(getJdbcUrl(), "root", getPassword());
+ Statement st = connection.createStatement()) {
+ st.execute(
+ String.format(
+ "GRANT ALL PRIVILEGES ON *.* TO '%s'@'%%' WITH GRANT OPTION",
+ getUsername()));
+ st.execute("FLUSH PRIVILEGES");
+
+ if (xaActive) {
+ st.execute(
+ String.format("GRANT XA_RECOVER_ADMIN ON *.* TO '%s'@'%%'", getUsername()));
st.execute("FLUSH PRIVILEGES");
- // if the reason of task cancellation failure is waiting for a lock
- // then failing transactions with a relevant message would ease debugging
+ }
+ // if the reason of task cancellation failure is waiting for a lock
+ // then failing transactions with a relevant message would ease debugging
+ if (lockWaitTimeout > 0) {
st.execute("SET GLOBAL innodb_lock_wait_timeout = " + lockWaitTimeout);
// st.execute("SET GLOBAL innodb_status_output = ON");
// st.execute("SET GLOBAL innodb_status_output_locks = ON");
}
+ } catch (Exception e) {
+ ExceptionUtils.rethrow(e);
}
}
/** InnoDB status logger. */
- class InnoDbStatusLogger {
+ public static class InnoDbStatusLogger {
private static final Logger LOG = LoggerFactory.getLogger(InnoDbStatusLogger.class);
private final Thread thread;
private volatile boolean running;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlDatabase.java
new file mode 100644
index 0000000..2ce66c1
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlDatabase.java
@@ -0,0 +1,59 @@
+/*
+ * 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.mysql;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.testcontainers.containers.MySQLContainer;
+
+/** A MySql database for testing. */
+public class MySqlDatabase extends DatabaseExtension implements MySqlImages {
+
+ private static final MySQLContainer<?> CONTAINER =
+ new MySqlContainer(MYSQL_8_0)
+ .withXa()
+ .withLockWaitTimeout(50_000L)
+ .withCommand("--character-set-server=utf8")
+ .withEnv("MYSQL_ROOT_HOST", "%");
+
+ private static MySqlMetadata metadata;
+
+ public static MySqlMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new MySqlMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlImages.java
similarity index 76%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlImages.java
index 26e183f..93a176a 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlImages.java
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.oracle;
+package org.apache.flink.connector.jdbc.testutils.databases.mysql;
-/** Oracle docker images. */
-public interface OracleImages {
- String ORACLE_18 = "gvenzl/oracle-xe:18.4.0-slim";
- String ORACLE_21 = "gvenzl/oracle-xe:21.3.0-slim-faststart";
+/** MySql docker images. */
+public interface MySqlImages {
+ String MYSQL_5_6 = "mysql:5.6.51";
+ String MYSQL_5_7 = "mysql:5.7.41";
+ String MYSQL_8_0 = "mysql:8.0.32";
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlMetadata.java
similarity index 95%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlMetadata.java
index d553f15..e92c65d 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/mysql/MySqlMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/mysql/MySqlMetadata.java
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.mysql;
+package org.apache.flink.connector.jdbc.testutils.databases.mysql;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import com.mysql.cj.jdbc.MysqlXADataSource;
import org.testcontainers.containers.MySQLContainer;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleDatabase.java
new file mode 100644
index 0000000..f922581
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleDatabase.java
@@ -0,0 +1,58 @@
+/*
+ * 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.oracle;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.testcontainers.containers.OracleContainer;
+
+/** A Oracle database for testing. */
+public class OracleDatabase extends DatabaseExtension implements OracleImages {
+
+ private static final OracleContainer CONTAINER =
+ new OracleContainer(ORACLE_21)
+ .withStartupTimeoutSeconds(240)
+ .withConnectTimeoutSeconds(120)
+ .usingSid();
+
+ private static OracleMetadata metadata;
+
+ public static OracleMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new OracleMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleImages.java
similarity index 93%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleImages.java
index 26e183f..8c31b51 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleImages.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleImages.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.oracle;
+package org.apache.flink.connector.jdbc.testutils.databases.oracle;
/** Oracle docker images. */
public interface OracleImages {
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleMetadata.java
similarity index 95%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleMetadata.java
index b62c0e0..9309420 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/oracle/OracleMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/oracle/OracleMetadata.java
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.oracle;
+package org.apache.flink.connector.jdbc.testutils.databases.oracle;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import oracle.jdbc.xa.client.OracleXADataSource;
import org.testcontainers.containers.OracleContainer;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresDatabase.java
similarity index 70%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresDatabase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresDatabase.java
index daa607b..4d33d78 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresDatabase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresDatabase.java
@@ -15,36 +15,49 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.postgres;
+package org.apache.flink.connector.jdbc.testutils.databases.postgres;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.FlinkRuntimeException;
import org.testcontainers.containers.PostgreSQLContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
import org.testcontainers.utility.DockerImageName;
import static org.apache.flink.util.Preconditions.checkArgument;
/** A Postgres database for testing. */
-@Testcontainers
-public interface PostgresDatabase extends DatabaseTest {
+public class PostgresDatabase extends DatabaseExtension implements PostgresImages {
- String POSTGRES_9 = "postgres:9.6.24";
- String POSTGRES_15 = "postgres:15.1";
-
- @Container
- PostgreSQLContainer<?> CONTAINER =
+ private static final PostgreSQLContainer<?> CONTAINER =
new PostgresXaContainer(POSTGRES_15).withMaxConnections(10).withMaxTransactions(50);
+ private static PostgresMetadata metadata;
+
+ public static PostgresMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new PostgresMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
@Override
- default DatabaseMetadata getMetadata() {
- return new PostgresMetadata(CONTAINER);
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
}
/** {@link PostgreSQLContainer} with XA enabled (by setting max_prepared_transactions). */
- class PostgresXaContainer extends PostgreSQLContainer<PostgresXaContainer> {
+ public static class PostgresXaContainer extends PostgreSQLContainer<PostgresXaContainer> {
private static final int SUPERUSER_RESERVED_CONNECTIONS = 1;
private int maxConnections = SUPERUSER_RESERVED_CONNECTIONS + 1;
private int maxTransactions = 1;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresImages.java
similarity index 78%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresImages.java
index 802a468..6cfc715 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/DatabaseTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresImages.java
@@ -15,10 +15,10 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases;
+package org.apache.flink.connector.jdbc.testutils.databases.postgres;
-/** Base interface for tests that have dependency in a database. */
-public interface DatabaseTest {
-
- DatabaseMetadata getMetadata();
+/** Postgres docker images. */
+public interface PostgresImages {
+ String POSTGRES_9 = "postgres:9.6.24";
+ String POSTGRES_15 = "postgres:15.1";
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresMetadata.java
similarity index 95%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresMetadata.java
index 6ed24b0..1d66d0d 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/postgres/PostgresMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/postgres/PostgresMetadata.java
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.postgres;
+package org.apache.flink.connector.jdbc.testutils.databases.postgres;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import org.postgresql.xa.PGXADataSource;
import org.testcontainers.containers.PostgreSQLContainer;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerDatabase.java
new file mode 100644
index 0000000..9d76186
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerDatabase.java
@@ -0,0 +1,99 @@
+/*
+ * 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.sqlserver;
+
+import org.apache.flink.connector.jdbc.testutils.DatabaseExtension;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.util.ExceptionUtils;
+import org.apache.flink.util.FlinkRuntimeException;
+
+import org.testcontainers.containers.MSSQLServerContainer;
+import org.testcontainers.utility.DockerImageName;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.Statement;
+
+/** A SqlServer database for testing. */
+public class SqlServerDatabase extends DatabaseExtension implements SqlServerImages {
+
+ private static final MSSQLServerContainer<?> CONTAINER =
+ new SqlServerContainer(MSSQL_AZURE_SQL_EDGE).acceptLicense().withXa();
+
+ private static SqlServerMetadata metadata;
+
+ public static SqlServerMetadata getMetadata() {
+ if (!CONTAINER.isRunning()) {
+ throw new FlinkRuntimeException("Container is stopped.");
+ }
+ if (metadata == null) {
+ metadata = new SqlServerMetadata(CONTAINER, true);
+ }
+ return metadata;
+ }
+
+ @Override
+ protected DatabaseMetadata startDatabase() throws Exception {
+ CONTAINER.start();
+ return getMetadata();
+ }
+
+ @Override
+ protected void stopDatabase() throws Exception {
+ CONTAINER.stop();
+ metadata = null;
+ }
+
+ /** {@link MSSQLServerContainer} with Xa. */
+ public static class SqlServerContainer extends MSSQLServerContainer<SqlServerContainer> {
+ private boolean xaActive = false;
+
+ public SqlServerContainer(String dockerImageName) {
+ super(DockerImageName.parse(dockerImageName));
+ }
+
+ public SqlServerContainer(DockerImageName dockerImageName) {
+ super(dockerImageName);
+ }
+
+ public SqlServerContainer withXa() {
+ this.xaActive = true;
+ return this.self();
+ }
+
+ @Override
+ public void start() {
+ super.start();
+ prepareDb();
+ }
+
+ private void prepareDb() {
+ try (Connection connection =
+ DriverManager.getConnection(
+ getJdbcUrl(), getUsername(), getPassword());
+ Statement st = connection.createStatement()) {
+
+ if (xaActive) {
+ st.execute("EXEC sp_sqljdbc_xa_install");
+ }
+ } catch (Exception e) {
+ ExceptionUtils.rethrow(e);
+ }
+ }
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerDatabase.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerImages.java
similarity index 62%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerDatabase.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerImages.java
index 59e1a0a..43263ba 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerDatabase.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerImages.java
@@ -15,33 +15,16 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.sqlserver;
+package org.apache.flink.connector.jdbc.testutils.databases.sqlserver;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.flink.connector.jdbc.databases.DatabaseTest;
-
-import org.testcontainers.containers.MSSQLServerContainer;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
import org.testcontainers.utility.DockerImageName;
-/** A SqlServer database for testing. */
-@Testcontainers
-public interface SqlServerDatabase extends DatabaseTest {
-
+/** SqlServer docker images. */
+public interface SqlServerImages {
DockerImageName MSSQL_AZURE_SQL_EDGE =
DockerImageName.parse("mcr.microsoft.com/azure-sql-edge")
.asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server");
String MSSQL_SERVER_2017 = "mcr.microsoft.com/mssql/server:2017-CU12";
String MSSQL_SERVER_2019 = "mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04";
-
- @Container
- MSSQLServerContainer<?> CONTAINER =
- new MSSQLServerContainer<>(MSSQL_AZURE_SQL_EDGE).acceptLicense();
-
- @Override
- default DatabaseMetadata getMetadata() {
- return new SqlServerMetadata(CONTAINER);
- }
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerMetadata.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerMetadata.java
similarity index 89%
rename from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerMetadata.java
rename to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerMetadata.java
index 94a52f9..86558d0 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/databases/sqlserver/SqlServerMetadata.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/databases/sqlserver/SqlServerMetadata.java
@@ -15,11 +15,11 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.databases.sqlserver;
+package org.apache.flink.connector.jdbc.testutils.databases.sqlserver;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
-import org.postgresql.xa.PGXADataSource;
+import com.microsoft.sqlserver.jdbc.SQLServerXADataSource;
import org.testcontainers.containers.MSSQLServerContainer;
import javax.sql.XADataSource;
@@ -74,8 +74,8 @@
throw new UnsupportedOperationException();
}
- PGXADataSource xaDataSource = new PGXADataSource();
- xaDataSource.setUrl(getJdbcUrl());
+ SQLServerXADataSource xaDataSource = new SQLServerXADataSource();
+ xaDataSource.setURL(getJdbcUrl());
xaDataSource.setUser(getUsername());
xaDataSource.setPassword(getPassword());
return xaDataSource;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/functions/JdbcResultSetBuilder.java
similarity index 65%
copy from flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
copy to flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/functions/JdbcResultSetBuilder.java
index 61314af..d6734be 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/catalog/PostgresTablePathTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/functions/JdbcResultSetBuilder.java
@@ -16,17 +16,15 @@
* limitations under the License.
*/
-package org.apache.flink.connector.jdbc.catalog;
+package org.apache.flink.connector.jdbc.testutils.functions;
-import org.junit.jupiter.api.Test;
+import java.io.Serializable;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.List;
-import static org.assertj.core.api.Assertions.assertThat;
-
-/** Test for {@link PostgresTablePath}. */
-class PostgresTablePathTest {
- @Test
- void testFromFlinkTableName() {
- assertThat(PostgresTablePath.fromFlinkTableName("public.topic"))
- .isEqualTo(new PostgresTablePath("public", "topic"));
- }
+/** ResultSet builder. * */
+@FunctionalInterface
+public interface JdbcResultSetBuilder<T> extends Serializable {
+ List<T> accept(ResultSet rs) throws SQLException;
}
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
new file mode 100644
index 0000000..d8cbd79
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBase.java
@@ -0,0 +1,294 @@
+/*
+ * 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.tables;
+
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+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.functions.JdbcResultSetBuilder;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.api.Schema;
+import org.apache.flink.table.catalog.Column;
+import org.apache.flink.table.catalog.ResolvedSchema;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.table.types.logical.LogicalType;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.util.Preconditions;
+import org.apache.flink.util.function.FunctionWithException;
+
+import java.sql.Connection;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+/** Base table operations. * */
+public abstract class TableBase<T> implements TableManaged {
+
+ private final String name;
+ private final TableField[] fields;
+
+ protected TableBase(String name, 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.fields = fields;
+ }
+
+ protected abstract JdbcResultSetBuilder<T> getResultSetBuilder();
+
+ public String getTableName() {
+ return name;
+ }
+
+ private Stream<TableField> getStreamFields() {
+ return Arrays.stream(this.fields);
+ }
+
+ private Stream<String> getStreamFieldNames() {
+ return getStreamFields().map(TableField::getName);
+ }
+
+ private Stream<DataType> getStreamDataTypes() {
+ return getStreamFields().map(TableField::getDataType);
+ }
+
+ public String[] getTableFields() {
+ return getStreamFieldNames().toArray(String[]::new);
+ }
+
+ public DataTypes.Field[] getTableDataFields() {
+ return getStreamFields()
+ .map(field -> DataTypes.FIELD(field.getName(), field.getDataType()))
+ .toArray(DataTypes.Field[]::new);
+ }
+
+ public DataType[] getTableDataTypes() {
+ return getStreamDataTypes().toArray(DataType[]::new);
+ }
+
+ public RowTypeInfo getTableRowTypeInfo() {
+ TypeInformation<?>[] typesArray =
+ getStreamDataTypes()
+ .map(TypeConversions::fromDataTypeToLegacyInfo)
+ .toArray(TypeInformation[]::new);
+ String[] fieldsArray = getTableFields();
+ return new RowTypeInfo(typesArray, fieldsArray);
+ }
+
+ public RowType getTableRowType() {
+ LogicalType[] typesArray =
+ getStreamDataTypes().map(DataType::getLogicalType).toArray(LogicalType[]::new);
+ String[] fieldsArray = getTableFields();
+ return RowType.of(typesArray, fieldsArray);
+ }
+
+ public int[] getTableTypes() {
+ return getStreamDataTypes()
+ .map(DataType::getLogicalType)
+ .map(LogicalType::getTypeRoot)
+ .map(JdbcTypeUtil::logicalTypeToSqlType)
+ .mapToInt(x -> x)
+ .toArray();
+ }
+
+ public Schema getTableSchema() {
+ Schema.Builder schema = Schema.newBuilder();
+ getStreamFields().forEach(field -> schema.column(field.getName(), field.getDataType()));
+
+ String pkFields =
+ getStreamFields()
+ .filter(TableField::isPkField)
+ .map(TableField::getName)
+ .collect(Collectors.joining(", "));
+ schema.primaryKeyNamed("PRIMARY", pkFields);
+
+ return schema.build();
+ }
+
+ public ResolvedSchema getTableResolvedSchema() {
+ return ResolvedSchema.of(
+ getStreamFields()
+ .map(field -> Column.physical(field.getName(), field.getDataType()))
+ .collect(Collectors.toList()));
+ }
+
+ public String getCreateQuery() {
+ String pkFields =
+ getStreamFields()
+ .filter(TableField::isPkField)
+ .map(TableField::getName)
+ .collect(Collectors.joining(", "));
+ return String.format(
+ "CREATE TABLE %s (%s%s)",
+ name,
+ getStreamFields().map(TableField::asString).collect(Collectors.joining(", ")),
+ pkFields.isEmpty() ? "" : String.format(", PRIMARY KEY (%s)", pkFields));
+ }
+
+ public String getCreateQueryForFlink(DatabaseMetadata metadata, String newName) {
+ return getCreateQueryForFlink(metadata, newName, Collections.emptyList());
+ }
+
+ public String getCreateQueryForFlink(
+ DatabaseMetadata metadata, String newName, List<String> withParams) {
+ return getCreateQueryForFlink(
+ metadata, newName, Arrays.asList(getTableFields()), withParams);
+ }
+
+ public String getCreateQueryForFlink(
+ DatabaseMetadata metadata,
+ String newName,
+ List<String> newFields,
+ List<String> withParams) {
+
+ Map<String, TableField> fieldsMap =
+ getStreamFields().collect(Collectors.toMap(TableField::getName, f -> f));
+
+ String fields =
+ newFields.stream()
+ .map(fieldsMap::get)
+ .map(field -> String.format("%s %s", field.getName(), field.getDataType()))
+ .collect(Collectors.joining(", "));
+ String pkFields =
+ getStreamFields()
+ .filter(TableField::isPkField)
+ .map(TableField::getName)
+ .collect(Collectors.joining(", "));
+
+ String primaryKey =
+ (pkFields.isEmpty())
+ ? ""
+ : String.format(", PRIMARY KEY (%s) NOT ENFORCED", pkFields);
+
+ List<String> params = new ArrayList<>();
+ params.add("'connector'='jdbc'");
+ params.add(String.format("'table-name'='%s'", getTableName()));
+ params.add(String.format("'url'='%s'", metadata.getJdbcUrl()));
+ params.add(String.format("'username'='%s'", metadata.getUsername()));
+ params.add(String.format("'password'='%s'", metadata.getPassword()));
+ params.addAll(withParams);
+
+ return String.format(
+ "CREATE TABLE %s (%s%s) WITH (%s)",
+ newName, fields, primaryKey, String.join(", ", params));
+ }
+
+ private String getInsertIntoQuery(String... values) {
+ return String.format(
+ "INSERT INTO %s (%s) VALUES %s",
+ name,
+ getStreamFieldNames().collect(Collectors.joining(", ")),
+ Arrays.stream(values)
+ .map(v -> String.format("(%s)", v))
+ .collect(Collectors.joining(",")));
+ }
+
+ public String getInsertIntoQuery() {
+ return getInsertIntoQuery(
+ getStreamFieldNames().map(x -> "?").collect(Collectors.joining(", ")));
+ }
+
+ public String getSelectAllQuery() {
+ return String.format(
+ "SELECT %s FROM %s", getStreamFieldNames().collect(Collectors.joining(", ")), name);
+ }
+
+ protected String getDeleteFromQuery() {
+ return String.format("DELETE FROM %s", name);
+ }
+
+ public String getDropTableQuery() {
+ return String.format("DROP TABLE %s", name);
+ }
+
+ public void createTable(Connection conn) throws SQLException {
+ executeUpdate(conn, getCreateQuery());
+ }
+
+ public void insertIntoTableValues(Connection conn, String... values) throws SQLException {
+ executeUpdate(conn, getInsertIntoQuery(values));
+ }
+
+ public List<T> selectAllTable(DatabaseMetadata metadata) throws SQLException {
+ try (Connection conn = metadata.getConnection()) {
+ return selectAllTable(conn);
+ }
+ }
+
+ public List<T> selectAllTable(Connection conn) throws SQLException {
+ return executeStatement(conn, getSelectAllQuery(), getResultSetBuilder());
+ }
+
+ public void deleteTable(Connection conn) throws SQLException {
+ executeUpdate(conn, getDeleteFromQuery());
+ }
+
+ public void dropTable(Connection conn) throws SQLException {
+ executeUpdate(conn, getDropTableQuery());
+ }
+
+ protected void executeUpdate(Connection conn, String sql) throws SQLException {
+ try (Statement st = conn.createStatement()) {
+ st.executeUpdate(sql);
+ }
+ }
+
+ protected <T> List<T> executeStatement(
+ Connection conn, String sql, JdbcResultSetBuilder<T> rsGetter) throws SQLException {
+ try (Statement st = conn.createStatement();
+ ResultSet rs = st.executeQuery(sql)) {
+ return rsGetter.accept(rs);
+ }
+ }
+
+ protected <T> int[] executeStatement(
+ Connection conn, String sql, JdbcStatementBuilder<T> psSetter, List<T> values)
+ throws SQLException {
+ try (PreparedStatement ps = conn.prepareStatement(sql)) {
+ for (T value : values) {
+ psSetter.accept(ps, value);
+ ps.addBatch();
+ }
+ return ps.executeBatch();
+ }
+ }
+
+ protected <T> T getNullable(ResultSet rs, FunctionWithException<ResultSet, T, SQLException> get)
+ throws SQLException {
+ T value = get.apply(rs);
+ return getNullable(rs, value);
+ }
+
+ protected <T> T getNullable(ResultSet rs, T value) throws SQLException {
+ return rs.wasNull() ? null : value;
+ }
+}
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
new file mode 100644
index 0000000..6a8f80d
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java
@@ -0,0 +1,54 @@
+/*
+ * 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.tables;
+
+import org.apache.flink.table.types.DataType;
+
+/** Table builder. * */
+public final class TableBuilder {
+
+ public static TableRow tableRow(String name, TableField... fields) {
+ return new TableRow(name, fields);
+ }
+
+ public static TableField field(String name, DataType dataType) {
+ return field(name, null, dataType);
+ }
+
+ public static TableField field(String name, TableField.DbType dbType, DataType dataType) {
+ return createField(name, dbType, dataType, false);
+ }
+
+ public static TableField pkField(String name, DataType dataType) {
+ return pkField(name, null, dataType);
+ }
+
+ public static TableField pkField(String name, TableField.DbType dbType, DataType dataType) {
+ return createField(name, dbType, dataType, true);
+ }
+
+ public static TableField.DbType dbType(String type) {
+ return new TableField.DbType(type);
+ }
+
+ private static TableField createField(
+ String name, TableField.DbType dbType, DataType dataType, boolean pkField) {
+ return new TableField(name, dataType, dbType, pkField);
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilderTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilderTest.java
new file mode 100644
index 0000000..bd40ed2
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilderTest.java
@@ -0,0 +1,120 @@
+/*
+ * 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.tables;
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo;
+import org.apache.flink.api.java.typeutils.RowTypeInfo;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.logical.IntType;
+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.junit.jupiter.api.Test;
+
+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.pkField;
+import static org.apache.flink.connector.jdbc.testutils.tables.TableBuilder.tableRow;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+class TableBuilderTest {
+
+ TableBase<?> table =
+ tableRow(
+ "test",
+ pkField("id", DataTypes.INT().notNull()),
+ field("name", DataTypes.VARCHAR(10)));
+
+ @Test
+ void testTableCreationFails() {
+ assertThrows(
+ IllegalArgumentException.class, () -> tableRow(""), "Table name must be defined");
+ assertThrows(
+ IllegalArgumentException.class,
+ () -> tableRow("test"),
+ "Table fields must be defined");
+ }
+
+ @Test
+ void testTableCreationOk() {
+ assertEquals("test", table.getTableName());
+ assertEquals(2, table.getTableFields().length);
+ assertEquals("id", table.getTableFields()[0]);
+ assertEquals("name", table.getTableFields()[1]);
+ }
+
+ @Test
+ void testQueryCreation() {
+ String expected = "CREATE TABLE test (id INT NOT NULL, name VARCHAR(10), PRIMARY KEY (id))";
+ assertEquals(expected, table.getCreateQuery());
+ }
+
+ @Test
+ void testQueryCreationWithDbType() {
+ TableRow table =
+ tableRow(
+ "test",
+ pkField("id", dbType("DOUBLE").notNull(), DataTypes.FLOAT().notNull()),
+ field("type", dbType("REAL"), DataTypes.FLOAT()));
+ String expected = "CREATE TABLE test (id DOUBLE NOT NULL, type REAL, PRIMARY KEY (id))";
+ assertEquals(expected, table.getCreateQuery());
+ }
+
+ @Test
+ void testQueryInsertInto() {
+ String expected = "INSERT INTO test (id, name) VALUES (?, ?)";
+ assertEquals(expected, table.getInsertIntoQuery());
+ }
+
+ @Test
+ void testQuerySelectAll() {
+ String expected = "SELECT id, name FROM test";
+ assertEquals(expected, table.getSelectAllQuery());
+ }
+
+ @Test
+ void testQueryDeleteFrom() {
+ String expected = "DELETE FROM test";
+ assertEquals(expected, table.getDeleteFromQuery());
+ }
+
+ @Test
+ void testQueryDropTable() {
+ String expected = "DROP TABLE test";
+ assertEquals(expected, table.getDropTableQuery());
+ }
+
+ @Test
+ void testRowTypeInfo() {
+ RowTypeInfo expected =
+ new RowTypeInfo(BasicTypeInfo.INT_TYPE_INFO, BasicTypeInfo.STRING_TYPE_INFO);
+ assertEquals(expected, table.getTableRowTypeInfo());
+ }
+
+ @Test
+ void testRowType() {
+ RowType expected =
+ RowType.of(
+ new LogicalType[] {new IntType(false), new VarCharType(10)},
+ new String[] {"id", "name"});
+
+ assertEquals(expected, table.getTableRowType());
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableField.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableField.java
new file mode 100644
index 0000000..97e7721
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableField.java
@@ -0,0 +1,82 @@
+/*
+ * 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.tables;
+
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.util.Preconditions;
+
+/** Table field. * */
+public class TableField {
+ private final String name;
+ private final DbType dbType;
+ private final DataType dataType;
+ private final boolean pkField;
+
+ protected TableField(String name, DataType dataType, DbType dbType, boolean pkField) {
+ Preconditions.checkNotNull(name, "Column name can not be null.");
+ Preconditions.checkNotNull(dataType, "Column data type can not be null.");
+ this.name = name;
+ this.dataType = dataType;
+ this.dbType = dbType;
+ this.pkField = pkField;
+ }
+
+ public String getName() {
+ return this.name;
+ }
+
+ public DataType getDataType() {
+ return this.dataType;
+ }
+
+ public boolean isPkField() {
+ return pkField;
+ }
+
+ public String asString() {
+ String fieldType =
+ (this.dbType != null) ? this.dbType.toString() : this.dataType.toString();
+ return String.format("%s %s", this.name, fieldType);
+ }
+
+ @Override
+ public String toString() {
+ return asString();
+ }
+
+ /** Field definition for database. */
+ public static class DbType {
+ private final String type;
+ private Boolean nullable = true;
+
+ public DbType(String type) {
+ this.type = type;
+ }
+
+ public DbType notNull() {
+ this.nullable = false;
+ return this;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("%s%s", this.type, this.nullable ? "" : " NOT NULL");
+ }
+ }
+}
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
new file mode 100644
index 0000000..87308fb
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableRow.java
@@ -0,0 +1,121 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.connector.jdbc.testutils.tables;
+
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.functions.JdbcResultSetBuilder;
+import org.apache.flink.connector.jdbc.utils.JdbcTypeUtil;
+import org.apache.flink.table.api.DataTypes;
+import org.apache.flink.table.types.DataType;
+import org.apache.flink.types.Row;
+
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Row table. * */
+public class TableRow extends TableBase<Row> {
+
+ protected TableRow(String name, TableField[] fields) {
+ super(name, fields);
+ }
+
+ protected JdbcResultSetBuilder<Row> getResultSetBuilder() {
+ return (rs) -> {
+ List<Row> result = new ArrayList<>();
+ DataTypes.Field[] fields = getTableDataFields();
+ while (rs.next()) {
+ Row row = new Row(fields.length);
+ for (int i = 0; i < fields.length; i++) {
+ Object dbValue;
+ Class<?> conversionClass = fields[i].getDataType().getConversionClass();
+ if (conversionClass.equals(LocalTime.class)) {
+ dbValue = rs.getTime(i + 1);
+ } else if (conversionClass.equals(LocalDate.class)) {
+ dbValue = rs.getDate(i + 1);
+ } else if (conversionClass.equals(LocalDateTime.class)) {
+ dbValue = rs.getTimestamp(i + 1);
+ } else {
+ dbValue = rs.getObject(i + 1, conversionClass);
+ }
+ row.setField(i, getNullable(rs, dbValue));
+ }
+ result.add(row);
+ }
+ return result;
+ };
+ }
+
+ private final JdbcStatementBuilder<Row> statementBuilder =
+ (ps, row) -> {
+ DataTypes.Field[] fields = getTableDataFields();
+ for (int i = 0; i < row.getArity(); i++) {
+ DataType type = fields[i].getDataType();
+ int dbType =
+ JdbcTypeUtil.logicalTypeToSqlType(type.getLogicalType().getTypeRoot());
+ if (row.getField(i) == null) {
+ ps.setNull(i + 1, dbType);
+ } else {
+ if (type.getConversionClass().equals(LocalTime.class)) {
+ Time time = Time.valueOf(row.<LocalTime>getFieldAs(i));
+ ps.setTime(i + 1, time);
+ } else if (type.getConversionClass().equals(LocalDate.class)) {
+ ps.setDate(i + 1, Date.valueOf(row.<LocalDate>getFieldAs(i)));
+ } else if (type.getConversionClass().equals(LocalDateTime.class)) {
+ ps.setTimestamp(
+ i + 1, Timestamp.valueOf(row.<LocalDateTime>getFieldAs(i)));
+ } else {
+ ps.setObject(i + 1, row.getField(i));
+ }
+ }
+ }
+ };
+
+ public void insertIntoTableValues(Connection conn, List<Row> values) throws SQLException {
+ executeStatement(conn, getInsertIntoQuery(), statementBuilder, values);
+ }
+
+ public void checkContent(DatabaseMetadata metadata, Row... content) throws SQLException {
+ try (Connection dbConn = metadata.getConnection()) {
+ String[] results =
+ selectAllTable(dbConn).stream()
+ .map(Row::toString)
+ .sorted()
+ .toArray(String[]::new);
+
+ assertThat(results)
+ .isEqualTo(
+ Arrays.stream(content)
+ .map(Row::toString)
+ .sorted()
+ .toArray(String[]::new));
+ }
+ }
+}
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
new file mode 100644
index 0000000..c1bb4d8
--- /dev/null
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/templates/BooksTable.java
@@ -0,0 +1,178 @@
+/*
+ * 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.tables.templates;
+
+import org.apache.flink.connector.jdbc.JdbcStatementBuilder;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.functions.JdbcResultSetBuilder;
+import org.apache.flink.connector.jdbc.testutils.tables.TableBase;
+import org.apache.flink.connector.jdbc.testutils.tables.TableField;
+
+import org.apache.commons.lang3.builder.EqualsBuilder;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static java.lang.String.format;
+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.pkField;
+import static org.apache.flink.table.api.DataTypes.DOUBLE;
+import static org.apache.flink.table.api.DataTypes.INT;
+import static org.apache.flink.table.api.DataTypes.VARCHAR;
+
+/** Book table template. * */
+public class BooksTable extends TableBase<BooksTable.BookEntry> implements TableManaged {
+
+ private final JdbcStatementBuilder<BookEntry> statementBuilder =
+ (ps, book) -> {
+ ps.setInt(1, book.id);
+ ps.setString(2, book.title);
+ ps.setString(3, book.author);
+ if (book.price == null) {
+ ps.setNull(4, Types.DOUBLE);
+ } else {
+ ps.setDouble(4, book.price);
+ }
+ ps.setInt(5, book.qty);
+ };
+
+ private final JdbcResultSetBuilder<BookEntry> resultSetBuilder =
+ (rs) -> {
+ List<BookEntry> result = new ArrayList<>();
+ while (rs.next()) {
+ result.add(
+ new BookEntry(
+ getNullable(rs, r -> r.getInt(1)),
+ getNullable(rs, r -> r.getString(2)),
+ getNullable(rs, r -> r.getString(3)),
+ getNullable(rs, r -> r.getDouble(4)),
+ getNullable(rs, r -> r.getInt(5))));
+ }
+ return result;
+ };
+
+ public BooksTable(String name) {
+ super(
+ name,
+ Arrays.asList(
+ pkField("id", INT().notNull()),
+ field("title", VARCHAR(50)),
+ field("author", VARCHAR(50)),
+ field("price", dbType("FLOAT"), DOUBLE()),
+ field("qty", INT()))
+ .toArray(new TableField[0]));
+ }
+
+ public String getSelectByIdBetweenQuery() {
+ return format("%s WHERE id BETWEEN ? AND ?", getSelectAllQuery());
+ }
+
+ public String getSelectByAuthorQuery() {
+ return format("%s WHERE author = ?", getSelectAllQuery());
+ }
+
+ public String getSelectAllNoQuantityQuery() {
+ return format("%s WHERE QTY < 0", getSelectAllQuery());
+ }
+
+ // public List<BookEntry> getTestData() {
+ // return Arrays.asList(BooksStore.TEST_DATA);
+ // }
+ // public void insertTableTestData(Connection conn) throws SQLException {
+ // executeStatement(conn, getInsertIntoQuery(), statementBuilder, getTestData());
+ // }
+
+ public JdbcStatementBuilder<BookEntry> getStatementBuilder() {
+ return statementBuilder;
+ }
+
+ @Override
+ protected JdbcResultSetBuilder<BookEntry> getResultSetBuilder() {
+ return resultSetBuilder;
+ }
+
+ public List<BookEntry> selectAllTable(Connection conn) throws SQLException {
+ return executeStatement(conn, getSelectAllQuery(), resultSetBuilder);
+ }
+
+ /** Book table entry. * */
+ public static class BookEntry implements Serializable {
+ public final Integer id;
+ public final String title;
+ public final String author;
+ public final Double price;
+ public final Integer qty;
+
+ public BookEntry(Integer id, String title, String author, Double price, Integer qty) {
+ this.id = id;
+ this.title = title;
+ this.author = author;
+ this.price = price;
+ this.qty = qty;
+ }
+
+ @Override
+ public String toString() {
+ return new ToStringBuilder(this)
+ .append("id", id)
+ .append("title", title)
+ .append("author", author)
+ .append("price", price)
+ .append("qty", qty)
+ .toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+
+ BookEntry bookEntry = (BookEntry) o;
+ return new EqualsBuilder()
+ .append(id, bookEntry.id)
+ .append(title, bookEntry.title)
+ .append(author, bookEntry.author)
+ .append(price, bookEntry.price)
+ .append(qty, bookEntry.qty)
+ .isEquals();
+ }
+
+ @Override
+ public int hashCode() {
+ return new HashCodeBuilder(17, 37)
+ .append(id)
+ .append(title)
+ .append(author)
+ .append(price)
+ .append(qty)
+ .toHashCode();
+ }
+ }
+}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java
index e03a7fa..8f6043b 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcExactlyOnceSinkE2eTest.java
@@ -18,29 +18,27 @@
package org.apache.flink.connector.jdbc.xa;
import org.apache.flink.api.common.functions.RichMapFunction;
+import org.apache.flink.api.common.state.CheckpointListener;
import org.apache.flink.api.common.state.ListState;
import org.apache.flink.api.common.state.ListStateDescriptor;
import org.apache.flink.api.common.time.Time;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.jdbc.JdbcExactlyOnceOptions;
import org.apache.flink.connector.jdbc.JdbcExecutionOptions;
-import org.apache.flink.connector.jdbc.JdbcITCase;
import org.apache.flink.connector.jdbc.JdbcSink;
-import org.apache.flink.connector.jdbc.JdbcTestBase;
-import org.apache.flink.connector.jdbc.JdbcTestFixture.TestEntry;
-import org.apache.flink.runtime.state.CheckpointListener;
+import org.apache.flink.connector.jdbc.testutils.DatabaseTest;
+import org.apache.flink.connector.jdbc.testutils.TableManaged;
+import org.apache.flink.connector.jdbc.testutils.tables.templates.BooksTable;
import org.apache.flink.runtime.state.FunctionInitializationContext;
import org.apache.flink.runtime.state.FunctionSnapshotContext;
import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
import org.apache.flink.streaming.api.CheckpointingMode;
-import org.apache.flink.streaming.api.TimeCharacteristic;
import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction;
import org.apache.flink.streaming.api.functions.source.SourceFunction;
import org.apache.flink.test.junit5.MiniClusterExtension;
import org.apache.flink.util.ExceptionUtils;
-import org.apache.flink.util.function.SerializableSupplier;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Test;
@@ -48,15 +46,14 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
-import javax.sql.XADataSource;
-
import java.time.Duration;
+import java.util.Collections;
import java.util.List;
import java.util.Map;
-import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
@@ -65,24 +62,20 @@
import static org.apache.flink.api.common.restartstrategy.RestartStrategies.fixedDelayRestart;
import static org.apache.flink.configuration.JobManagerOptions.EXECUTION_FAILOVER_STRATEGY;
import static org.apache.flink.configuration.TaskManagerOptions.TASK_CANCELLATION_TIMEOUT;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.INPUT_TABLE;
-import static org.apache.flink.connector.jdbc.JdbcTestFixture.INSERT_TEMPLATE;
import static org.apache.flink.connector.jdbc.xa.JdbcXaFacadeTestHelper.getInsertedIds;
import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_TIMEOUT;
import static org.apache.flink.util.Preconditions.checkState;
import static org.assertj.core.api.Assertions.assertThat;
/** A simple end-to-end test for {@link JdbcXaSinkFunction}. */
-public abstract class JdbcExactlyOnceSinkE2eTest extends JdbcTestBase {
- private static final Random RANDOM = new Random(System.currentTimeMillis());
+public abstract class JdbcExactlyOnceSinkE2eTest implements DatabaseTest {
private static final Logger LOG = LoggerFactory.getLogger(JdbcExactlyOnceSinkE2eTest.class);
+ private static final BooksTable OUTPUT_TABLE = new BooksTable("XaTable");
protected static final int PARALLELISM = 4;
- protected static final long CHECKPOINT_TIMEOUT_MS = 20_000L;
- protected static final long TASK_CANCELLATION_TIMEOUT_MS = 20_000L;
-
- protected abstract SerializableSupplier<XADataSource> getDataSourceSupplier();
+ protected static final long CHECKPOINT_TIMEOUT_MS = 5_000L;
+ protected static final long TASK_CANCELLATION_TIMEOUT_MS = 10_000L;
@RegisterExtension static final MiniClusterExtension MINI_CLUSTER = createCluster();
@@ -102,6 +95,11 @@
.build());
}
+ @Override
+ public List<TableManaged> getManagedTables() {
+ return Collections.singletonList(OUTPUT_TABLE);
+ }
+
// track active sources for:
// 1. if any cancels, cancel others ASAP
// 2. wait for others (to participate in checkpointing)
@@ -113,7 +111,6 @@
private static final Map<Integer, CountDownLatch> inactiveMappers = new ConcurrentHashMap<>();
@AfterEach
- @Override
public void after() {
activeSources.clear();
inactiveMappers.clear();
@@ -124,35 +121,32 @@
long started = System.currentTimeMillis();
LOG.info("Test insert for {}", getMetadata().getVersion());
int elementsPerSource = 50;
- int numElementsPerCheckpoint = 7;
- int minElementsPerFailure = numElementsPerCheckpoint / 3;
- int maxElementsPerFailure = numElementsPerCheckpoint * 3;
+ int numElementsPerCheckpoint = 10;
+ int expectedFailures = elementsPerSource / numElementsPerCheckpoint;
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(PARALLELISM);
- env.setRestartStrategy(fixedDelayRestart(elementsPerSource * 2, Time.milliseconds(100)));
- env.setStreamTimeCharacteristic(TimeCharacteristic.ProcessingTime);
+ env.setRestartStrategy(fixedDelayRestart(expectedFailures * 2, Time.milliseconds(100)));
+ env.getConfig().setAutoWatermarkInterval(0L);
env.enableCheckpointing(50, CheckpointingMode.EXACTLY_ONCE);
- // timeout checkpoints as some tasks may fail while triggering
- env.getCheckpointConfig().setCheckpointTimeout(1000);
// NOTE: keep operator chaining enabled to prevent memory exhaustion by sources while maps
// are still initializing
env.addSource(new TestEntrySource(elementsPerSource, numElementsPerCheckpoint))
.setParallelism(PARALLELISM)
- .map(new FailingMapper(minElementsPerFailure, maxElementsPerFailure))
+ .map(new FailingMapper(numElementsPerCheckpoint + (numElementsPerCheckpoint / 2)))
.addSink(
JdbcSink.exactlyOnceSink(
- String.format(INSERT_TEMPLATE, INPUT_TABLE),
- JdbcITCase.TEST_ENTRY_JDBC_STATEMENT_BUILDER,
+ OUTPUT_TABLE.getInsertIntoQuery(),
+ OUTPUT_TABLE.getStatementBuilder(),
JdbcExecutionOptions.builder().withMaxRetries(0).build(),
JdbcExactlyOnceOptions.builder()
.withTransactionPerConnection(true)
.build(),
- this.getDataSourceSupplier()));
+ getMetadata().getXaSourceSupplier()));
env.execute();
- List<Integer> insertedIds = getInsertedIds(getMetadata(), INPUT_TABLE);
+ List<Integer> insertedIds = getInsertedIds(getMetadata(), OUTPUT_TABLE.getTableName());
List<Integer> expectedIds =
IntStream.range(0, elementsPerSource * PARALLELISM)
.boxed()
@@ -160,14 +154,18 @@
assertThat(insertedIds)
.as(insertedIds.toString())
.containsExactlyInAnyOrderElementsOf(expectedIds);
+
LOG.info(
"Test insert for {} finished in {} ms.",
getMetadata().getVersion(),
System.currentTimeMillis() - started);
}
- /** {@link SourceFunction} emits {@link TestEntry test entries} and waits for the checkpoint. */
- private static class TestEntrySource extends RichParallelSourceFunction<TestEntry>
+ /**
+ * {@link SourceFunction} emits {@link BooksTable.BookEntry test entries} and waits for the
+ * checkpoint.
+ */
+ private static class TestEntrySource extends RichParallelSourceFunction<BooksTable.BookEntry>
implements CheckpointListener, CheckpointedFunction {
private final int numElements;
private final int numElementsPerCheckpoint;
@@ -184,14 +182,14 @@
}
@Override
- public void run(SourceContext<TestEntry> ctx) throws Exception {
+ public void run(SourceContext<BooksTable.BookEntry> ctx) throws Exception {
try {
waitForConsumers();
for (SourceRange range : ranges.get()) {
emitRange(range, ctx);
}
} finally {
- activeSources.get(getRuntimeContext().getAttemptNumber()).countDown();
+ getActiveSources().countDown();
}
waitOtherSources(); // participate in checkpointing
}
@@ -203,7 +201,7 @@
inactiveMappers.get(getRuntimeContext().getAttemptNumber()).await();
}
- private void emitRange(SourceRange range, SourceContext<TestEntry> ctx) {
+ private void emitRange(SourceRange range, SourceContext<BooksTable.BookEntry> ctx) {
for (int i = range.from; i < range.to && running; ) {
int count = Math.min(range.to - i, numElementsPerCheckpoint);
emit(i, count, range, ctx);
@@ -212,14 +210,17 @@
}
private void emit(
- int start, int count, SourceRange toAdvance, SourceContext<TestEntry> ctx) {
+ int start,
+ int count,
+ SourceRange toAdvance,
+ SourceContext<BooksTable.BookEntry> ctx) {
synchronized (ctx.getCheckpointLock()) {
lastCheckpointId = -1L;
lastSnapshotConfirmed = false;
for (int j = start; j < start + count && running; j++) {
try {
ctx.collect(
- new TestEntry(
+ new BooksTable.BookEntry(
j,
Integer.toString(j),
Integer.toString(j),
@@ -277,7 +278,6 @@
}
private void sleep(Supplier<Boolean> condition) {
- long start = System.currentTimeMillis();
while (condition.get()
&& running
&& !Thread.currentThread().isInterrupted()
@@ -292,16 +292,17 @@
}
private void waitOtherSources() throws InterruptedException {
- long start = System.currentTimeMillis();
while (running && haveActiveSources()) {
- activeSources
- .get(getRuntimeContext().getAttemptNumber())
- .await(100, TimeUnit.MILLISECONDS);
+ getActiveSources().await(100, TimeUnit.MILLISECONDS);
}
}
+ private CountDownLatch getActiveSources() {
+ return activeSources.get(getRuntimeContext().getAttemptNumber());
+ }
+
private boolean haveActiveSources() {
- return activeSources.get(getRuntimeContext().getAttemptNumber()).getCount() > 0;
+ return getActiveSources().getCount() > 0;
}
private static final class SourceRange {
@@ -330,19 +331,17 @@
}
}
- private static class FailingMapper extends RichMapFunction<TestEntry, TestEntry> {
- private final int minElementsPerFailure;
- private final int maxElementsPerFailure;
- private transient int remaining;
+ private static class FailingMapper
+ extends RichMapFunction<BooksTable.BookEntry, BooksTable.BookEntry> {
+ private final int failingMessage;
+ private transient AtomicInteger counter;
- public FailingMapper(int minElementsPerFailure, int maxElementsPerFailure) {
- this.minElementsPerFailure = minElementsPerFailure;
- this.maxElementsPerFailure = maxElementsPerFailure;
+ public FailingMapper(int failingMessage) {
+ this.failingMessage = failingMessage;
}
@Override
public void open(Configuration parameters) throws Exception {
- remaining = minElementsPerFailure + RANDOM.nextInt(maxElementsPerFailure);
inactiveMappers
.computeIfAbsent(
getRuntimeContext().getAttemptNumber(),
@@ -350,12 +349,13 @@
new CountDownLatch(
getRuntimeContext().getNumberOfParallelSubtasks()))
.countDown();
- LOG.debug("Mapper will fail after {} records", remaining);
+ counter = new AtomicInteger(failingMessage);
+ LOG.debug("Mapper will fail after {} records.", failingMessage);
}
@Override
- public TestEntry map(TestEntry value) throws Exception {
- if (--remaining <= 0) {
+ public BooksTable.BookEntry map(BooksTable.BookEntry value) throws Exception {
+ if (counter.getAndDecrement() <= 0) {
LOG.debug("Mapper failing intentionally.");
throw new TestException();
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeImplTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeImplTest.java
index 970ed24..12d3762 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeImplTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeImplTest.java
@@ -19,7 +19,6 @@
import org.apache.flink.connector.jdbc.JdbcTestBase;
import org.apache.flink.connector.jdbc.JdbcTestFixture;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
import org.junit.jupiter.api.Test;
@@ -97,9 +96,4 @@
verify(connection).close();
verify(xaConnection).close();
}
-
- @Override
- public DatabaseMetadata getMetadata() {
- return JdbcTestFixture.DERBY_EBOOKSHOP_DB;
- }
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeTestHelper.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeTestHelper.java
index 9d4dfc3..b167d62 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeTestHelper.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaFacadeTestHelper.java
@@ -18,7 +18,7 @@
package org.apache.flink.connector.jdbc.xa;
import org.apache.flink.connector.jdbc.JdbcTestCheckpoint;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
import java.sql.Connection;
import java.sql.ResultSet;
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkDerbyTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkDerbyTest.java
index 19b657d..1f899af 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkDerbyTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkDerbyTest.java
@@ -18,9 +18,7 @@
package org.apache.flink.connector.jdbc.xa;
import org.apache.flink.connector.jdbc.JdbcTestFixture;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
-import org.apache.derby.jdbc.EmbeddedXADataSource;
import org.junit.jupiter.api.Test;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.TEST_DATA;
@@ -165,13 +163,4 @@
sinkHelper.emit(TEST_DATA[0]); // duplicate
assertThatThrownBy(() -> sinkHelper.snapshotState(0)).isInstanceOf(Exception.class);
}
-
- static EmbeddedXADataSource derbyXaDs() {
- return (EmbeddedXADataSource) JdbcTestFixture.DERBY_EBOOKSHOP_DB.buildXaDataSource();
- }
-
- @Override
- public DatabaseMetadata getMetadata() {
- return JdbcTestFixture.DERBY_EBOOKSHOP_DB;
- }
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkH2Test.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkH2Test.java
index 38a904a..8045ace 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkH2Test.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkH2Test.java
@@ -18,8 +18,10 @@
package org.apache.flink.connector.jdbc.xa;
import org.apache.flink.connector.jdbc.JdbcTestFixture;
-import org.apache.flink.connector.jdbc.databases.h2.H2XaDatabase;
-import org.apache.flink.connector.jdbc.databases.h2.xa.H2XaDsWrapper;
+import org.apache.flink.connector.jdbc.databases.h2.H2XaTestBase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.h2.H2XaDatabase;
+import org.apache.flink.connector.jdbc.testutils.databases.h2.xa.H2XaDsWrapper;
import org.junit.jupiter.api.Test;
@@ -31,7 +33,12 @@
* transaction is not yet committed). But XA support isn't full, so for some scenarios {@link
* H2XaDsWrapper wrapper} is used, and for some - Derby.
*/
-class JdbcXaSinkH2Test extends JdbcXaSinkTestBase implements H2XaDatabase {
+class JdbcXaSinkH2Test extends JdbcXaSinkTestBase implements H2XaTestBase {
+
+ @Override
+ public DatabaseMetadata getMetadata() {
+ return H2XaDatabase.getMetadata();
+ }
@Test
void testIgnoreDuplicatedNotification() throws Exception {
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkMigrationTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkMigrationTest.java
index e4199a9..19cb0ea 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkMigrationTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkMigrationTest.java
@@ -23,18 +23,16 @@
import org.apache.flink.connector.jdbc.JdbcTestBase;
import org.apache.flink.connector.jdbc.JdbcTestFixture;
import org.apache.flink.connector.jdbc.JdbcTestFixture.TestEntry;
-import org.apache.flink.connector.jdbc.databases.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.derby.DerbyDatabase;
import org.apache.flink.runtime.checkpoint.OperatorSubtaskState;
import org.apache.flink.streaming.api.operators.StreamSink;
import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness;
-import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
-import org.apache.flink.testutils.junit.extensions.parameterized.Parameters;
import org.apache.flink.util.Preconditions;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.TestTemplate;
-import org.junit.jupiter.api.extension.ExtendWith;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
import javax.transaction.xa.Xid;
@@ -50,13 +48,11 @@
import static org.apache.flink.connector.jdbc.JdbcTestFixture.CP0;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.cleanUpDatabasesStatic;
import static org.apache.flink.connector.jdbc.JdbcTestFixture.initSchema;
-import static org.apache.flink.connector.jdbc.xa.JdbcXaSinkDerbyTest.derbyXaDs;
import static org.apache.flink.connector.jdbc.xa.JdbcXaSinkTestBase.buildInitCtx;
import static org.apache.flink.streaming.util.OperatorSnapshotUtil.readStateHandle;
import static org.apache.flink.streaming.util.OperatorSnapshotUtil.writeStateHandle;
/** Tests state migration for {@link JdbcXaSinkFunction}. */
-@ExtendWith(ParameterizedTestExtension.class)
public class JdbcXaSinkMigrationTest extends JdbcTestBase {
// write a snapshot:
@@ -65,23 +61,19 @@
// mvn exec:java -Dexec.mainClass="<CLASS_NAME>" -Dexec.args='<VERSION>'
// -Dexec.classpathScope=test -Dexec.cleanupDaemonThreads=false
public static void main(String[] args) throws Exception {
- writeSnapshot(parseVersionArg(args));
+ new DerbyDatabase().startDatabase();
+ JdbcXaSinkMigrationTest test = new JdbcXaSinkMigrationTest();
+ test.writeSnapshot(parseVersionArg(args));
}
- @Parameters
public static Collection<FlinkVersion> getReadVersions() {
return Collections.emptyList();
}
- public JdbcXaSinkMigrationTest(FlinkVersion readVersion) {
- this.readVersion = readVersion;
- }
-
- private final FlinkVersion readVersion;
-
- @TestTemplate
+ @ParameterizedTest
+ @MethodSource("getReadVersions")
@Disabled // as getReadVersions is empty and fails
- void testCommitFromSnapshot() throws Exception {
+ void testCommitFromSnapshot(FlinkVersion readVersion) throws Exception {
preparePendingTransaction();
try (OneInputStreamOperatorTestHarness<TestEntry, Object> harness =
createHarness(buildSink())) {
@@ -99,11 +91,6 @@
cancelAllTx();
}
- @Override
- public DatabaseMetadata getMetadata() {
- return JdbcTestFixture.DERBY_EBOOKSHOP_DB;
- }
-
private void preparePendingTransaction() throws Exception {
try (JdbcXaSinkTestHelper sinkHelper =
new JdbcXaSinkTestHelper(buildSink(), new XaSinkStateHandlerImpl())) {
@@ -113,7 +100,7 @@
}
}
- private static OperatorSubtaskState captureState() throws Exception {
+ private OperatorSubtaskState captureState() throws Exception {
try (JdbcXaSinkTestHelper sinkHelper =
new JdbcXaSinkTestHelper(buildSink(), new XaSinkStateHandlerImpl())) {
try (OneInputStreamOperatorTestHarness<TestEntry, Object> harness =
@@ -141,7 +128,7 @@
};
}
- private static String getSnapshotPath(FlinkVersion version) {
+ private String getSnapshotPath(FlinkVersion version) {
return String.format(
"src/test/resources/jdbc-exactly-once-sink-migration-%s-snapshot", version);
}
@@ -164,33 +151,34 @@
+ Arrays.toString(FlinkVersion.values())));
}
- private static JdbcXaSinkFunction<TestEntry> buildSink() {
+ private JdbcXaSinkFunction<TestEntry> buildSink() {
return JdbcXaSinkTestBase.buildSink(
getXidGenerator(),
- XaFacadeImpl.fromXaDataSource(derbyXaDs()),
+ XaFacadeImpl.fromXaDataSource(getMetadata().buildXaDataSource()),
new XaSinkStateHandlerImpl(new XaSinkStateSerializer()),
1);
}
- private static void cancelAllTx() throws Exception {
+ private void cancelAllTx() throws Exception {
try (JdbcXaFacadeTestHelper xa =
- new JdbcXaFacadeTestHelper(
- JdbcTestFixture.DERBY_EBOOKSHOP_DB, JdbcTestFixture.INPUT_TABLE)) {
+ new JdbcXaFacadeTestHelper(getMetadata(), JdbcTestFixture.INPUT_TABLE)) {
xa.cancelAllTx();
}
}
- private static void writeSnapshot(FlinkVersion v) throws Exception {
- String path = getSnapshotPath(v);
+ private void writeSnapshot(FlinkVersion flinkVersion) throws Exception {
+ String path = getSnapshotPath(flinkVersion);
+
+ // Files.createFile(Paths.get(path));/
Preconditions.checkArgument(
!Files.exists(Paths.get(path)),
- String.format("snapshot for version %s already exist: %s", v, path));
- initSchema(JdbcTestFixture.DERBY_EBOOKSHOP_DB);
+ String.format("snapshot for version %s already exist: %s", flinkVersion, path));
+ initSchema(getMetadata());
try {
writeStateHandle(captureState(), path);
} finally {
cancelAllTx();
- cleanUpDatabasesStatic(JdbcTestFixture.DERBY_EBOOKSHOP_DB);
+ cleanUpDatabasesStatic(getMetadata());
}
}
diff --git a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkNoInsertionTest.java b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkNoInsertionTest.java
index 9c028d1..d9a2c06 100644
--- a/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkNoInsertionTest.java
+++ b/flink-connector-jdbc/src/test/java/org/apache/flink/connector/jdbc/xa/JdbcXaSinkNoInsertionTest.java
@@ -18,7 +18,9 @@
package org.apache.flink.connector.jdbc.xa;
import org.apache.flink.connector.jdbc.JdbcTestFixture;
-import org.apache.flink.connector.jdbc.databases.h2.H2XaDatabase;
+import org.apache.flink.connector.jdbc.databases.h2.H2XaTestBase;
+import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata;
+import org.apache.flink.connector.jdbc.testutils.databases.h2.H2XaDatabase;
import org.junit.jupiter.api.Test;
@@ -26,7 +28,12 @@
import static org.assertj.core.api.Assertions.assertThat;
/** Tests that data is not inserted ahead of time. */
-class JdbcXaSinkNoInsertionTest extends JdbcXaSinkTestBase implements H2XaDatabase {
+class JdbcXaSinkNoInsertionTest extends JdbcXaSinkTestBase implements H2XaTestBase {
+
+ @Override
+ public DatabaseMetadata getMetadata() {
+ return H2XaDatabase.getMetadata();
+ }
@Test
void testNoInsertAfterInvoke() throws Exception {
diff --git a/flink-connector-jdbc/src/test/resources/mysql-scripts/catalog-init-for-test.sql b/flink-connector-jdbc/src/test/resources/mysql-scripts/catalog-init-for-test.sql
deleted file mode 100644
index 98f44b4..0000000
--- a/flink-connector-jdbc/src/test/resources/mysql-scripts/catalog-init-for-test.sql
+++ /dev/null
@@ -1,129 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-/**
-* The test for mysql 5.6.X & 5.7.X & 8.0.X versions.
-* The init script contains some types that are incompatible with lower versions.
-*/
-
--- Creates test user info and grants privileges.
-CREATE USER 'mysql'@'%' IDENTIFIED BY 'mysql';
-GRANT ALL ON *.* TO 'mysql'@'%';
-FLUSH PRIVILEGES;
-
--- Create the `test` database.
-DROP DATABASE IF EXISTS `test`;
-CREATE DATABASE `test` CHARSET=utf8;
-
--- Uses `test` database.
-use `test`;
-
--- Create test tables.
--- ----------------------------
--- Table structure for t_all_types
--- ----------------------------
-DROP TABLE IF EXISTS `t_all_types`;
-CREATE TABLE `t_all_types` (
- `pid` bigint(20) NOT NULL AUTO_INCREMENT,
- `col_bigint` bigint(20) DEFAULT NULL,
- `col_bigint_unsigned` bigint(20) unsigned DEFAULT NULL,
- `col_binary` binary(100) DEFAULT NULL,
- `col_bit` bit(1) DEFAULT NULL,
- `col_blob` blob,
- `col_char` char(10) DEFAULT NULL,
- `col_date` date DEFAULT NULL,
- `col_datetime` datetime DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
- `col_decimal` decimal(10,0) DEFAULT NULL,
- `col_decimal_unsigned` decimal(10,0) unsigned DEFAULT NULL,
- `col_double` double DEFAULT NULL,
- `col_double_unsigned` double unsigned DEFAULT NULL,
- `col_enum` enum('enum1','enum2','enum11') DEFAULT NULL,
- `col_float` float DEFAULT NULL,
- `col_float_unsigned` float unsigned DEFAULT NULL,
- `col_int` int(11) DEFAULT NULL,
- `col_int_unsigned` int(10) unsigned DEFAULT NULL,
- `col_integer` int(11) DEFAULT NULL,
- `col_integer_unsigned` int(10) unsigned DEFAULT NULL,
- `col_longblob` longblob,
- `col_longtext` longtext CHARACTER SET utf8mb4 COLLATE utf8mb4_bin,
- `col_mediumblob` mediumblob,
- `col_mediumint` mediumint(9) DEFAULT NULL,
- `col_mediumint_unsigned` mediumint(8) unsigned DEFAULT NULL,
- `col_mediumtext` mediumtext,
- `col_numeric` decimal(10,0) DEFAULT NULL,
- `col_numeric_unsigned` decimal(10,0) unsigned DEFAULT NULL,
- `col_real` double DEFAULT NULL,
- `col_real_unsigned` double unsigned DEFAULT NULL,
- `col_set` set('set_ele1','set_ele12') DEFAULT NULL,
- `col_smallint` smallint(6) DEFAULT NULL,
- `col_smallint_unsigned` smallint(5) unsigned DEFAULT NULL,
- `col_text` text,
- `col_time` time DEFAULT NULL,
- `col_timestamp` timestamp NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP,
- `col_tinytext` tinytext,
- `col_tinyint` tinyint DEFAULT NULL,
- `col_tinyint_unsinged` tinyint(255) unsigned DEFAULT NULL,
- `col_tinyblob` tinyblob,
- `col_varchar` varchar(255) DEFAULT NULL,
- `col_datetime_p3` datetime(3) NOT NULL DEFAULT CURRENT_TIMESTAMP(3) ON UPDATE CURRENT_TIMESTAMP(3) COMMENT 'Test for precision value in default and user define precision',
- `col_time_p3` time(3) DEFAULT NULL COMMENT 'Test for precision value in default and user define precision',
- `col_timestamp_p3` timestamp(3) NULL DEFAULT CURRENT_TIMESTAMP(3) ON UPDATE CURRENT_TIMESTAMP(3) COMMENT 'Test for precision value in default and user define precision',
- `col_varbinary` varbinary(255) DEFAULT NULL,
- PRIMARY KEY (`pid`) USING BTREE
-) ENGINE=InnoDB AUTO_INCREMENT=3 DEFAULT CHARSET=utf8;
-
--- ----------------------------
--- Records of t_all_types
--- ----------------------------
-INSERT INTO `t_all_types` VALUES (1, -1, 1, null, b'1', null, 'hello', '2021-08-04', '2021-08-04 01:54:16', -1, 1, -1, 1, 'enum2', -9.1, 9.1, -1, 1, -1, 1, null, 'col_longtext', null, -1, 1, 'col_mediumtext', -99, 99, -1, 1, 'set_ele1', -1, 1, 'col_text', '10:32:34', '2021-08-04 01:54:16', 'col_tinytext', -1, 1, null, 'col_varchar', '2021-08-04 01:54:16.463', '09:33:43.000', '2021-08-04 01:54:16.463', null);
-INSERT INTO `t_all_types` VALUES (2, -1, 1, null, b'1', null, 'hello', '2021-08-04', '2021-08-04 01:53:19', -1, 1, -1, 1, 'enum2', -9.1, 9.1, -1, 1, -1, 1, null, 'col_longtext', null, -1, 1, 'col_mediumtext', -99, 99, -1, 1, 'set_ele1,set_ele12', -1, 1, 'col_text', '10:32:34', '2021-08-04 01:53:19', 'col_tinytext', -1, 1, null, 'col_varchar', '2021-08-04 01:53:19.098', '09:33:43.000', '2021-08-04 01:53:19.098', null);
-
--- Create test table t_all_types_sink.
-DROP TABLE IF EXISTS `t_all_types_sink`;
-CREATE TABLE `t_all_types_sink` select * from t_all_types where 1=2;
-
--- Create test table t_grouped_by_sink.
-DROP TABLE IF EXISTS `t_grouped_by_sink`;
-CREATE TABLE `t_grouped_by_sink` (
- `pid` bigint(20) NOT NULL AUTO_INCREMENT,
- `col_bigint` bigint(20) DEFAULT NULL,
- PRIMARY KEY (`pid`) USING BTREE
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
--- Create test table t_pk.
-DROP TABLE IF EXISTS `t_pk`;
-CREATE TABLE `t_pk` (
- `uid` bigint(20) NOT NULL AUTO_INCREMENT,
- `col_bigint` bigint(20) DEFAULT NULL,
- PRIMARY KEY (`uid`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;
-
--- Create the `test2` database.
-DROP DATABASE IF EXISTS `test2`;
-CREATE DATABASE `test2` CHARSET=utf8;
-
--- Uses `test2` database.
-use `test2`;
-
--- Create test table t_pk.
-DROP TABLE IF EXISTS `t_pk`;
-CREATE TABLE `t_pk` (
- `pid` int(11) NOT NULL AUTO_INCREMENT,
- `col_varchar` varchar(255) DEFAULT NULL,
- PRIMARY KEY (`pid`)
-) ENGINE=InnoDB DEFAULT CHARSET=utf8;