├── .asf.yaml ├── .github ├── boring-cyborg.yml └── workflows │ ├── backwards_compatibility.yml │ ├── push_pr.yml │ └── weekly.yml ├── .gitignore ├── .gitmodules ├── .idea └── vcs.xml ├── LICENSE ├── NOTICE ├── README.md ├── docs ├── content.zh │ └── docs │ │ └── connectors │ │ ├── datastream │ │ └── jdbc.md │ │ └── table │ │ └── jdbc.md ├── content │ └── docs │ │ └── connectors │ │ ├── datastream │ │ └── jdbc.md │ │ └── table │ │ └── jdbc.md └── data │ └── jdbc.yml ├── flink-connector-jdbc-architecture ├── archunit-violations │ ├── 10da9e19-442f-49fb-b340-a2e401062d33 │ ├── 3547d327-0923-4fa7-8319-b18102cc22d0 │ ├── 47967139-2deb-41b4-91c3-b19e56e5fc76 │ ├── 7d5a32d5-922b-4760-825b-c30427dc987e │ ├── afc13e03-aa50-408b-b87e-aa491cf0b8cb │ ├── b97067ac-f37f-4793-8578-bdbe6d2abf5b │ ├── e391ec0f-7ba6-4b7e-8125-cbd364cbedd9 │ ├── f09f3e26-9371-41f4-86a1-cd59b0fe624c │ └── stored.rules ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── architecture │ │ ├── ProductionCodeArchitectureTest.java │ │ └── TestCodeArchitectureTest.java │ └── resources │ └── archunit.properties ├── flink-connector-jdbc-backward-compatibility ├── pom.xml └── src │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── backward │ │ └── compatibility │ │ ├── DataStreamSinkTest.java │ │ ├── DataStreamSourceTest.java │ │ ├── DynamicTableSinkTest.java │ │ └── DynamicTableSourceTest.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-core ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ ├── JdbcConnectionOptions.java │ │ │ ├── JdbcExactlyOnceOptions.java │ │ │ ├── JdbcExecutionOptions.java │ │ │ ├── JdbcInputFormat.java │ │ │ ├── JdbcRowOutputFormat.java │ │ │ ├── JdbcStatementBuilder.java │ │ │ ├── core │ │ │ ├── database │ │ │ │ ├── JdbcFactory.java │ │ │ │ ├── JdbcFactoryLoader.java │ │ │ │ ├── catalog │ │ │ │ │ ├── AbstractJdbcCatalog.java │ │ │ │ │ ├── JdbcCatalog.java │ │ │ │ │ ├── JdbcCatalogTypeMapper.java │ │ │ │ │ └── factory │ │ │ │ │ │ ├── JdbcCatalogFactory.java │ │ │ │ │ │ └── JdbcCatalogFactoryOptions.java │ │ │ │ └── dialect │ │ │ │ │ ├── AbstractDialect.java │ │ │ │ │ ├── AbstractDialectConverter.java │ │ │ │ │ ├── JdbcDialect.java │ │ │ │ │ └── JdbcDialectConverter.java │ │ │ ├── datastream │ │ │ │ ├── Jdbc.java │ │ │ │ ├── sink │ │ │ │ │ ├── JdbcSink.java │ │ │ │ │ ├── JdbcSinkBuilder.java │ │ │ │ │ ├── committer │ │ │ │ │ │ ├── JdbcCommitable.java │ │ │ │ │ │ ├── JdbcCommitableSerializer.java │ │ │ │ │ │ └── JdbcCommitter.java │ │ │ │ │ └── writer │ │ │ │ │ │ ├── JdbcWriter.java │ │ │ │ │ │ ├── JdbcWriterState.java │ │ │ │ │ │ └── JdbcWriterStateSerializer.java │ │ │ │ └── source │ │ │ │ │ ├── JdbcSource.java │ │ │ │ │ ├── JdbcSourceBuilder.java │ │ │ │ │ ├── JdbcSourceOptions.java │ │ │ │ │ ├── config │ │ │ │ │ └── ContinuousUnBoundingSettings.java │ │ │ │ │ ├── enumerator │ │ │ │ │ ├── JdbcSourceEnumStateSerializer.java │ │ │ │ │ ├── JdbcSourceEnumerator.java │ │ │ │ │ ├── JdbcSourceEnumeratorState.java │ │ │ │ │ ├── JdbcSqlSplitEnumeratorBase.java │ │ │ │ │ └── SqlTemplateSplitEnumerator.java │ │ │ │ │ ├── reader │ │ │ │ │ ├── JdbcRecordEmitter.java │ │ │ │ │ ├── JdbcSourceReader.java │ │ │ │ │ ├── JdbcSourceSplitReader.java │ │ │ │ │ ├── RecordAndOffset.java │ │ │ │ │ └── extractor │ │ │ │ │ │ ├── ResultExtractor.java │ │ │ │ │ │ └── RowResultExtractor.java │ │ │ │ │ └── split │ │ │ │ │ ├── CheckpointedOffset.java │ │ │ │ │ ├── JdbcSourceSplit.java │ │ │ │ │ ├── JdbcSourceSplitSerializer.java │ │ │ │ │ └── JdbcSourceSplitState.java │ │ │ └── table │ │ │ │ ├── FilterHandlingPolicy.java │ │ │ │ ├── JdbcConnectorOptions.java │ │ │ │ ├── JdbcDynamicTableFactory.java │ │ │ │ ├── sink │ │ │ │ ├── JdbcDynamicTableSink.java │ │ │ │ └── JdbcOutputFormatBuilder.java │ │ │ │ └── source │ │ │ │ ├── JdbcDynamicTableSource.java │ │ │ │ ├── JdbcFilterPushdownPreparedStatementVisitor.java │ │ │ │ ├── JdbcRowDataInputFormat.java │ │ │ │ ├── JdbcRowDataLookupFunction.java │ │ │ │ └── ParameterizedPredicate.java │ │ │ ├── datasource │ │ │ ├── connections │ │ │ │ ├── JdbcConnectionProvider.java │ │ │ │ ├── SimpleJdbcConnectionProvider.java │ │ │ │ └── xa │ │ │ │ │ ├── PoolingXaConnectionProvider.java │ │ │ │ │ ├── SimpleXaConnectionProvider.java │ │ │ │ │ ├── XaCommand.java │ │ │ │ │ └── XaConnectionProvider.java │ │ │ ├── statements │ │ │ │ ├── JdbcQueryStatement.java │ │ │ │ └── SimpleJdbcQueryStatement.java │ │ │ └── transactions │ │ │ │ └── xa │ │ │ │ ├── XaTransaction.java │ │ │ │ ├── XaTransactionResult.java │ │ │ │ ├── domain │ │ │ │ └── TransactionId.java │ │ │ │ ├── exceptions │ │ │ │ ├── EmptyTransactionXaException.java │ │ │ │ ├── TransientXaException.java │ │ │ │ └── XaError.java │ │ │ │ └── xid │ │ │ │ ├── XidImpl.java │ │ │ │ └── XidSerializer.java │ │ │ ├── derby │ │ │ └── database │ │ │ │ ├── DerbyFactory.java │ │ │ │ └── dialect │ │ │ │ ├── DerbyDialect.java │ │ │ │ └── DerbyDialectConverter.java │ │ │ ├── internal │ │ │ ├── GenericJdbcSinkFunction.java │ │ │ ├── JdbcOutputFormat.java │ │ │ ├── JdbcOutputSerializer.java │ │ │ ├── RowJdbcOutputFormat.java │ │ │ ├── TableJdbcUpsertOutputFormat.java │ │ │ ├── executor │ │ │ │ ├── InsertOrUpdateJdbcExecutor.java │ │ │ │ ├── JdbcBatchStatementExecutor.java │ │ │ │ ├── KeyedBatchStatementExecutor.java │ │ │ │ ├── SimpleBatchStatementExecutor.java │ │ │ │ ├── TableBufferReducedStatementExecutor.java │ │ │ │ ├── TableBufferedStatementExecutor.java │ │ │ │ ├── TableInsertOrUpdateStatementExecutor.java │ │ │ │ └── TableSimpleStatementExecutor.java │ │ │ └── options │ │ │ │ ├── InternalJdbcConnectionOptions.java │ │ │ │ ├── JdbcDmlOptions.java │ │ │ │ ├── JdbcInsertOptions.java │ │ │ │ ├── JdbcReadOptions.java │ │ │ │ └── JdbcTypedQueryOptions.java │ │ │ ├── split │ │ │ ├── CompositeJdbcParameterValuesProvider.java │ │ │ ├── JdbcGenericParameterValuesProvider.java │ │ │ ├── JdbcNumericBetweenParametersProvider.java │ │ │ ├── JdbcParameterValuesProvider.java │ │ │ └── JdbcSlideTimingParameterProvider.java │ │ │ ├── statement │ │ │ ├── FieldNamedPreparedStatement.java │ │ │ ├── FieldNamedPreparedStatementImpl.java │ │ │ └── StatementFactory.java │ │ │ └── utils │ │ │ ├── JdbcTypeUtil.java │ │ │ └── JdbcUtils.java │ └── resources │ │ └── META-INF │ │ └── services │ │ ├── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ │ └── org.apache.flink.table.factories.Factory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ ├── base │ │ └── source │ │ │ └── reader │ │ │ └── splitreader │ │ │ └── TestingSplitsChange.java │ │ └── jdbc │ │ ├── JdbcConnectionOptionsTest.java │ │ ├── JdbcDataTestBase.java │ │ ├── JdbcITCase.java │ │ ├── JdbcInputFormatTest.java │ │ ├── JdbcRowOutputFormatTest.java │ │ ├── JdbcTestBase.java │ │ ├── JdbcTestCheckpoint.java │ │ ├── JdbcTestFixture.java │ │ ├── core │ │ ├── database │ │ │ ├── catalog │ │ │ │ └── AbstractJdbcCatalogTest.java │ │ │ └── dialect │ │ │ │ ├── JdbcDialectConverterTest.java │ │ │ │ └── JdbcDialectTest.java │ │ ├── datastream │ │ │ ├── sink │ │ │ │ ├── AtLeastOnceJdbcSinkTest.java │ │ │ │ ├── BaseJdbcSinkTest.java │ │ │ │ ├── ExactlyOnceJdbcSinkTest.java │ │ │ │ └── writer │ │ │ │ │ ├── AlLeastOnceJdbcWriterTest.java │ │ │ │ │ ├── BaseJdbcWriterTest.java │ │ │ │ │ ├── ExactlyOnceJdbcWriterTest.java │ │ │ │ │ ├── JdbcWriterStateSerializerTest.java │ │ │ │ │ └── JdbcWriterStateTest.java │ │ │ └── source │ │ │ │ ├── JdbcSourceBuilderTest.java │ │ │ │ ├── JdbcSourceITCase.java │ │ │ │ ├── JdbcSourceStreamRelatedITCase.java │ │ │ │ ├── enumerator │ │ │ │ ├── JdbcSourceEnumStateSerializerTest.java │ │ │ │ └── JdbcSourceEnumeratorTest.java │ │ │ │ ├── reader │ │ │ │ ├── JdbcSourceReaderTest.java │ │ │ │ └── JdbcSourceSplitReaderTest.java │ │ │ │ └── split │ │ │ │ └── JdbcSourceSplitSerializerTest.java │ │ └── table │ │ │ ├── JdbcDynamicTableFactoryTest.java │ │ │ ├── JdbcTablePlanTest.java │ │ │ ├── sink │ │ │ ├── JdbcAppendOnlyWriterTest.java │ │ │ ├── JdbcDynamicTableSinkITCase.java │ │ │ └── JdbcOutputFormatTest.java │ │ │ └── source │ │ │ ├── JdbcDynamicTableSourceITCase.java │ │ │ ├── JdbcFilterPushdownPreparedStatementVisitorTest.java │ │ │ ├── JdbcLookupTestBase.java │ │ │ ├── JdbcRowDataInputFormatTest.java │ │ │ └── JdbcRowDataLookupFunctionTest.java │ │ ├── datasource │ │ ├── connections │ │ │ └── xa │ │ │ │ ├── SimpleXaConnectionProviderTest.java │ │ │ │ └── TransactionIdConnectionTest.java │ │ └── transactions │ │ │ └── xa │ │ │ ├── domain │ │ │ └── TransactionIdTest.java │ │ │ └── xid │ │ │ ├── XaXidTest.java │ │ │ └── XidSerializersTest.java │ │ ├── derby │ │ ├── DerbyTestBase.java │ │ ├── database │ │ │ └── DerbyDialectTest.java │ │ ├── table │ │ │ ├── DerbyDynamicTableSinkITCase.java │ │ │ └── DerbyDynamicTableSourceITCase.java │ │ └── testutils │ │ │ ├── DerbyDatabase.java │ │ │ └── DerbyMetadata.java │ │ ├── fakedb │ │ ├── FakeDBUtils.java │ │ └── driver │ │ │ ├── FakeConnection.java │ │ │ ├── FakeConnection1.java │ │ │ ├── FakeConnection2.java │ │ │ ├── FakeConnection3.java │ │ │ ├── FakeDriver1.java │ │ │ ├── FakeDriver2.java │ │ │ └── FakeDriver3.java │ │ ├── h2 │ │ ├── H2XaTestBase.java │ │ └── testutils │ │ │ ├── H2Metadata.java │ │ │ ├── H2XaDatabase.java │ │ │ └── xa │ │ │ ├── H2XaConnectionWrapper.java │ │ │ ├── H2XaDsWrapper.java │ │ │ ├── H2XaResourceWrapper.java │ │ │ └── package-info.java │ │ ├── internal │ │ ├── JdbcFullTest.java │ │ ├── JdbcOutputSerializerTest.java │ │ ├── JdbcTableOutputFormatTest.java │ │ └── connection │ │ │ ├── SimpleJdbcConnectionProviderDriverClassConcurrentLoadingITCase.java │ │ │ └── SimpleJdbcConnectionProviderTest.java │ │ ├── split │ │ └── NumericBetweenParametersProviderTest.java │ │ ├── testutils │ │ ├── DatabaseExtension.java │ │ ├── DatabaseMetadata.java │ │ ├── DatabaseResource.java │ │ ├── DatabaseTest.java │ │ ├── JdbcITCaseBase.java │ │ ├── TableManaged.java │ │ ├── functions │ │ │ └── JdbcResultSetBuilder.java │ │ ├── resources │ │ │ ├── DockerResource.java │ │ │ └── MemoryResource.java │ │ └── tables │ │ │ ├── TableBase.java │ │ │ ├── TableBuilder.java │ │ │ ├── TableBuilderTest.java │ │ │ ├── TableField.java │ │ │ ├── TableRow.java │ │ │ └── templates │ │ │ └── BooksTable.java │ │ └── utils │ │ └── JdbcTypeUtilTest.java │ └── resources │ ├── META-INF │ └── services │ │ └── java.sql.Driver │ ├── log4j2-test.properties │ └── org │ └── apache │ └── flink │ └── connector │ └── jdbc │ └── core │ └── table │ └── JdbcTablePlanTest.xml ├── flink-connector-jdbc-cratedb ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── cratedb │ │ │ └── database │ │ │ ├── CrateDBFactory.java │ │ │ ├── catalog │ │ │ ├── CrateDBCatalog.java │ │ │ ├── CrateDBTablePath.java │ │ │ └── CrateDBTypeMapper.java │ │ │ └── dialect │ │ │ ├── CrateDBDialect.java │ │ │ └── CrateDBDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── cratedb │ │ ├── CrateDBTestBase.java │ │ ├── database │ │ ├── catalog │ │ │ ├── CrateDBCatalogITCase.java │ │ │ ├── CrateDBCatalogTest.java │ │ │ ├── CrateDBCatalogTestBase.java │ │ │ └── CrateDBTablePathTest.java │ │ └── dialect │ │ │ └── CrateDBDialectTest.java │ │ └── testutils │ │ ├── CrateDBDatabase.java │ │ ├── CrateDBImages.java │ │ └── CrateDBMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-db2 ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── db2 │ │ │ └── database │ │ │ ├── Db2Factory.java │ │ │ └── dialect │ │ │ ├── Db2Dialect.java │ │ │ └── Db2DialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── db2 │ │ ├── Db2TestBase.java │ │ ├── database │ │ ├── dialect │ │ │ ├── Db2DialectTest.java │ │ │ └── Db2PreparedStatementTest.java │ │ └── table │ │ │ ├── Db2DynamicTableSinkITCase.java │ │ │ └── Db2DynamicTableSourceITCase.java │ │ └── testutils │ │ ├── Db2Database.java │ │ ├── Db2Images.java │ │ └── Db2Metadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-mysql ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── mysql │ │ │ └── database │ │ │ ├── MySqlFactory.java │ │ │ ├── catalog │ │ │ ├── MySqlCatalog.java │ │ │ └── MySqlTypeMapper.java │ │ │ └── dialect │ │ │ ├── MySQLDialectConverter.java │ │ │ └── MySqlDialect.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── mysql │ │ ├── MySqlTestBase.java │ │ ├── database │ │ ├── catalog │ │ │ ├── MySqlCatalogITCase.java │ │ │ └── MySqlCatalogTestBase.java │ │ └── dialect │ │ │ ├── MySqlDialectTest.java │ │ │ └── MysqlPreparedStatementTest.java │ │ ├── table │ │ ├── MySqlDynamicTableSinkITCase.java │ │ ├── MySqlDynamicTableSourceITCase.java │ │ └── UnsignedTypeConversionITCase.java │ │ └── testutils │ │ ├── MySqlContainer.java │ │ ├── MySqlDatabase.java │ │ ├── MySqlImages.java │ │ └── MySqlMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-oceanbase ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── oceanbase │ │ │ └── database │ │ │ ├── OceanBaseFactory.java │ │ │ ├── catalog │ │ │ ├── OceanBaseCatalog.java │ │ │ └── OceanBaseTypeMapper.java │ │ │ └── dialect │ │ │ ├── OceanBaseCompatibleMode.java │ │ │ ├── OceanBaseDialect.java │ │ │ └── OceanBaseDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── oceanbase │ │ ├── OceanBaseMysqlTestBase.java │ │ ├── OceanBaseOracleTestBase.java │ │ ├── database │ │ ├── catalog │ │ │ ├── OceanBaseCatalogITCaseBase.java │ │ │ ├── OceanBaseMysqlCatalogITCase.java │ │ │ └── OceanBaseOracleCatalogITCase.java │ │ └── dialect │ │ │ ├── OceanBaseDialectTest.java │ │ │ ├── OceanBaseMysqlDialectTest.java │ │ │ └── OceanBaseOracleDialectTest.java │ │ ├── table │ │ ├── OceanBaseMySqlDynamicTableSinkITCase.java │ │ ├── OceanBaseMySqlDynamicTableSourceITCase.java │ │ ├── OceanBaseOracleDynamicTableSinkITCase.java │ │ ├── OceanBaseOracleDynamicTableSourceITCase.java │ │ └── OceanBaseTableRow.java │ │ └── testutils │ │ ├── OceanBaseDatabase.java │ │ ├── OceanBaseImages.java │ │ └── OceanBaseMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-oracle ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── oracle │ │ │ └── database │ │ │ ├── OracleFactory.java │ │ │ └── dialect │ │ │ ├── OracleDialect.java │ │ │ └── OracleDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── oracle │ │ ├── OracleTestBase.java │ │ ├── database │ │ └── dialect │ │ │ ├── OracleDialectTest.java │ │ │ └── OraclePreparedStatementTest.java │ │ ├── table │ │ ├── OracleDynamicTableSinkITCase.java │ │ ├── OracleDynamicTableSourceITCase.java │ │ └── OracleTableSourceITCase.java │ │ └── testutils │ │ ├── OracleDatabase.java │ │ ├── OracleImages.java │ │ └── OracleMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-postgres ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── postgres │ │ │ └── database │ │ │ ├── PostgresFactory.java │ │ │ ├── catalog │ │ │ ├── PostgresCatalog.java │ │ │ ├── PostgresTablePath.java │ │ │ └── PostgresTypeMapper.java │ │ │ └── dialect │ │ │ ├── CompatiblePostgresDialect.java │ │ │ ├── CompatiblePostgresDialectConverter.java │ │ │ ├── PostgresDialect.java │ │ │ └── PostgresDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── postgres │ │ ├── PostgresTestBase.java │ │ ├── database │ │ ├── PostgresFactoryTest.java │ │ ├── catalog │ │ │ ├── PostgresCatalogITCase.java │ │ │ ├── PostgresCatalogTest.java │ │ │ ├── PostgresCatalogTestBase.java │ │ │ └── PostgresTablePathTest.java │ │ └── dialect │ │ │ └── PostgresDialectTest.java │ │ ├── table │ │ ├── PostgresDynamicTableSinkITCase.java │ │ └── PostgresDynamicTableSourceITCase.java │ │ └── testutils │ │ ├── PostgresDatabase.java │ │ ├── PostgresImages.java │ │ └── PostgresMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-sqlserver ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── sqlserver │ │ │ └── database │ │ │ ├── SqlServerFactory.java │ │ │ └── dialect │ │ │ ├── SqlServerDialect.java │ │ │ └── SqlServerDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── sqlserver │ │ ├── SqlServerTestBase.java │ │ ├── database │ │ └── dialect │ │ │ └── SqlServerPreparedStatementTest.java │ │ ├── table │ │ ├── SqlServerDynamicTableSinkITCase.java │ │ ├── SqlServerDynamicTableSourceITCase.java │ │ └── SqlServerTableSourceITCase.java │ │ └── testutils │ │ ├── SqlServerDatabase.java │ │ ├── SqlServerImages.java │ │ └── SqlServerMetadata.java │ └── resources │ └── log4j2-test.properties ├── flink-connector-jdbc-trino ├── pom.xml └── src │ ├── main │ ├── java │ │ └── org │ │ │ └── apache │ │ │ └── flink │ │ │ └── connector │ │ │ └── jdbc │ │ │ └── trino │ │ │ └── database │ │ │ ├── TrinoFactory.java │ │ │ └── dialect │ │ │ ├── TrinoDialect.java │ │ │ └── TrinoDialectConverter.java │ └── resources │ │ └── META-INF │ │ └── services │ │ └── org.apache.flink.connector.jdbc.core.database.JdbcFactory │ └── test │ ├── java │ └── org │ │ └── apache │ │ └── flink │ │ └── connector │ │ └── jdbc │ │ └── trino │ │ ├── TrinoTestBase.java │ │ ├── database │ │ └── dialect │ │ │ ├── TrinoDialectTest.java │ │ │ └── TrinoPreparedStatementTest.java │ │ ├── table │ │ ├── TrinoDynamicTableSinkITCase.java │ │ └── TrinoDynamicTableSourceITCase.java │ │ └── testutils │ │ ├── TrinoDatabase.java │ │ ├── TrinoImages.java │ │ └── TrinoMetadata.java │ └── resources │ └── log4j2-test.properties ├── pom.xml └── tools ├── ci └── log4j.properties └── maven ├── checkstyle.xml └── suppressions.xml /.asf.yaml: -------------------------------------------------------------------------------- 1 | github: 2 | enabled_merge_buttons: 3 | squash: true 4 | merge: false 5 | rebase: true 6 | labels: 7 | - flink 8 | - jdbc 9 | - connector 10 | - datastream 11 | - table 12 | - sql 13 | autolink_jira: FLINK 14 | collaborators: 15 | - flinkbot 16 | notifications: 17 | commits: commits@flink.apache.org 18 | issues: issues@flink.apache.org 19 | pullrequests: issues@flink.apache.org 20 | jobs: builds@flink.apache.org 21 | jira_options: link label 22 | -------------------------------------------------------------------------------- /.github/workflows/push_pr.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # We need to specify repo related information here since Apache INFRA doesn't differentiate 20 | # between several workflows with the same names while preparing a report for GHA usage 21 | # https://infra-reports.apache.org/#ghactions 22 | name: Flink Connector JDBC CI 23 | on: [push, pull_request] 24 | concurrency: 25 | group: ${{ github.workflow }}-${{ github.ref }} 26 | cancel-in-progress: true 27 | jobs: 28 | compile_and_test: 29 | strategy: 30 | matrix: 31 | flink: [2.0.0] 32 | jdk: [ '17' ] 33 | uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils 34 | with: 35 | flink_version: ${{ matrix.flink }} 36 | jdk_version: ${{ matrix.jdk }} 37 | -------------------------------------------------------------------------------- /.github/workflows/weekly.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # We need to specify repo related information here since Apache INFRA doesn't differentiate 20 | # between several workflows with the same names while preparing a report for GHA usage 21 | # https://infra-reports.apache.org/#ghactions 22 | name: Weekly Flink Connector JDBC 23 | on: 24 | schedule: 25 | - cron: "0 0 * * 0" 26 | workflow_dispatch: 27 | jobs: 28 | compile_and_test: 29 | if: github.repository_owner == 'apache' 30 | strategy: 31 | matrix: 32 | flink_branches: [{ 33 | flink: 2.1-SNAPSHOT, 34 | jdk: '17', 35 | branch: main 36 | }, { 37 | flink: 2.0-SNAPSHOT, 38 | jdk: '17', 39 | branch: main 40 | }, { 41 | flink: 2.0.0, 42 | jdk: '17', 43 | branch: main 44 | }, { 45 | flink: 1.19.1, 46 | jdk: '8, 11, 17, 21', 47 | branch: v3.2 48 | }, { 49 | flink: 1.18.0, 50 | jdk: '8, 11, 17', 51 | branch: v3.1 52 | }] 53 | uses: apache/flink-connector-shared-utils/.github/workflows/ci.yml@ci_utils 54 | with: 55 | flink_version: ${{ matrix.flink_branches.flink }} 56 | connector_branch: ${{ matrix.flink_branches.branch }} 57 | jdk_version: ${{ matrix.flink_branches.jdk || '8, 11' }} 58 | run_dependency_convergence: false 59 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | .eslintcache 2 | .cache 3 | scalastyle-output.xml 4 | .classpath 5 | .idea/* 6 | !.idea/vcs.xml 7 | .metadata 8 | .settings 9 | .project 10 | .version.properties 11 | filter.properties 12 | logs.zip 13 | .mvn/wrapper/*.jar 14 | target 15 | tmp 16 | *.class 17 | *.iml 18 | *.swp 19 | *.jar 20 | *.zip 21 | *.log 22 | *.pyc 23 | .DS_Store 24 | build-target 25 | atlassian-ide-plugin.xml 26 | out/ 27 | /docs/api 28 | /docs/.bundle 29 | /docs/.rubydeps 30 | /docs/ruby2/.bundle 31 | /docs/ruby2/.rubydeps 32 | /docs/.jekyll-metadata 33 | *.ipr 34 | *.iws 35 | tools/flink 36 | tools/flink-* 37 | tools/releasing/release 38 | tools/japicmp-output -------------------------------------------------------------------------------- /.gitmodules: -------------------------------------------------------------------------------- 1 | [submodule "tools/releasing/shared"] 2 | path = tools/releasing/shared 3 | url = git@github.com:apache/flink-connector-shared-utils.git 4 | branch = release_utils 5 | -------------------------------------------------------------------------------- /.idea/vcs.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 4 | 20 | 21 | 22 | 23 | 24 | -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Flink JDBC Connector 2 | Copyright 2014-2024 The Apache Software Foundation 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | 7 | Permission to use, copy, modify, and/or distribute this software for any purpose with or without fee is hereby 8 | granted, provided that this permission notice appear in all copies. 9 | 10 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES WITH REGARD TO THIS SOFTWARE INCLUDING 11 | ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR ANY SPECIAL, 12 | DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, 13 | WHETHER IN AN ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF OR IN CONNECTION WITH THE 14 | USE OR PERFORMANCE OF THIS SOFTWARE. 15 | 16 | 17 | 18 | -------------------------------------------------------------------------------- /docs/data/jdbc.yml: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | version: 3.3.0-SNAPSHOT 20 | variants: 21 | - maven: flink-connector-jdbc 22 | sql_url: https://repo.maven.apache.org/maven2/org/apache/flink/flink-connector-jdbc/$full_version/flink-connector-jdbc-$full_version.jar 23 | -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/10da9e19-442f-49fb-b340-a2e401062d33: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/10da9e19-442f-49fb-b340-a2e401062d33 -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/3547d327-0923-4fa7-8319-b18102cc22d0: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/3547d327-0923-4fa7-8319-b18102cc22d0 -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/47967139-2deb-41b4-91c3-b19e56e5fc76: -------------------------------------------------------------------------------- 1 | org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialect.getRowConverter(org.apache.flink.table.types.logical.RowType): Returned leaf type org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialectConverter does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated 2 | org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialect.getRowConverter(org.apache.flink.table.types.logical.RowType): Returned leaf type org.apache.flink.connector.jdbc.postgres.database.dialect.PostgresDialectConverter does not satisfy: reside outside of package 'org.apache.flink..' or reside in any package ['..shaded..'] or annotated with @Public or annotated with @PublicEvolving or annotated with @Deprecated 3 | -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/7d5a32d5-922b-4760-825b-c30427dc987e: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/7d5a32d5-922b-4760-825b-c30427dc987e -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/b97067ac-f37f-4793-8578-bdbe6d2abf5b: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/b97067ac-f37f-4793-8578-bdbe6d2abf5b -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/e391ec0f-7ba6-4b7e-8125-cbd364cbedd9: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/e391ec0f-7ba6-4b7e-8125-cbd364cbedd9 -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/f09f3e26-9371-41f4-86a1-cd59b0fe624c: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/apache/flink-connector-jdbc/dd89235981af1b64755225c1d49e9f6dc1cf637b/flink-connector-jdbc-architecture/archunit-violations/f09f3e26-9371-41f4-86a1-cd59b0fe624c -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/archunit-violations/stored.rules: -------------------------------------------------------------------------------- 1 | # 2 | #Wed Oct 23 14:35:32 CEST 2024 3 | Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @Public\ must\ be\ annotated\ with\ @Public.=e391ec0f-7ba6-4b7e-8125-cbd364cbedd9 4 | ITCASE\ tests\ should\ use\ a\ MiniCluster\ resource\ or\ extension=f09f3e26-9371-41f4-86a1-cd59b0fe624c 5 | Production\ code\ must\ not\ call\ methods\ annotated\ with\ @VisibleForTesting=3547d327-0923-4fa7-8319-b18102cc22d0 6 | Options\ for\ connectors\ and\ formats\ should\ reside\ in\ a\ consistent\ package\ and\ be\ public\ API.=10da9e19-442f-49fb-b340-a2e401062d33 7 | Tests\ inheriting\ from\ AbstractTestBase\ should\ have\ name\ ending\ with\ ITCase=7d5a32d5-922b-4760-825b-c30427dc987e 8 | Return\ and\ argument\ types\ of\ methods\ annotated\ with\ @PublicEvolving\ must\ be\ annotated\ with\ @Public(Evolving).=47967139-2deb-41b4-91c3-b19e56e5fc76 9 | Classes\ in\ API\ packages\ should\ have\ at\ least\ one\ API\ visibility\ annotation.=b97067ac-f37f-4793-8578-bdbe6d2abf5b 10 | Connector\ production\ code\ must\ depend\ only\ on\ public\ API\ when\ outside\ of\ connector\ packages=afc13e03-aa50-408b-b87e-aa491cf0b8cb 11 | -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/src/test/java/org/apache/flink/connector/jdbc/architecture/ProductionCodeArchitectureTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.architecture; 20 | 21 | import org.apache.flink.architecture.ProductionCodeArchitectureBase; 22 | import org.apache.flink.architecture.common.ImportOptions; 23 | 24 | import com.tngtech.archunit.core.importer.ImportOption; 25 | import com.tngtech.archunit.junit.AnalyzeClasses; 26 | import com.tngtech.archunit.junit.ArchTest; 27 | import com.tngtech.archunit.junit.ArchTests; 28 | 29 | /** production code for Architecture tests. */ 30 | @AnalyzeClasses( 31 | packages = "org.apache.flink.connector.jdbc", 32 | importOptions = { 33 | ImportOption.DoNotIncludeTests.class, 34 | ImportOption.DoNotIncludeArchives.class, 35 | ImportOptions.ExcludeScalaImportOption.class, 36 | ImportOptions.ExcludeShadedImportOption.class 37 | }) 38 | public class ProductionCodeArchitectureTest { 39 | 40 | @ArchTest 41 | public static final ArchTests COMMON_TESTS = ArchTests.in(ProductionCodeArchitectureBase.class); 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/src/test/java/org/apache/flink/connector/jdbc/architecture/TestCodeArchitectureTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.architecture; 20 | 21 | import org.apache.flink.architecture.TestCodeArchitectureTestBase; 22 | import org.apache.flink.architecture.common.ImportOptions; 23 | 24 | import com.tngtech.archunit.core.importer.ImportOption; 25 | import com.tngtech.archunit.junit.AnalyzeClasses; 26 | import com.tngtech.archunit.junit.ArchTest; 27 | import com.tngtech.archunit.junit.ArchTests; 28 | 29 | /** Architecture tests for test code. */ 30 | @AnalyzeClasses( 31 | packages = "org.apache.flink.connector.jdbc", 32 | importOptions = { 33 | ImportOption.OnlyIncludeTests.class, 34 | ImportOptions.ExcludeScalaImportOption.class, 35 | ImportOptions.ExcludeShadedImportOption.class 36 | }) 37 | public class TestCodeArchitectureTest { 38 | 39 | @ArchTest 40 | public static final ArchTests COMMON_TESTS = ArchTests.in(TestCodeArchitectureTestBase.class); 41 | } 42 | -------------------------------------------------------------------------------- /flink-connector-jdbc-architecture/src/test/resources/archunit.properties: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | # 18 | 19 | # By default we allow removing existing violations, but fail when new violations are added. 20 | freeze.store.default.allowStoreUpdate=true 21 | 22 | # Enable this if a new (frozen) rule has been added in order to create the initial store and record the existing violations. 23 | #freeze.store.default.allowStoreCreation=true 24 | 25 | # Enable this to add allow new violations to be recorded. 26 | # NOTE: Adding new violations should be avoided when possible. If the rule was correct to flag a new 27 | # violation, please try to avoid creating the violation. If the violation was created due to a 28 | # shortcoming of the rule, file a JIRA issue so the rule can be improved. 29 | #freeze.refreeze=true 30 | 31 | freeze.store.default.path=archunit-violations 32 | 33 | archRule.failOnEmptyShould = false 34 | -------------------------------------------------------------------------------- /flink-connector-jdbc-backward-compatibility/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/JdbcStatementBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.connector.jdbc.internal.executor.JdbcBatchStatementExecutor; 22 | import org.apache.flink.util.function.BiConsumerWithException; 23 | 24 | import java.io.Serializable; 25 | import java.sql.PreparedStatement; 26 | import java.sql.SQLException; 27 | 28 | /** 29 | * Sets {@link PreparedStatement} parameters to use in JDBC Sink based on a specific type of 30 | * StreamRecord. 31 | * 32 | * @param type of payload in {@link org.apache.flink.streaming.runtime.streamrecord.StreamRecord 33 | * StreamRecord} 34 | * @see JdbcBatchStatementExecutor 35 | */ 36 | @PublicEvolving 37 | public interface JdbcStatementBuilder 38 | extends BiConsumerWithException, Serializable {} 39 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/database/catalog/JdbcCatalog.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.database.catalog; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.table.catalog.Catalog; 5 | 6 | import java.io.Serializable; 7 | 8 | /** Catalogs for relational databases via JDBC. */ 9 | @PublicEvolving 10 | public interface JdbcCatalog extends Catalog, Serializable {} 11 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/database/catalog/JdbcCatalogTypeMapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.database.catalog; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.table.catalog.ObjectPath; 23 | import org.apache.flink.table.types.DataType; 24 | 25 | import java.sql.ResultSetMetaData; 26 | import java.sql.SQLException; 27 | 28 | /** Separate the jdbc meta-information type to flink table type into the interface. */ 29 | @PublicEvolving 30 | public interface JdbcCatalogTypeMapper { 31 | 32 | DataType mapping(ObjectPath tablePath, ResultSetMetaData metadata, int colIndex) 33 | throws SQLException; 34 | } 35 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/database/catalog/factory/JdbcCatalogFactoryOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.database.catalog.factory; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.configuration.ConfigOption; 23 | import org.apache.flink.configuration.ConfigOptions; 24 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 25 | import org.apache.flink.connector.jdbc.core.table.JdbcConnectorOptions; 26 | import org.apache.flink.table.catalog.CommonCatalogOptions; 27 | 28 | /** {@link ConfigOption}s for {@link JdbcCatalog}. */ 29 | @Internal 30 | public class JdbcCatalogFactoryOptions { 31 | 32 | public static final String IDENTIFIER = "jdbc"; 33 | 34 | public static final ConfigOption DEFAULT_DATABASE = 35 | ConfigOptions.key(CommonCatalogOptions.DEFAULT_DATABASE_KEY) 36 | .stringType() 37 | .noDefaultValue(); 38 | 39 | public static final ConfigOption USERNAME = JdbcConnectorOptions.USERNAME; 40 | 41 | public static final ConfigOption PASSWORD = JdbcConnectorOptions.PASSWORD; 42 | 43 | public static final ConfigOption BASE_URL = 44 | ConfigOptions.key("base-url").stringType().noDefaultValue(); 45 | 46 | public static final ConfigOption COMPATIBLE_MODE = 47 | ConfigOptions.key("compatible-mode").stringType().noDefaultValue(); 48 | 49 | private JdbcCatalogFactoryOptions() {} 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/database/dialect/JdbcDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.database.dialect; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.jdbc.statement.FieldNamedPreparedStatement; 23 | import org.apache.flink.table.data.RowData; 24 | 25 | import java.io.Serializable; 26 | import java.sql.ResultSet; 27 | import java.sql.SQLException; 28 | 29 | /** 30 | * Converter that is responsible to convert between JDBC object and Flink SQL internal data 31 | * structure {@link RowData}. 32 | */ 33 | @PublicEvolving 34 | public interface JdbcDialectConverter extends Serializable { 35 | 36 | /** 37 | * Convert data retrieved from {@link ResultSet} to internal {@link RowData}. 38 | * 39 | * @param resultSet ResultSet from JDBC 40 | */ 41 | RowData toInternal(ResultSet resultSet) throws SQLException; 42 | 43 | /** 44 | * Convert data retrieved from Flink internal RowData to JDBC Object. 45 | * 46 | * @param rowData The given internal {@link RowData}. 47 | * @param statement The statement to be filled. 48 | * @return The filled statement. 49 | */ 50 | FieldNamedPreparedStatement toExternal(RowData rowData, FieldNamedPreparedStatement statement) 51 | throws SQLException; 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/Jdbc.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.core.datastream; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.connector.jdbc.core.datastream.sink.JdbcSink; 22 | import org.apache.flink.connector.jdbc.core.datastream.sink.JdbcSinkBuilder; 23 | import org.apache.flink.connector.jdbc.core.datastream.source.JdbcSource; 24 | import org.apache.flink.connector.jdbc.core.datastream.source.JdbcSourceBuilder; 25 | 26 | /** Facade to create JDBC stream sources and sinks. */ 27 | @PublicEvolving 28 | public class Jdbc { 29 | 30 | /** Create a JDBC source builder. */ 31 | public static JdbcSourceBuilder sourceBuilder() { 32 | return JdbcSource.builder(); 33 | } 34 | 35 | /** Create a JDBC sink builder. */ 36 | public static JdbcSinkBuilder sinkBuilder() { 37 | return JdbcSink.builder(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/sink/committer/JdbcCommitable.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.datastream.sink.committer; 2 | 3 | import org.apache.flink.annotation.Internal; 4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.XaTransaction; 5 | 6 | import javax.annotation.Nullable; 7 | import javax.transaction.xa.Xid; 8 | 9 | import java.io.Serializable; 10 | import java.util.Optional; 11 | 12 | /** A pair of Xid and transaction that can be committed. */ 13 | @Internal 14 | public class JdbcCommitable implements Serializable { 15 | 16 | private final Xid xid; 17 | private final XaTransaction transaction; 18 | 19 | protected JdbcCommitable(Xid xid, @Nullable XaTransaction transaction) { 20 | this.xid = xid; 21 | this.transaction = transaction; 22 | } 23 | 24 | public static JdbcCommitable of(Xid xid) { 25 | return of(xid, null); 26 | } 27 | 28 | public static JdbcCommitable of(Xid xid, XaTransaction transaction) { 29 | return new JdbcCommitable(xid, transaction); 30 | } 31 | 32 | public Xid getXid() { 33 | return xid; 34 | } 35 | 36 | public Optional getTransaction() { 37 | return Optional.ofNullable(transaction); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/sink/committer/JdbcCommitableSerializer.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.datastream.sink.committer; 2 | 3 | import org.apache.flink.annotation.Internal; 4 | import org.apache.flink.api.common.typeutils.TypeSerializer; 5 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.xid.XidSerializer; 6 | import org.apache.flink.core.io.SimpleVersionedSerializer; 7 | import org.apache.flink.core.memory.DataInputDeserializer; 8 | import org.apache.flink.core.memory.DataOutputSerializer; 9 | 10 | import javax.transaction.xa.Xid; 11 | 12 | import java.io.IOException; 13 | 14 | /** {@link JdbcCommitable} serializer. */ 15 | @Internal 16 | public class JdbcCommitableSerializer implements SimpleVersionedSerializer { 17 | 18 | private final TypeSerializer xidSerializer = new XidSerializer(); 19 | 20 | @Override 21 | public int getVersion() { 22 | return 1; 23 | } 24 | 25 | @Override 26 | public byte[] serialize(JdbcCommitable commitable) throws IOException { 27 | final DataOutputSerializer out = new DataOutputSerializer(1); 28 | xidSerializer.serialize(commitable.getXid(), out); 29 | return out.getSharedBuffer(); 30 | } 31 | 32 | @Override 33 | public JdbcCommitable deserialize(int version, byte[] serialized) throws IOException { 34 | final DataInputDeserializer in = new DataInputDeserializer(serialized); 35 | return JdbcCommitable.of(xidSerializer.deserialize(in)); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/reader/JdbcRecordEmitter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.datastream.source.reader; 20 | 21 | import org.apache.flink.api.connector.source.SourceOutput; 22 | import org.apache.flink.connector.base.source.reader.RecordEmitter; 23 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit; 24 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplitState; 25 | 26 | /** 27 | * The JDBC resorce emitter. 28 | * 29 | * @param The type of the record. 30 | * @param The type of JDBC split. 31 | */ 32 | public class JdbcRecordEmitter 33 | implements RecordEmitter, T, JdbcSourceSplitState> { 34 | 35 | @Override 36 | public void emitRecord( 37 | RecordAndOffset element, 38 | SourceOutput output, 39 | JdbcSourceSplitState splitState) 40 | throws Exception { 41 | 42 | output.collect(element.getRecord()); 43 | splitState.setPosition(element.getOffset(), element.getRecordSkipCount()); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/reader/RecordAndOffset.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.datastream.source.reader; 20 | 21 | import org.apache.flink.connector.jdbc.core.datastream.source.split.CheckpointedOffset; 22 | 23 | /** 24 | * Util class to represent the record with the corresponding information. 25 | * 26 | * @param The type of the record. 27 | */ 28 | public class RecordAndOffset { 29 | 30 | public static final long NO_OFFSET = CheckpointedOffset.NO_OFFSET; 31 | 32 | final E record; 33 | // The two fields are not enabled for using now. 34 | final long offset; 35 | final long recordSkipCount; 36 | 37 | public RecordAndOffset(E record, long offset, long recordSkipCount) { 38 | this.record = record; 39 | this.offset = offset; 40 | this.recordSkipCount = recordSkipCount; 41 | } 42 | 43 | // ------------------------------------------------------------------------ 44 | 45 | public E getRecord() { 46 | return record; 47 | } 48 | 49 | public long getOffset() { 50 | return offset; 51 | } 52 | 53 | public long getRecordSkipCount() { 54 | return recordSkipCount; 55 | } 56 | 57 | // ------------------------------------------------------------------------ 58 | 59 | @Override 60 | public String toString() { 61 | return String.format("%s @ %d", record, offset); 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/reader/extractor/ResultExtractor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.datastream.source.reader.extractor; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.types.Row; 23 | 24 | import java.io.Serializable; 25 | import java.sql.ResultSet; 26 | import java.sql.SQLException; 27 | 28 | /** 29 | * The Extractor to extract the data from {@link ResultSet}. 30 | * 31 | * @param The target data type. 32 | */ 33 | @PublicEvolving 34 | public interface ResultExtractor extends Serializable { 35 | 36 | /** 37 | * Extract the data from the current point line of the result. 38 | * 39 | * @param resultSet Result set queried from a sql. 40 | * @return The data object filled by the current line of the resultSet. 41 | * @throws SQLException SQL exception. 42 | */ 43 | T extract(ResultSet resultSet) throws SQLException; 44 | 45 | /** 46 | * The identifier of the extractor. 47 | * 48 | * @return identifier in {@link String} type. 49 | */ 50 | default String identifier() { 51 | return this.getClass().getSimpleName(); 52 | } 53 | 54 | static ResultExtractor ofRowResultExtractor() { 55 | return new RowResultExtractor(); 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/datastream/source/reader/extractor/RowResultExtractor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.datastream.source.reader.extractor; 20 | 21 | import org.apache.flink.types.Row; 22 | import org.apache.flink.util.Preconditions; 23 | 24 | import java.sql.ResultSet; 25 | import java.sql.SQLException; 26 | 27 | /** The extractor is used to extract the data from {@link ResultSet} into a {@link Row} object. */ 28 | public class RowResultExtractor implements ResultExtractor { 29 | @Override 30 | public Row extract(ResultSet resultSet) throws SQLException { 31 | int arity = resultSet.getMetaData().getColumnCount(); 32 | Row row = new Row(arity); 33 | Preconditions.checkArgument(!resultSet.isClosed()); 34 | for (int index = 0; index < row.getArity(); index++) { 35 | row.setField(index, resultSet.getObject(index + 1)); 36 | } 37 | return row; 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/table/FilterHandlingPolicy.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.table; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.configuration.DescribedEnum; 5 | import org.apache.flink.configuration.description.InlineElement; 6 | 7 | import static org.apache.flink.configuration.description.TextElement.text; 8 | 9 | /** Fine-grained configuration to control filter push down for jdbc Table/SQL source. */ 10 | @PublicEvolving 11 | public enum FilterHandlingPolicy implements DescribedEnum { 12 | ALWAYS("always", text("Always push the supported filters to database.")), 13 | 14 | NEVER("never", text("Never push any filters to database.")); 15 | 16 | private final String name; 17 | private final InlineElement description; 18 | 19 | FilterHandlingPolicy(String name, InlineElement description) { 20 | this.name = name; 21 | this.description = description; 22 | } 23 | 24 | @Override 25 | public InlineElement getDescription() { 26 | return description; 27 | } 28 | 29 | @Override 30 | public String toString() { 31 | return name; 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/core/table/source/ParameterizedPredicate.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.table.source; 20 | 21 | import org.apache.flink.annotation.Experimental; 22 | 23 | import org.apache.commons.lang3.ArrayUtils; 24 | 25 | import java.io.Serializable; 26 | 27 | /** A data class that model parameterized sql predicate. */ 28 | @Experimental 29 | public class ParameterizedPredicate { 30 | private String predicate; 31 | private Serializable[] parameters; 32 | 33 | public ParameterizedPredicate(String predicate) { 34 | this.predicate = predicate; 35 | this.parameters = new Serializable[0]; 36 | } 37 | 38 | public Serializable[] getParameters() { 39 | return parameters; 40 | } 41 | 42 | public void setParameters(Serializable[] parameters) { 43 | this.parameters = parameters; 44 | } 45 | 46 | public String getPredicate() { 47 | return predicate; 48 | } 49 | 50 | public void setPredicate(String predicate) { 51 | this.predicate = predicate; 52 | } 53 | 54 | public ParameterizedPredicate combine(String operator, ParameterizedPredicate that) { 55 | this.predicate = String.format("(%s %s %s)", this.predicate, operator, that.predicate); 56 | this.parameters = ArrayUtils.addAll(this.parameters, that.parameters); 57 | return this; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/connections/xa/XaConnectionProvider.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.datasource.connections.xa; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.connector.jdbc.datasource.connections.JdbcConnectionProvider; 5 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions.TransientXaException; 6 | 7 | import javax.transaction.xa.XAException; 8 | import javax.transaction.xa.Xid; 9 | 10 | import java.util.Collection; 11 | 12 | /** 13 | * Provider to the XA operations. 14 | * 15 | *

Typical workflow: 16 | * 17 | *

    18 | *
  1. {@link #open} 19 | *
  2. {@link #start} transaction 20 | *
  3. {@link #getConnection}, write some data 21 | *
  4. {@link #endAndPrepare} (or {@link #failAndRollback}) 22 | *
  5. {@link #commit} / {@link #rollback} 23 | *
  6. {@link #close} 24 | *
25 | * 26 | * {@link #recover} can be used to get abandoned prepared transactions for cleanup. 27 | */ 28 | @PublicEvolving 29 | public interface XaConnectionProvider extends JdbcConnectionProvider { 30 | 31 | void open() throws Exception; 32 | 33 | boolean isOpen(); 34 | 35 | /** Start a new transaction. */ 36 | void start(Xid xid) throws Exception; 37 | 38 | /** End and then prepare the transaction. Transaction can't be resumed afterwards. */ 39 | void endAndPrepare(Xid xid) throws Exception; 40 | 41 | /** 42 | * Commit previously prepared transaction. 43 | * 44 | * @param ignoreUnknown whether to ignore {@link XAException#XAER_NOTA XAER_NOTA} error. 45 | */ 46 | void commit(Xid xid, boolean ignoreUnknown) throws TransientXaException; 47 | 48 | /** Rollback previously prepared transaction. */ 49 | void rollback(Xid xid) throws TransientXaException; 50 | 51 | /** 52 | * End transaction as {@link javax.transaction.xa.XAResource#TMFAIL failed}; in case of error, 53 | * try to roll it back. 54 | */ 55 | void failAndRollback(Xid xid) throws TransientXaException; 56 | 57 | /** 58 | * Note: this can block on some non-MVCC databases if there are ended not prepared transactions. 59 | */ 60 | Collection recover() throws TransientXaException; 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/statements/JdbcQueryStatement.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.datasource.statements; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | 22 | import java.io.Serializable; 23 | import java.sql.PreparedStatement; 24 | import java.sql.SQLException; 25 | 26 | /** 27 | * Sets {@link PreparedStatement} parameters to use in JDBC Sink based on a specific type of record. 28 | */ 29 | @PublicEvolving 30 | public interface JdbcQueryStatement extends Serializable { 31 | String query(); 32 | 33 | void statement(PreparedStatement ps, T record) throws SQLException; 34 | } 35 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/statements/SimpleJdbcQueryStatement.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.datasource.statements; 19 | 20 | import org.apache.flink.annotation.PublicEvolving; 21 | import org.apache.flink.connector.jdbc.JdbcStatementBuilder; 22 | 23 | import java.sql.PreparedStatement; 24 | import java.sql.SQLException; 25 | 26 | /** A simple implementation for {@link JdbcQueryStatement}. */ 27 | @PublicEvolving 28 | public class SimpleJdbcQueryStatement implements JdbcQueryStatement { 29 | private final String query; 30 | private final JdbcStatementBuilder statement; 31 | 32 | public SimpleJdbcQueryStatement(String query, JdbcStatementBuilder statement) { 33 | this.query = query; 34 | this.statement = statement; 35 | } 36 | 37 | @Override 38 | public String query() { 39 | return query; 40 | } 41 | 42 | @Override 43 | public void statement(PreparedStatement ps, IN data) throws SQLException { 44 | statement.accept(ps, data); 45 | } 46 | } 47 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/exceptions/EmptyTransactionXaException.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.XaTransaction; 5 | import org.apache.flink.util.FlinkRuntimeException; 6 | 7 | import javax.transaction.xa.XAException; 8 | import javax.transaction.xa.Xid; 9 | 10 | /** 11 | * Thrown by {@link XaTransaction} when RM responds with {@link 12 | * javax.transaction.xa.XAResource#XA_RDONLY XA_RDONLY} indicating that the transaction doesn't 13 | * include any changes. When such a transaction is committed RM may return an error (usually, {@link 14 | * XAException#XAER_NOTA XAER_NOTA}). 15 | */ 16 | @PublicEvolving 17 | public class EmptyTransactionXaException extends FlinkRuntimeException { 18 | private final Xid xid; 19 | 20 | public EmptyTransactionXaException(Xid xid) { 21 | super("end response XA_RDONLY, xid: " + xid); 22 | this.xid = xid; 23 | } 24 | 25 | public Xid getXid() { 26 | return xid; 27 | } 28 | } 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/exceptions/TransientXaException.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.datasource.transactions.xa.exceptions; 2 | 3 | import org.apache.flink.annotation.PublicEvolving; 4 | import org.apache.flink.util.FlinkRuntimeException; 5 | 6 | import javax.transaction.xa.XAException; 7 | 8 | /** 9 | * Indicates a transient or unknown failure from the resource manager (see {@link 10 | * XAException#XA_RBTRANSIENT XA_RBTRANSIENT}, {@link XAException#XAER_RMFAIL XAER_RMFAIL}). 11 | */ 12 | @PublicEvolving 13 | public class TransientXaException extends FlinkRuntimeException { 14 | public TransientXaException(XAException cause) { 15 | super(cause); 16 | } 17 | } 18 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/derby/database/DerbyFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.derby.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.derby.database.dialect.DerbyDialect; 26 | 27 | /** Factory for {@link DerbyDialect}. */ 28 | @Internal 29 | public class DerbyFactory implements JdbcFactory { 30 | @Override 31 | public boolean acceptsURL(String url) { 32 | return url.startsWith("jdbc:derby:"); 33 | } 34 | 35 | @Override 36 | public JdbcDialect createDialect() { 37 | return new DerbyDialect(); 38 | } 39 | 40 | @Override 41 | public JdbcCatalog createCatalog( 42 | ClassLoader classLoader, 43 | String catalogName, 44 | String defaultDatabase, 45 | String username, 46 | String pwd, 47 | String baseUrl) { 48 | throw new UnsupportedOperationException("Catalog for Derby is not supported yet."); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/derby/database/dialect/DerbyDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.derby.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter; 23 | import org.apache.flink.table.types.logical.RowType; 24 | 25 | /** 26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for 27 | * Derby. 28 | */ 29 | @Internal 30 | public class DerbyDialectConverter extends AbstractDialectConverter { 31 | 32 | private static final long serialVersionUID = 1L; 33 | 34 | @Override 35 | public String converterName() { 36 | return "Derby"; 37 | } 38 | 39 | public DerbyDialectConverter(RowType rowType) { 40 | super(rowType); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/JdbcOutputSerializer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.internal; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.api.common.typeutils.TypeSerializer; 23 | 24 | import java.io.Serializable; 25 | 26 | /** A Serializer that have in account the actual configuration. */ 27 | @Internal 28 | public class JdbcOutputSerializer implements Serializable { 29 | 30 | private final TypeSerializer typeSerializer; 31 | private boolean objectReuse; 32 | 33 | private JdbcOutputSerializer(TypeSerializer typeSerializer, boolean objectReuse) { 34 | this.typeSerializer = typeSerializer; 35 | this.objectReuse = objectReuse; 36 | } 37 | 38 | public static JdbcOutputSerializer of(TypeSerializer typeSerializer) { 39 | return of(typeSerializer, false); 40 | } 41 | 42 | public static JdbcOutputSerializer of( 43 | TypeSerializer typeSerializer, boolean objectReuse) { 44 | return new JdbcOutputSerializer<>(typeSerializer, objectReuse); 45 | } 46 | 47 | public JdbcOutputSerializer withObjectReuseEnabled(boolean enabled) { 48 | this.objectReuse = enabled; 49 | return this; 50 | } 51 | 52 | public T serialize(T record) { 53 | return this.objectReuse ? typeSerializer.copy(record) : record; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/executor/JdbcBatchStatementExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.internal.executor; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.JdbcStatementBuilder; 23 | 24 | import java.sql.Connection; 25 | import java.sql.SQLException; 26 | import java.util.function.Function; 27 | 28 | /** Executes the given JDBC statement in batch for the accumulated records. */ 29 | @Internal 30 | public interface JdbcBatchStatementExecutor { 31 | 32 | /** Create statements from connection. */ 33 | void prepareStatements(Connection connection) throws SQLException; 34 | 35 | void addToBatch(T record) throws SQLException; 36 | 37 | /** Submits a batch of commands to the database for execution. */ 38 | void executeBatch() throws SQLException; 39 | 40 | /** Close JDBC related statements. */ 41 | void closeStatements() throws SQLException; 42 | 43 | static JdbcBatchStatementExecutor keyed( 44 | String sql, Function keyExtractor, JdbcStatementBuilder statementBuilder) { 45 | return new KeyedBatchStatementExecutor<>(sql, keyExtractor, statementBuilder); 46 | } 47 | 48 | static JdbcBatchStatementExecutor simple( 49 | String sql, JdbcStatementBuilder paramSetter) { 50 | return new SimpleBatchStatementExecutor<>(sql, paramSetter); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcInsertOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.internal.options; 19 | 20 | import org.apache.flink.util.Preconditions; 21 | 22 | import java.util.stream.IntStream; 23 | 24 | /** JDBC sink insert options. */ 25 | public class JdbcInsertOptions extends JdbcTypedQueryOptions { 26 | 27 | private static final long serialVersionUID = 1L; 28 | 29 | private final String query; 30 | 31 | public JdbcInsertOptions(String query, int[] typesArray) { 32 | super(typesArray); 33 | this.query = Preconditions.checkNotNull(query, "query is empty"); 34 | } 35 | 36 | public String getQuery() { 37 | return query; 38 | } 39 | 40 | public static JdbcInsertOptions from(String query, int firstFieldType, int... nextFieldTypes) { 41 | return new JdbcInsertOptions(query, concat(firstFieldType, nextFieldTypes)); 42 | } 43 | 44 | private static int[] concat(int first, int... next) { 45 | if (next == null || next.length == 0) { 46 | return new int[] {first}; 47 | } else { 48 | return IntStream.concat(IntStream.of(new int[] {first}), IntStream.of(next)).toArray(); 49 | } 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/internal/options/JdbcTypedQueryOptions.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.internal.options; 19 | 20 | import javax.annotation.Nullable; 21 | 22 | import java.io.Serializable; 23 | 24 | /** Jdbc query type options. */ 25 | abstract class JdbcTypedQueryOptions implements Serializable { 26 | 27 | @Nullable private final int[] fieldTypes; 28 | 29 | JdbcTypedQueryOptions(int[] fieldTypes) { 30 | this.fieldTypes = fieldTypes; 31 | } 32 | 33 | public int[] getFieldTypes() { 34 | return fieldTypes; 35 | } 36 | 37 | public abstract static class JdbcUpdateQueryOptionsBuilder< 38 | T extends JdbcUpdateQueryOptionsBuilder> { 39 | int[] fieldTypes; 40 | 41 | protected abstract T self(); 42 | 43 | public T withFieldTypes(int[] fieldTypes) { 44 | this.fieldTypes = fieldTypes; 45 | return self(); 46 | } 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/split/JdbcGenericParameterValuesProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.split; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.jdbc.JdbcInputFormat; 23 | 24 | import java.io.Serializable; 25 | 26 | /** 27 | * This splits generator actually does nothing but wrapping the query parameters computed by the 28 | * user before creating the {@link JdbcInputFormat} instance. 29 | */ 30 | @PublicEvolving 31 | public class JdbcGenericParameterValuesProvider implements JdbcParameterValuesProvider { 32 | 33 | private final Serializable[][] parameters; 34 | 35 | public JdbcGenericParameterValuesProvider(Serializable[][] parameters) { 36 | this.parameters = parameters; 37 | } 38 | 39 | @Override 40 | public Serializable[][] getParameterValues() { 41 | // do nothing...precomputed externally 42 | return parameters; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/split/JdbcParameterValuesProvider.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.split; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.connector.jdbc.JdbcInputFormat; 23 | 24 | import java.io.Serializable; 25 | 26 | /** 27 | * This interface is used by the {@link JdbcInputFormat} to compute the list of parallel query to 28 | * run (i.e. splits). Each query will be parameterized using a row of the matrix provided by each 29 | * {@link JdbcParameterValuesProvider} implementation. 30 | */ 31 | @PublicEvolving 32 | public interface JdbcParameterValuesProvider extends Serializable { 33 | 34 | /** Returns the necessary parameters array to use for query in parallel a table. */ 35 | Serializable[][] getParameterValues(); 36 | 37 | /** Get the latest optional state data. */ 38 | default Serializable getLatestOptionalState() { 39 | return null; 40 | } 41 | 42 | /** Set the optional state data. */ 43 | default void setOptionalState(Serializable optionalState) {} 44 | } 45 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/java/org/apache/flink/connector/jdbc/statement/StatementFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.statement; 20 | 21 | import java.sql.Connection; 22 | import java.sql.SQLException; 23 | 24 | /** A factory to create {@link FieldNamedPreparedStatement} with the given {@link Connection}. */ 25 | public interface StatementFactory { 26 | 27 | /** Creates {@link FieldNamedPreparedStatement} with the given {@link Connection}. */ 28 | FieldNamedPreparedStatement createStatement(Connection connection) throws SQLException; 29 | } 30 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.derby.database.DerbyFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.core.table.JdbcDynamicTableFactory 17 | org.apache.flink.connector.jdbc.core.database.catalog.factory.JdbcCatalogFactory 18 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/base/source/reader/splitreader/TestingSplitsChange.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.base.source.reader.splitreader; 20 | 21 | import org.apache.flink.connector.jdbc.core.datastream.source.split.JdbcSourceSplit; 22 | 23 | import java.util.List; 24 | 25 | /** Test util class for {@link SplitsChange}. */ 26 | public class TestingSplitsChange extends SplitsChange { 27 | 28 | public TestingSplitsChange(List splits) { 29 | super(splits); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/JdbcTestCheckpoint.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc; 19 | 20 | /** Holds id and indices of items in {@link JdbcTestFixture#TEST_DATA}. */ 21 | public class JdbcTestCheckpoint { 22 | public final long id; 23 | public final int[] dataItemsIdx; 24 | 25 | JdbcTestCheckpoint(long id, int... dataItemsIdx) { 26 | this.id = id; 27 | this.dataItemsIdx = dataItemsIdx; 28 | } 29 | 30 | public JdbcTestCheckpoint withCheckpointId(long id) { 31 | return new JdbcTestCheckpoint(id, dataItemsIdx); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/database/catalog/AbstractJdbcCatalogTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.core.database.catalog; 20 | 21 | import org.junit.jupiter.api.Test; 22 | 23 | import static org.assertj.core.api.Assertions.assertThatThrownBy; 24 | 25 | /** Test for {@link AbstractJdbcCatalog}. */ 26 | class AbstractJdbcCatalogTest { 27 | 28 | @Test 29 | void testJdbcUrl() { 30 | AbstractJdbcCatalog.validateJdbcUrl("jdbc:dialect://localhost:1234/"); 31 | AbstractJdbcCatalog.validateJdbcUrl("jdbc:dialect://localhost:1234"); 32 | } 33 | 34 | @Test 35 | void testInvalidJdbcUrl() { 36 | assertThatThrownBy( 37 | () -> 38 | AbstractJdbcCatalog.validateJdbcUrl( 39 | "jdbc:dialect://localhost:1234/db")) 40 | .isInstanceOf(IllegalArgumentException.class); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/AtLeastOnceJdbcSinkTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.core.datastream.sink; 19 | 20 | import org.apache.flink.connector.jdbc.JdbcConnectionOptions; 21 | 22 | /** Smoke tests for the {@link JdbcSink} and the underlying classes. */ 23 | class AtLeastOnceJdbcSinkTest extends BaseJdbcSinkTest { 24 | 25 | @Override 26 | protected JdbcSink finishSink(JdbcSinkBuilder builder) { 27 | return builder.buildAtLeastOnce( 28 | new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() 29 | .withUrl(getMetadata().getJdbcUrl()) 30 | .withUsername(getMetadata().getUsername()) 31 | .withPassword(getMetadata().getPassword()) 32 | .withDriverName(getMetadata().getDriverClass()) 33 | .build()); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/ExactlyOnceJdbcSinkTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.core.datastream.sink; 19 | 20 | import org.apache.flink.connector.jdbc.JdbcExactlyOnceOptions; 21 | import org.apache.flink.connector.jdbc.JdbcExecutionOptions; 22 | 23 | /** Smoke tests for the {@link JdbcSink} and the underlying classes. */ 24 | class ExactlyOnceJdbcSinkTest extends BaseJdbcSinkTest { 25 | 26 | @Override 27 | protected JdbcSink finishSink(JdbcSinkBuilder builder) { 28 | return builder.withExecutionOptions( 29 | JdbcExecutionOptions.builder().withMaxRetries(0).build()) 30 | .buildExactlyOnce( 31 | JdbcExactlyOnceOptions.defaults(), getMetadata().getXaSourceSupplier()); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/writer/JdbcWriterStateSerializerTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.datastream.sink.writer; 2 | 3 | import org.apache.flink.api.common.JobID; 4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.domain.TransactionId; 5 | 6 | import org.junit.jupiter.api.Test; 7 | 8 | import java.io.IOException; 9 | import java.util.Arrays; 10 | 11 | import static org.assertj.core.api.Assertions.assertThat; 12 | 13 | /** 14 | * Smoke test for {@link 15 | * org.apache.flink.connector.jdbc.core.datastream.sink.writer.JdbcWriterStateSerializer}. 16 | */ 17 | class JdbcWriterStateSerializerTest { 18 | 19 | @Test 20 | void testBasicSerDe() throws IOException { 21 | TransactionId baseTid = 22 | TransactionId.create( 23 | JobID.fromHexString("6b64d8a9a951e2e8767ae952ad951706").getBytes(), 1, 2); 24 | 25 | JdbcWriterState original = 26 | JdbcWriterState.of( 27 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)), 28 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L))); 29 | 30 | JdbcWriterStateSerializer tester = new JdbcWriterStateSerializer(); 31 | 32 | byte[] serialized = tester.serialize(original); 33 | JdbcWriterState deserialized = tester.deserialize(tester.getVersion(), serialized); 34 | 35 | assertThat(deserialized).isEqualTo(original); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/core/datastream/sink/writer/JdbcWriterStateTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.core.datastream.sink.writer; 2 | 3 | import org.apache.flink.api.common.JobID; 4 | import org.apache.flink.connector.jdbc.datasource.transactions.xa.domain.TransactionId; 5 | 6 | import org.junit.jupiter.api.Test; 7 | 8 | import java.util.Arrays; 9 | 10 | import static org.assertj.core.api.Assertions.assertThat; 11 | 12 | /** 13 | * Smoke test for {@link 14 | * org.apache.flink.connector.jdbc.core.datastream.sink.writer.JdbcWriterState}. 15 | */ 16 | class JdbcWriterStateTest { 17 | 18 | @Test 19 | void testEquals() { 20 | assertThat(JdbcWriterState.empty()).isEqualTo(JdbcWriterState.empty()); 21 | 22 | TransactionId baseTid = 23 | TransactionId.create( 24 | JobID.fromHexString("6b64d8a9a951e2e8767ae952ad951706").getBytes(), 1, 2); 25 | 26 | JdbcWriterState state1 = 27 | JdbcWriterState.of( 28 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)), 29 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L))); 30 | JdbcWriterState state2 = 31 | JdbcWriterState.of( 32 | Arrays.asList(baseTid.withBranch(1001L), baseTid.withBranch(1002L)), 33 | Arrays.asList(baseTid.withBranch(2001L), baseTid.withBranch(2002L))); 34 | 35 | assertThat(state1).isEqualTo(state2); 36 | } 37 | } 38 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/datasource/transactions/xa/xid/XidSerializersTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.datasource.transactions.xa.xid; 19 | 20 | import org.apache.flink.api.common.typeutils.SerializerTestBase; 21 | import org.apache.flink.api.common.typeutils.TypeSerializer; 22 | 23 | import javax.transaction.xa.Xid; 24 | 25 | /** XaSerializersTest. */ 26 | class XidSerializersTest extends SerializerTestBase { 27 | 28 | @Override 29 | protected TypeSerializer createSerializer() { 30 | return new XidSerializer(); 31 | } 32 | 33 | @Override 34 | protected int getLength() { 35 | return -1; 36 | } 37 | 38 | @Override 39 | protected Class getTypeClass() { 40 | return Xid.class; 41 | } 42 | 43 | @Override 44 | protected Xid[] getTestData() { 45 | return new Xid[] {XaXidTest.XID}; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/DerbyTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.derby; 20 | 21 | import org.apache.flink.connector.jdbc.derby.testutils.DerbyDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for Derby testing. */ 28 | @ExtendWith(DerbyDatabase.class) 29 | public interface DerbyTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return DerbyDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/table/DerbyDynamicTableSinkITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.derby.table; 20 | 21 | import org.apache.flink.connector.jdbc.core.table.sink.JdbcDynamicTableSinkITCase; 22 | import org.apache.flink.connector.jdbc.derby.DerbyTestBase; 23 | import org.apache.flink.connector.jdbc.derby.database.dialect.DerbyDialect; 24 | 25 | /** The Table Sink ITCase for {@link DerbyDialect}. */ 26 | class DerbyDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase implements DerbyTestBase {} 27 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/derby/testutils/DerbyMetadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.derby.testutils; 19 | 20 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 21 | 22 | import org.apache.derby.jdbc.EmbeddedXADataSource; 23 | 24 | import javax.sql.XADataSource; 25 | 26 | /** Derby Metadata. */ 27 | public class DerbyMetadata implements DatabaseMetadata { 28 | private final String dbName; 29 | 30 | public DerbyMetadata(String schemaName) { 31 | dbName = "memory:" + schemaName; 32 | } 33 | 34 | public String getDbName() { 35 | return dbName; 36 | } 37 | 38 | @Override 39 | public String getJdbcUrl() { 40 | return String.format("jdbc:derby:%s", dbName); 41 | } 42 | 43 | @Override 44 | public String getJdbcUrlWithCredentials() { 45 | return getJdbcUrl(); 46 | } 47 | 48 | @Override 49 | public String getUsername() { 50 | return ""; 51 | } 52 | 53 | @Override 54 | public String getPassword() { 55 | return ""; 56 | } 57 | 58 | @Override 59 | public XADataSource buildXaDataSource() { 60 | EmbeddedXADataSource ds = new EmbeddedXADataSource(); 61 | ds.setDatabaseName(dbName); 62 | return ds; 63 | } 64 | 65 | @Override 66 | public String getDriverClass() { 67 | return "org.apache.derby.jdbc.EmbeddedDriver"; 68 | } 69 | 70 | @Override 71 | public String getVersion() { 72 | return "derby:memory"; 73 | } 74 | } 75 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/FakeDBUtils.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.fakedb; 20 | 21 | /** Utilities and constants for FakeDB. */ 22 | public class FakeDBUtils { 23 | public static final String URL_PREFIX = "jdbc:fake:"; 24 | 25 | public static final String TEST_DB_URL = composeDBUrl("test"); 26 | public static final String TEST_DB_INVALID_URL = "jdbc:no-existing-driver:test"; 27 | 28 | public static final String DRIVER1_CLASS_NAME = 29 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver1"; 30 | public static final String DRIVER2_CLASS_NAME = 31 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver2"; 32 | public static final String DRIVER3_CLASS_NAME = 33 | "org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver3"; 34 | 35 | public static String composeDBUrl(String db) { 36 | return URL_PREFIX + db; 37 | } 38 | 39 | public static boolean acceptsUrl(String url) { 40 | return url.startsWith(URL_PREFIX); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection1.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.fakedb.driver; 20 | 21 | import java.util.Properties; 22 | 23 | /** Sql connection created by {@link FakeDriver1#connect(String, Properties)}. */ 24 | public class FakeConnection1 extends FakeConnection {} 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection2.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.fakedb.driver; 20 | 21 | import java.util.Properties; 22 | 23 | /** Sql connection created by {@link FakeDriver2#connect(String, Properties)}. */ 24 | public class FakeConnection2 extends FakeConnection {} 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeConnection3.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.fakedb.driver; 20 | 21 | import java.util.Properties; 22 | 23 | /** Sql connection created by {@link FakeDriver3#connect(String, Properties)}. */ 24 | public class FakeConnection3 extends FakeConnection {} 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/fakedb/driver/FakeDriver3.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.fakedb.driver; 20 | 21 | import org.apache.flink.connector.jdbc.fakedb.FakeDBUtils; 22 | 23 | import java.sql.Connection; 24 | import java.sql.Driver; 25 | import java.sql.DriverPropertyInfo; 26 | import java.sql.SQLException; 27 | import java.sql.SQLFeatureNotSupportedException; 28 | import java.util.Properties; 29 | import java.util.logging.Logger; 30 | 31 | /** Yet another {@link Driver} for FakeDB. */ 32 | public class FakeDriver3 implements Driver { 33 | 34 | @Override 35 | public Connection connect(String url, Properties info) throws SQLException { 36 | if (!acceptsURL(url)) { 37 | return null; 38 | } 39 | return new FakeConnection3(); 40 | } 41 | 42 | @Override 43 | public boolean acceptsURL(String url) throws SQLException { 44 | return FakeDBUtils.acceptsUrl(url); 45 | } 46 | 47 | @Override 48 | public DriverPropertyInfo[] getPropertyInfo(String url, Properties info) throws SQLException { 49 | return new DriverPropertyInfo[0]; 50 | } 51 | 52 | @Override 53 | public int getMajorVersion() { 54 | return 0; 55 | } 56 | 57 | @Override 58 | public int getMinorVersion() { 59 | return 0; 60 | } 61 | 62 | @Override 63 | public boolean jdbcCompliant() { 64 | return false; 65 | } 66 | 67 | @Override 68 | public Logger getParentLogger() throws SQLFeatureNotSupportedException { 69 | throw new SQLFeatureNotSupportedException(); 70 | } 71 | } 72 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/H2XaTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.h2; 20 | 21 | import org.apache.flink.connector.jdbc.h2.testutils.H2XaDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for H2 Xa testing. */ 28 | @ExtendWith(H2XaDatabase.class) 29 | public interface H2XaTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return H2XaDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/testutils/H2Metadata.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.h2.testutils; 19 | 20 | import org.apache.flink.connector.jdbc.h2.testutils.xa.H2XaDsWrapper; 21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 22 | 23 | import javax.sql.XADataSource; 24 | 25 | /** H2 Metadata. */ 26 | public class H2Metadata implements DatabaseMetadata { 27 | 28 | private final String schema; 29 | 30 | public H2Metadata(String schema) { 31 | this.schema = schema; 32 | } 33 | 34 | @Override 35 | public String getJdbcUrl() { 36 | return String.format("jdbc:h2:mem:%s", schema); 37 | } 38 | 39 | @Override 40 | public String getJdbcUrlWithCredentials() { 41 | return getJdbcUrl(); 42 | } 43 | 44 | @Override 45 | public String getUsername() { 46 | return ""; 47 | } 48 | 49 | @Override 50 | public String getPassword() { 51 | return ""; 52 | } 53 | 54 | @Override 55 | public XADataSource buildXaDataSource() { 56 | final org.h2.jdbcx.JdbcDataSource ds = new org.h2.jdbcx.JdbcDataSource(); 57 | ds.setUrl(getJdbcUrl()); 58 | return new H2XaDsWrapper(ds); 59 | } 60 | 61 | @Override 62 | public String getDriverClass() { 63 | return "org.h2.Driver"; 64 | } 65 | 66 | @Override 67 | public String getVersion() { 68 | return "h2:mem"; 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/h2/testutils/xa/package-info.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | /** 19 | * This package holds some workarounds for the H2 XA client, plus {@link 20 | * org.apache.flink.connector.jdbc.h2.testutils.H2Metadata}. Used only for testing. 21 | */ 22 | package org.apache.flink.connector.jdbc.h2.testutils.xa; 23 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/internal/JdbcOutputSerializerTest.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.internal; 2 | 3 | import org.apache.flink.api.common.ExecutionConfig; 4 | import org.apache.flink.api.common.typeinfo.TypeInformation; 5 | import org.apache.flink.api.common.typeutils.TypeSerializer; 6 | import org.apache.flink.types.Row; 7 | 8 | import org.junit.jupiter.api.Test; 9 | 10 | import static org.assertj.core.api.Assertions.assertThat; 11 | 12 | class JdbcOutputSerializerTest { 13 | 14 | @Test 15 | void testSerializer() { 16 | TypeInformation typeInformation = TypeInformation.of(Row.class); 17 | TypeSerializer typeSerializer = 18 | typeInformation.createSerializer(new ExecutionConfig().getSerializerConfig()); 19 | JdbcOutputSerializer serializer = JdbcOutputSerializer.of(typeSerializer); 20 | 21 | Row original = Row.of(123); 22 | Row noReuse = serializer.withObjectReuseEnabled(false).serialize(original); 23 | Row withReuse = serializer.withObjectReuseEnabled(true).serialize(original); 24 | 25 | assertThat(noReuse).isEqualTo(original); 26 | assertThat(withReuse).isEqualTo(original); 27 | 28 | original.setField(0, 321); 29 | 30 | // if disable object is reusable 31 | assertThat(noReuse).isEqualTo(original); 32 | // if enabled object is duplicate 33 | assertThat(withReuse).isNotEqualTo(original); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseResource.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.testutils; 2 | 3 | import org.junit.jupiter.api.extension.ExtensionContext.Store.CloseableResource; 4 | 5 | /** Database resource for testing. */ 6 | public interface DatabaseResource extends CloseableResource { 7 | 8 | void start(); 9 | 10 | void stop(); 11 | 12 | default void close() throws Throwable { 13 | stop(); 14 | } 15 | } 16 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/DatabaseTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.testutils; 19 | 20 | import java.util.Collections; 21 | import java.util.List; 22 | 23 | /** Base interface for tests that have dependency in a database. */ 24 | public interface DatabaseTest { 25 | 26 | DatabaseMetadata getMetadata(); 27 | 28 | default List getManagedTables() { 29 | return Collections.emptyList(); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/JdbcITCaseBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.testutils; 20 | 21 | import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; 22 | import org.apache.flink.test.junit5.MiniClusterExtension; 23 | 24 | import org.junit.jupiter.api.extension.RegisterExtension; 25 | 26 | /** A base for ITCase implementations. */ 27 | public interface JdbcITCaseBase { 28 | 29 | @RegisterExtension 30 | MiniClusterExtension MINI_CLUSTER = 31 | new MiniClusterExtension( 32 | new MiniClusterResourceConfiguration.Builder() 33 | .setNumberTaskManagers(2) 34 | .setNumberSlotsPerTaskManager(2) 35 | .build()); 36 | } 37 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/TableManaged.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.testutils; 20 | 21 | import java.sql.Connection; 22 | import java.sql.SQLException; 23 | 24 | /** Table that can be manage by {@link DatabaseExtension}. */ 25 | public interface TableManaged { 26 | 27 | String getTableName(); 28 | 29 | void createTable(Connection conn) throws SQLException; 30 | 31 | void deleteTable(Connection conn) throws SQLException; 32 | 33 | void dropTable(Connection conn) throws SQLException; 34 | } 35 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/functions/JdbcResultSetBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.testutils.functions; 20 | 21 | import java.io.Serializable; 22 | import java.sql.ResultSet; 23 | import java.sql.SQLException; 24 | import java.util.List; 25 | 26 | /** ResultSet builder. * */ 27 | @FunctionalInterface 28 | public interface JdbcResultSetBuilder extends Serializable { 29 | List accept(ResultSet rs) throws SQLException; 30 | } 31 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/resources/DockerResource.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.testutils.resources; 2 | 3 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource; 4 | 5 | import com.github.dockerjava.api.DockerClient; 6 | import org.slf4j.Logger; 7 | import org.slf4j.LoggerFactory; 8 | import org.testcontainers.DockerClientFactory; 9 | import org.testcontainers.containers.GenericContainer; 10 | import org.testcontainers.containers.JdbcDatabaseContainer; 11 | 12 | import java.util.Arrays; 13 | 14 | /** Docker based database resource. */ 15 | public class DockerResource implements DatabaseResource { 16 | 17 | protected static final Logger LOG = LoggerFactory.getLogger(DockerResource.class); 18 | 19 | private final JdbcDatabaseContainer container; 20 | 21 | public DockerResource(JdbcDatabaseContainer container) { 22 | this.container = container; 23 | } 24 | 25 | @Override 26 | public void start() { 27 | this.container.start(); 28 | } 29 | 30 | @Override 31 | public void stop() { 32 | this.container.stop(); 33 | } 34 | 35 | @Override 36 | public void close() throws Throwable { 37 | stop(); 38 | cleanContainers(container); 39 | } 40 | 41 | public static void cleanContainers(GenericContainer container) { 42 | try { 43 | DockerClient client = DockerClientFactory.instance().client(); 44 | // client.removeImageCmd(container.getDockerImageName()).exec(); 45 | client.listImagesCmd().exec().stream() 46 | .filter( 47 | image -> 48 | Arrays.stream(image.getRepoTags()) 49 | .anyMatch( 50 | tag -> 51 | !tag.contains("testcontainers/ryuk") 52 | && !tag.contains( 53 | container 54 | .getDockerImageName()))) 55 | .forEach(image -> client.removeImageCmd(image.getId()).exec()); 56 | 57 | } catch (Exception ignore) { 58 | LOG.warn("Error deleting image."); 59 | } 60 | } 61 | } 62 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/resources/MemoryResource.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.testutils.resources; 2 | 3 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource; 4 | 5 | /** Memory based database resource. */ 6 | public interface MemoryResource extends DatabaseResource {} 7 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/testutils/tables/TableBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.testutils.tables; 20 | 21 | import org.apache.flink.table.types.DataType; 22 | 23 | /** Table builder. * */ 24 | public final class TableBuilder { 25 | 26 | public static TableRow tableRow(String name, TableField... fields) { 27 | return new TableRow(name, fields); 28 | } 29 | 30 | public static TableField field(String name, DataType dataType) { 31 | return field(name, null, dataType); 32 | } 33 | 34 | public static TableField field(String name, TableField.DbType dbType, DataType dataType) { 35 | return createField(name, dbType, dataType, false); 36 | } 37 | 38 | public static TableField pkField(String name, DataType dataType) { 39 | return pkField(name, null, dataType); 40 | } 41 | 42 | public static TableField pkField(String name, TableField.DbType dbType, DataType dataType) { 43 | return createField(name, dbType, dataType, true); 44 | } 45 | 46 | public static TableField.DbType dbType(String type) { 47 | return new TableField.DbType(type); 48 | } 49 | 50 | private static TableField createField( 51 | String name, TableField.DbType dbType, DataType dataType, boolean pkField) { 52 | return new TableField(name, dataType, dbType, pkField); 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/java/org/apache/flink/connector/jdbc/utils/JdbcTypeUtilTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.utils; 20 | 21 | import org.apache.flink.table.types.logical.LogicalTypeRoot; 22 | 23 | import org.junit.jupiter.api.Test; 24 | 25 | import java.sql.Types; 26 | 27 | import static org.apache.flink.connector.jdbc.utils.JdbcTypeUtil.logicalTypeToSqlType; 28 | import static org.assertj.core.api.Assertions.assertThat; 29 | import static org.assertj.core.api.Assertions.assertThatThrownBy; 30 | 31 | /** Testing the type conversions from Flink to SQL types. */ 32 | class JdbcTypeUtilTest { 33 | 34 | @Test 35 | void testTypeConversions() { 36 | assertThat(logicalTypeToSqlType(LogicalTypeRoot.INTEGER)).isEqualTo(Types.INTEGER); 37 | testUnsupportedType(LogicalTypeRoot.RAW); 38 | testUnsupportedType(LogicalTypeRoot.MAP); 39 | } 40 | 41 | private static void testUnsupportedType(LogicalTypeRoot logicalTypeRoot) { 42 | assertThatThrownBy(() -> logicalTypeToSqlType(logicalTypeRoot)) 43 | .isInstanceOf(IllegalArgumentException.class); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/resources/META-INF/services/java.sql.Driver: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver1 17 | org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver2 18 | # Comment intentionally for unregistered driver 19 | # org.apache.flink.connector.jdbc.fakedb.driver.FakeDriver3 20 | -------------------------------------------------------------------------------- /flink-connector-jdbc-core/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/CrateDBFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.cratedb.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.cratedb.database.catalog.CrateDBCatalog; 26 | import org.apache.flink.connector.jdbc.cratedb.database.dialect.CrateDBDialect; 27 | 28 | /** Factory for {@link CrateDBDialect}. */ 29 | @Internal 30 | public class CrateDBFactory implements JdbcFactory { 31 | @Override 32 | public boolean acceptsURL(String url) { 33 | return url.startsWith("jdbc:crate:"); 34 | } 35 | 36 | @Override 37 | public JdbcDialect createDialect() { 38 | return new CrateDBDialect(); 39 | } 40 | 41 | @Override 42 | public JdbcCatalog createCatalog( 43 | ClassLoader classLoader, 44 | String catalogName, 45 | String defaultDatabase, 46 | String username, 47 | String pwd, 48 | String baseUrl) { 49 | return new CrateDBCatalog( 50 | classLoader, catalogName, defaultDatabase, username, pwd, baseUrl); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/catalog/CrateDBTypeMapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.cratedb.database.catalog; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.postgres.database.catalog.PostgresTypeMapper; 23 | import org.apache.flink.table.api.DataTypes; 24 | import org.apache.flink.table.types.DataType; 25 | 26 | import org.slf4j.Logger; 27 | import org.slf4j.LoggerFactory; 28 | 29 | /** CrateDBTypeMapper util class. */ 30 | @Internal 31 | public class CrateDBTypeMapper extends PostgresTypeMapper { 32 | 33 | private static final Logger LOG = LoggerFactory.getLogger(CrateDBTypeMapper.class); 34 | 35 | // CrateDB jdbc driver uses very similar mapping 36 | // to PostgreSQL driver, and adds some extras: 37 | private static final String PG_STRING = "string"; 38 | private static final String PG_STRING_ARRAY = "_string"; 39 | 40 | @Override 41 | protected DataType getMapping(String pgType, int precision, int scale) { 42 | switch (pgType) { 43 | case PG_SERIAL: 44 | case PG_BIGSERIAL: 45 | return null; 46 | case PG_STRING: 47 | return DataTypes.STRING(); 48 | case PG_STRING_ARRAY: 49 | return DataTypes.ARRAY(DataTypes.STRING()); 50 | default: 51 | return super.getMapping(pgType, precision, scale); 52 | } 53 | } 54 | 55 | @Override 56 | protected String getDBType() { 57 | return "CrateDB"; 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/dialect/CrateDBDialect.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.cratedb.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialect; 23 | import org.apache.flink.table.types.logical.RowType; 24 | 25 | import java.util.Optional; 26 | 27 | /** JDBC dialect for CrateDB. */ 28 | @Internal 29 | public class CrateDBDialect extends CompatiblePostgresDialect { 30 | 31 | private static final long serialVersionUID = 1L; 32 | 33 | @Override 34 | public String compatibleDialectName() { 35 | return "CrateDB"; 36 | } 37 | 38 | @Override 39 | public CrateDBDialectConverter compatibleRowConverter(RowType rowType) { 40 | return new CrateDBDialectConverter(rowType); 41 | } 42 | 43 | @Override 44 | public Optional compatibleDriverName() { 45 | return Optional.of("io.crate.client.jdbc.CrateDriver"); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/main/java/org/apache/flink/connector/jdbc/cratedb/database/dialect/CrateDBDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.cratedb.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.postgres.database.dialect.CompatiblePostgresDialectConverter; 23 | import org.apache.flink.table.types.logical.RowType; 24 | 25 | /** 26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for 27 | * CrateDB. 28 | */ 29 | @Internal 30 | public class CrateDBDialectConverter extends CompatiblePostgresDialectConverter { 31 | 32 | private static final long serialVersionUID = 1L; 33 | 34 | public CrateDBDialectConverter(RowType rowType) { 35 | super(rowType); 36 | } 37 | 38 | @Override 39 | public String compatibleConverterName() { 40 | return "CrateDB"; 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.cratedb.database.CrateDBFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/test/java/org/apache/flink/connector/jdbc/cratedb/CrateDBTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.cratedb; 20 | 21 | import org.apache.flink.connector.jdbc.cratedb.testutils.CrateDBDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for CrateDB testing. */ 28 | @ExtendWith(CrateDBDatabase.class) 29 | public interface CrateDBTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return CrateDBDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/test/java/org/apache/flink/connector/jdbc/cratedb/testutils/CrateDBImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.cratedb.testutils; 19 | 20 | /** Postgres docker images. */ 21 | public interface CrateDBImages { 22 | String CRATEDB_5 = "crate:5.7.2"; 23 | } 24 | -------------------------------------------------------------------------------- /flink-connector-jdbc-cratedb/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/main/java/org/apache/flink/connector/jdbc/db2/database/Db2Factory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.db2.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.db2.database.dialect.Db2Dialect; 26 | 27 | /** Factory for {@link Db2Dialect}. */ 28 | @Internal 29 | public class Db2Factory implements JdbcFactory { 30 | @Override 31 | public boolean acceptsURL(String url) { 32 | return url.startsWith("jdbc:db2:"); 33 | } 34 | 35 | @Override 36 | public JdbcDialect createDialect() { 37 | return new Db2Dialect(); 38 | } 39 | 40 | @Override 41 | public JdbcCatalog createCatalog( 42 | ClassLoader classLoader, 43 | String catalogName, 44 | String defaultDatabase, 45 | String username, 46 | String pwd, 47 | String baseUrl) { 48 | throw new UnsupportedOperationException("Catalog for DB2 is not supported yet."); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/main/java/org/apache/flink/connector/jdbc/db2/database/dialect/Db2DialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.db2.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter; 23 | import org.apache.flink.table.types.logical.LogicalType; 24 | import org.apache.flink.table.types.logical.RowType; 25 | 26 | /** 27 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for 28 | * Db2. 29 | */ 30 | @Internal 31 | public class Db2DialectConverter extends AbstractDialectConverter { 32 | 33 | private static final long serialVersionUID = 1L; 34 | 35 | @Override 36 | public String converterName() { 37 | return "Db2"; 38 | } 39 | 40 | public Db2DialectConverter(RowType rowType) { 41 | super(rowType); 42 | } 43 | 44 | @Override 45 | protected JdbcDeserializationConverter createInternalConverter(LogicalType type) { 46 | return super.createInternalConverter(type); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.db2.database.Db2Factory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/Db2TestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.db2; 20 | 21 | import org.apache.flink.connector.jdbc.db2.testutils.Db2Database; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for Db2 testing. */ 28 | @ExtendWith(Db2Database.class) 29 | public interface Db2TestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return Db2Database.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/database/dialect/Db2DialectTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.db2.database.dialect; 20 | 21 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialectTest; 22 | import org.apache.flink.connector.jdbc.db2.Db2TestBase; 23 | 24 | import java.util.Arrays; 25 | import java.util.List; 26 | 27 | class Db2DialectTest extends JdbcDialectTest implements Db2TestBase { 28 | 29 | @Override 30 | protected List testData() { 31 | return Arrays.asList( 32 | createTestItem("CHAR"), 33 | createTestItem("VARCHAR"), 34 | createTestItem("BOOLEAN"), 35 | createTestItem("TINYINT"), 36 | createTestItem("SMALLINT"), 37 | createTestItem("INTEGER"), 38 | createTestItem("BIGINT"), 39 | createTestItem("FLOAT"), 40 | createTestItem("DOUBLE"), 41 | createTestItem("DECIMAL(10, 4)"), 42 | createTestItem("DECIMAL(31, 18)"), 43 | createTestItem("DATE"), 44 | createTestItem("TIME"), 45 | createTestItem("TIMESTAMP(3)"), 46 | createTestItem("TIMESTAMP WITHOUT TIME ZONE"), 47 | 48 | // Not valid data 49 | createTestItem("BINARY", "The Db2 dialect doesn't support type: BINARY(1)."), 50 | createTestItem( 51 | "VARBINARY(10)", "The Db2 dialect doesn't support type: VARBINARY(10).")); 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/test/java/org/apache/flink/connector/jdbc/db2/testutils/Db2Images.java: -------------------------------------------------------------------------------- 1 | package org.apache.flink.connector.jdbc.db2.testutils; 2 | 3 | /** DB2 docker images. */ 4 | public interface Db2Images { 5 | String DB2_11 = "icr.io/db2_community/db2:11.5.8.0"; 6 | } 7 | -------------------------------------------------------------------------------- /flink-connector-jdbc-db2/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/main/java/org/apache/flink/connector/jdbc/mysql/database/MySqlFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.mysql.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.mysql.database.catalog.MySqlCatalog; 26 | import org.apache.flink.connector.jdbc.mysql.database.dialect.MySqlDialect; 27 | 28 | /** Factory for {@link MySqlDialect}. */ 29 | @Internal 30 | public class MySqlFactory implements JdbcFactory { 31 | @Override 32 | public boolean acceptsURL(String url) { 33 | return url.startsWith("jdbc:mysql:"); 34 | } 35 | 36 | @Override 37 | public JdbcDialect createDialect() { 38 | return new MySqlDialect(); 39 | } 40 | 41 | @Override 42 | public JdbcCatalog createCatalog( 43 | ClassLoader classLoader, 44 | String catalogName, 45 | String defaultDatabase, 46 | String username, 47 | String pwd, 48 | String baseUrl) { 49 | return new MySqlCatalog(classLoader, catalogName, defaultDatabase, username, pwd, baseUrl); 50 | } 51 | } 52 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/main/java/org/apache/flink/connector/jdbc/mysql/database/dialect/MySQLDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.mysql.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter; 23 | import org.apache.flink.table.types.logical.RowType; 24 | 25 | /** 26 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for 27 | * MySQL. 28 | */ 29 | @Internal 30 | public class MySQLDialectConverter extends AbstractDialectConverter { 31 | 32 | private static final long serialVersionUID = 1L; 33 | 34 | @Override 35 | public String converterName() { 36 | return "MySQL"; 37 | } 38 | 39 | public MySQLDialectConverter(RowType rowType) { 40 | super(rowType); 41 | } 42 | } 43 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.mysql.database.MySqlFactory -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/MySqlTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.mysql; 20 | 21 | import org.apache.flink.connector.jdbc.mysql.testutils.MySqlDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for MySql testing. */ 28 | @ExtendWith(MySqlDatabase.class) 29 | public interface MySqlTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return MySqlDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/database/catalog/MySqlCatalogITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.mysql.database.catalog; 20 | 21 | import org.apache.flink.connector.jdbc.mysql.MySqlTestBase; 22 | 23 | /** E2E test for {@link MySqlCatalog}. */ 24 | class MySqlCatalogITCase extends MySqlCatalogTestBase implements MySqlTestBase {} 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/testutils/MySqlDatabase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.mysql.testutils; 19 | 20 | import org.apache.flink.connector.jdbc.testutils.DatabaseExtension; 21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource; 23 | import org.apache.flink.connector.jdbc.testutils.resources.DockerResource; 24 | import org.apache.flink.util.FlinkRuntimeException; 25 | 26 | import org.testcontainers.containers.MySQLContainer; 27 | 28 | /** A MySql database for testing. */ 29 | public class MySqlDatabase extends DatabaseExtension implements MySqlImages { 30 | 31 | private static final MySQLContainer CONTAINER = 32 | new MySqlContainer(MYSQL_8) 33 | .withXa() 34 | .withLockWaitTimeout(50_000L) 35 | .withCommand("--character-set-server=utf8") 36 | .withEnv("MYSQL_ROOT_HOST", "%"); 37 | 38 | private static MySqlMetadata metadata; 39 | 40 | public static MySqlMetadata getMetadata() { 41 | if (!CONTAINER.isRunning()) { 42 | throw new FlinkRuntimeException("Container is stopped."); 43 | } 44 | if (metadata == null) { 45 | metadata = new MySqlMetadata(CONTAINER, true); 46 | } 47 | return metadata; 48 | } 49 | 50 | @Override 51 | protected DatabaseMetadata getMetadataDB() { 52 | return getMetadata(); 53 | } 54 | 55 | @Override 56 | protected DatabaseResource getResource() { 57 | return new DockerResource(CONTAINER); 58 | } 59 | } 60 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/test/java/org/apache/flink/connector/jdbc/mysql/testutils/MySqlImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.mysql.testutils; 19 | 20 | /** MySql docker images. */ 21 | public interface MySqlImages { 22 | String MYSQL_8 = "mysql:9.0.0"; 23 | } 24 | -------------------------------------------------------------------------------- /flink-connector-jdbc-mysql/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/main/java/org/apache/flink/connector/jdbc/oceanbase/database/dialect/OceanBaseCompatibleMode.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase.database.dialect; 20 | 21 | /** Compatible mode of OceanBase. */ 22 | public enum OceanBaseCompatibleMode { 23 | MySQL, 24 | Oracle; 25 | 26 | public static OceanBaseCompatibleMode parse(String text) { 27 | if (text == null || text.trim().isEmpty()) { 28 | return OceanBaseCompatibleMode.MySQL; 29 | } 30 | switch (text.trim().toLowerCase()) { 31 | case "mysql": 32 | return OceanBaseCompatibleMode.MySQL; 33 | case "oracle": 34 | return OceanBaseCompatibleMode.Oracle; 35 | default: 36 | throw new IllegalArgumentException("Unsupported compatible mode: " + text); 37 | } 38 | } 39 | 40 | public boolean isMySQLMode() { 41 | return this == MySQL; 42 | } 43 | 44 | @Override 45 | public String toString() { 46 | return this.name().toLowerCase(); 47 | } 48 | } 49 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.oceanbase.database.OceanBaseFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/OceanBaseMysqlTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase; 20 | 21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode; 22 | import org.apache.flink.connector.jdbc.oceanbase.table.OceanBaseTableRow; 23 | import org.apache.flink.connector.jdbc.oceanbase.testutils.OceanBaseDatabase; 24 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 25 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 26 | import org.apache.flink.connector.jdbc.testutils.tables.TableField; 27 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow; 28 | 29 | import org.junit.jupiter.api.extension.ExtendWith; 30 | 31 | /** Base class for OceanBase Mysql mode testing. */ 32 | @ExtendWith(OceanBaseDatabase.class) 33 | public interface OceanBaseMysqlTestBase extends DatabaseTest { 34 | 35 | static TableRow tableRow(String name, TableField... fields) { 36 | return new OceanBaseTableRow(OceanBaseCompatibleMode.MySQL, name, fields); 37 | } 38 | 39 | @Override 40 | default DatabaseMetadata getMetadata() { 41 | return OceanBaseDatabase.getMetadata(); 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/OceanBaseOracleTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase; 20 | 21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode; 22 | import org.apache.flink.connector.jdbc.oceanbase.table.OceanBaseTableRow; 23 | import org.apache.flink.connector.jdbc.oceanbase.testutils.OceanBaseMetadata; 24 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 25 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 26 | import org.apache.flink.connector.jdbc.testutils.tables.TableField; 27 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow; 28 | 29 | /** Base class for OceanBase Oracle mode testing. */ 30 | public interface OceanBaseOracleTestBase extends DatabaseTest { 31 | 32 | static TableRow tableRow(String name, TableField... fields) { 33 | return new OceanBaseTableRow(OceanBaseCompatibleMode.Oracle, name, fields); 34 | } 35 | 36 | @Override 37 | default DatabaseMetadata getMetadata() { 38 | return new OceanBaseMetadata( 39 | System.getenv("test.oceanbase.username"), 40 | System.getenv("test.oceanbase.password"), 41 | System.getenv("test.oceanbase.url"), 42 | "com.oceanbase.jdbc.Driver", 43 | "test"); 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/database/dialect/OceanBaseDialectTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase.database.dialect; 20 | 21 | import org.junit.jupiter.api.Test; 22 | 23 | import static org.assertj.core.api.Assertions.assertThat; 24 | 25 | /** Tests for {@link OceanBaseDialect}. */ 26 | class OceanBaseDialectTest { 27 | 28 | @Test 29 | void testMysqlAppendDefaultUrlProperties() { 30 | OceanBaseDialect dialect = new OceanBaseDialect(OceanBaseCompatibleMode.MySQL); 31 | String jdbcUrl = "jdbc:oceanbase://localhost:2883/foo"; 32 | 33 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl)) 34 | .isEqualTo(jdbcUrl + "?rewriteBatchedStatements=true"); 35 | 36 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl + "?foo=bar")) 37 | .isEqualTo(jdbcUrl + "?foo=bar&rewriteBatchedStatements=true"); 38 | 39 | assertThat( 40 | dialect.appendDefaultUrlProperties( 41 | jdbcUrl + "?foo=bar&rewriteBatchedStatements=false")) 42 | .isEqualTo(jdbcUrl + "?foo=bar&rewriteBatchedStatements=false"); 43 | } 44 | 45 | @Test 46 | void testOracleAppendDefaultUrlProperties() { 47 | OceanBaseDialect dialect = new OceanBaseDialect(OceanBaseCompatibleMode.Oracle); 48 | String jdbcUrl = "jdbc:oceanbase://localhost:2883/foo"; 49 | 50 | assertThat(dialect.appendDefaultUrlProperties(jdbcUrl)).isEqualTo(jdbcUrl); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/table/OceanBaseTableRow.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase.table; 20 | 21 | import org.apache.flink.connector.jdbc.oceanbase.database.dialect.OceanBaseCompatibleMode; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.tables.TableField; 24 | import org.apache.flink.connector.jdbc.testutils.tables.TableRow; 25 | 26 | import java.util.ArrayList; 27 | import java.util.List; 28 | 29 | /** TableRow for OceanBase. */ 30 | public class OceanBaseTableRow extends TableRow { 31 | 32 | private final OceanBaseCompatibleMode compatibleMode; 33 | 34 | public OceanBaseTableRow( 35 | OceanBaseCompatibleMode compatibleMode, String name, TableField[] fields) { 36 | super(name, fields); 37 | this.compatibleMode = compatibleMode; 38 | } 39 | 40 | @Override 41 | public String getCreateQueryForFlink( 42 | DatabaseMetadata metadata, 43 | String newName, 44 | List newFields, 45 | List withParams) { 46 | List params = new ArrayList<>(withParams); 47 | params.add("'compatible-mode'='" + compatibleMode + "'"); 48 | return super.getCreateQueryForFlink(metadata, newName, newFields, params); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/java/org/apache/flink/connector/jdbc/oceanbase/testutils/OceanBaseImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oceanbase.testutils; 20 | 21 | /** OceanBase docker images. */ 22 | public interface OceanBaseImages { 23 | String OCEANBASE_CE_4 = "oceanbase/oceanbase-ce:4.2.1-lts"; 24 | } 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oceanbase/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/main/java/org/apache/flink/connector/jdbc/oracle/database/OracleFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oracle.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.oracle.database.dialect.OracleDialect; 26 | 27 | /** Factory for {@link OracleDialect}. */ 28 | @Internal 29 | public class OracleFactory implements JdbcFactory { 30 | @Override 31 | public boolean acceptsURL(String url) { 32 | return url.startsWith("jdbc:oracle:"); 33 | } 34 | 35 | @Override 36 | public JdbcDialect createDialect() { 37 | return new OracleDialect(); 38 | } 39 | 40 | @Override 41 | public JdbcCatalog createCatalog( 42 | ClassLoader classLoader, 43 | String catalogName, 44 | String defaultDatabase, 45 | String username, 46 | String pwd, 47 | String baseUrl) { 48 | throw new UnsupportedOperationException("Catalog for Oracle is not supported yet."); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.oracle.database.OracleFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/OracleTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.oracle; 20 | 21 | import org.apache.flink.connector.jdbc.oracle.testutils.OracleDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for Oracle testing. */ 28 | @ExtendWith(OracleDatabase.class) 29 | public interface OracleTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return OracleDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/testutils/OracleDatabase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.oracle.testutils; 19 | 20 | import org.apache.flink.connector.jdbc.testutils.DatabaseExtension; 21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseResource; 23 | import org.apache.flink.connector.jdbc.testutils.resources.DockerResource; 24 | import org.apache.flink.util.FlinkRuntimeException; 25 | 26 | import org.testcontainers.containers.OracleContainer; 27 | 28 | /** A Oracle database for testing. */ 29 | public class OracleDatabase extends DatabaseExtension implements OracleImages { 30 | 31 | private static final OracleContainer CONTAINER = 32 | new OracleContainer(ORACLE_21) 33 | .withStartupTimeoutSeconds(240) 34 | .withConnectTimeoutSeconds(120) 35 | .usingSid(); 36 | 37 | private static OracleMetadata metadata; 38 | 39 | public static OracleMetadata getMetadata() { 40 | if (!CONTAINER.isRunning()) { 41 | throw new FlinkRuntimeException("Container is stopped."); 42 | } 43 | if (metadata == null) { 44 | metadata = new OracleMetadata(CONTAINER, true); 45 | } 46 | return metadata; 47 | } 48 | 49 | @Override 50 | protected DatabaseMetadata getMetadataDB() { 51 | return getMetadata(); 52 | } 53 | 54 | @Override 55 | protected DatabaseResource getResource() { 56 | return new DockerResource(CONTAINER); 57 | } 58 | } 59 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/test/java/org/apache/flink/connector/jdbc/oracle/testutils/OracleImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.oracle.testutils; 19 | 20 | /** Oracle docker images. */ 21 | public interface OracleImages { 22 | String ORACLE_21 = "gvenzl/oracle-xe:21.3.0-slim-faststart"; 23 | } 24 | -------------------------------------------------------------------------------- /flink-connector-jdbc-oracle/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/PostgresFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.postgres.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.postgres.database.catalog.PostgresCatalog; 26 | import org.apache.flink.connector.jdbc.postgres.database.dialect.PostgresDialect; 27 | 28 | /** Factory for {@link PostgresDialect}. */ 29 | @Internal 30 | public class PostgresFactory implements JdbcFactory { 31 | @Override 32 | public boolean acceptsURL(String url) { 33 | return url.startsWith("jdbc:postgresql:"); 34 | } 35 | 36 | @Override 37 | public JdbcDialect createDialect() { 38 | return new PostgresDialect(); 39 | } 40 | 41 | @Override 42 | public JdbcCatalog createCatalog( 43 | ClassLoader classLoader, 44 | String catalogName, 45 | String defaultDatabase, 46 | String username, 47 | String pwd, 48 | String baseUrl) { 49 | return new PostgresCatalog( 50 | classLoader, catalogName, defaultDatabase, username, pwd, baseUrl); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/dialect/CompatiblePostgresDialect.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.postgres.database.dialect; 20 | 21 | import org.apache.flink.annotation.PublicEvolving; 22 | import org.apache.flink.table.types.logical.RowType; 23 | 24 | import java.util.Optional; 25 | 26 | /** JDBC dialect for PostgreSQL compatible databases. */ 27 | @PublicEvolving 28 | public abstract class CompatiblePostgresDialect extends PostgresDialect { 29 | 30 | private static final long serialVersionUID = 1L; 31 | 32 | protected abstract String compatibleDialectName(); 33 | 34 | protected abstract CompatiblePostgresDialectConverter compatibleRowConverter(RowType rowType); 35 | 36 | protected abstract Optional compatibleDriverName(); 37 | 38 | @Override 39 | public String dialectName() { 40 | return compatibleDialectName(); 41 | } 42 | 43 | @Override 44 | public CompatiblePostgresDialectConverter getRowConverter(RowType rowType) { 45 | return compatibleRowConverter(rowType); 46 | } 47 | 48 | @Override 49 | public Optional defaultDriverName() { 50 | return compatibleDriverName(); 51 | } 52 | } 53 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/main/java/org/apache/flink/connector/jdbc/postgres/database/dialect/CompatiblePostgresDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.postgres.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.table.types.logical.RowType; 23 | 24 | /** JDBC converter for PostgreSQL compatible databases. */ 25 | @Internal 26 | public abstract class CompatiblePostgresDialectConverter extends PostgresDialectConverter { 27 | 28 | private static final long serialVersionUID = 1L; 29 | 30 | protected CompatiblePostgresDialectConverter(RowType rowType) { 31 | super(rowType); 32 | } 33 | 34 | protected abstract String compatibleConverterName(); 35 | 36 | @Override 37 | public String converterName() { 38 | return compatibleConverterName(); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.postgres.database.PostgresFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/PostgresTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.postgres; 20 | 21 | import org.apache.flink.connector.jdbc.postgres.testutils.PostgresDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for Postgres testing. */ 28 | @ExtendWith(PostgresDatabase.class) 29 | public interface PostgresTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return PostgresDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/table/PostgresDynamicTableSinkITCase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.postgres.table; 20 | 21 | import org.apache.flink.connector.jdbc.core.table.sink.JdbcDynamicTableSinkITCase; 22 | import org.apache.flink.connector.jdbc.postgres.PostgresTestBase; 23 | import org.apache.flink.connector.jdbc.postgres.database.dialect.PostgresDialect; 24 | 25 | /** The Table Sink ITCase for {@link PostgresDialect}. */ 26 | class PostgresDynamicTableSinkITCase extends JdbcDynamicTableSinkITCase 27 | implements PostgresTestBase {} 28 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/test/java/org/apache/flink/connector/jdbc/postgres/testutils/PostgresImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.postgres.testutils; 19 | 20 | /** Postgres docker images. */ 21 | public interface PostgresImages { 22 | String POSTGRES_15 = "postgres:15.7"; 23 | String POSTGRES_16 = "postgres:16.3"; 24 | } 25 | -------------------------------------------------------------------------------- /flink-connector-jdbc-postgres/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/main/java/org/apache/flink/connector/jdbc/sqlserver/database/SqlServerFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.sqlserver.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.sqlserver.database.dialect.SqlServerDialect; 26 | 27 | /** Factory for {@link SqlServerDialect}. */ 28 | @Internal 29 | public class SqlServerFactory implements JdbcFactory { 30 | @Override 31 | public boolean acceptsURL(String url) { 32 | return url.startsWith("jdbc:sqlserver:"); 33 | } 34 | 35 | @Override 36 | public JdbcDialect createDialect() { 37 | return new SqlServerDialect(); 38 | } 39 | 40 | @Override 41 | public JdbcCatalog createCatalog( 42 | ClassLoader classLoader, 43 | String catalogName, 44 | String defaultDatabase, 45 | String username, 46 | String pwd, 47 | String baseUrl) { 48 | throw new UnsupportedOperationException("Catalog for SqlServer is not supported yet."); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/main/java/org/apache/flink/connector/jdbc/sqlserver/database/dialect/SqlServerDialectConverter.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.sqlserver.database.dialect; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.dialect.AbstractDialectConverter; 23 | import org.apache.flink.table.types.logical.LogicalType; 24 | import org.apache.flink.table.types.logical.RowType; 25 | 26 | /** 27 | * Runtime converter that responsible to convert between JDBC object and Flink internal object for 28 | * MsSql. 29 | */ 30 | @Internal 31 | public class SqlServerDialectConverter extends AbstractDialectConverter { 32 | 33 | private static final long serialVersionUID = 1L; 34 | 35 | @Override 36 | public String converterName() { 37 | return "SqlServer"; 38 | } 39 | 40 | public SqlServerDialectConverter(RowType rowType) { 41 | super(rowType); 42 | } 43 | 44 | @Override 45 | protected JdbcDeserializationConverter createInternalConverter(LogicalType type) { 46 | switch (type.getTypeRoot()) { 47 | case TINYINT: 48 | return val -> ((Short) val).byteValue(); 49 | default: 50 | return super.createInternalConverter(type); 51 | } 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.sqlserver.database.SqlServerFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/test/java/org/apache/flink/connector/jdbc/sqlserver/SqlServerTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.sqlserver; 20 | 21 | import org.apache.flink.connector.jdbc.sqlserver.testutils.SqlServerDatabase; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 23 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for SqlServer testing. */ 28 | @ExtendWith(SqlServerDatabase.class) 29 | public interface SqlServerTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return SqlServerDatabase.getMetadata(); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/test/java/org/apache/flink/connector/jdbc/sqlserver/testutils/SqlServerImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.sqlserver.testutils; 19 | 20 | import org.testcontainers.utility.DockerImageName; 21 | 22 | /** SqlServer docker images. */ 23 | public interface SqlServerImages { 24 | DockerImageName MSSQL_AZURE_SQL_EDGE = 25 | DockerImageName.parse("mcr.microsoft.com/azure-sql-edge") 26 | .asCompatibleSubstituteFor("mcr.microsoft.com/mssql/server"); 27 | } 28 | -------------------------------------------------------------------------------- /flink-connector-jdbc-sqlserver/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /flink-connector-jdbc-trino/src/main/java/org/apache/flink/connector/jdbc/trino/database/TrinoFactory.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.trino.database; 20 | 21 | import org.apache.flink.annotation.Internal; 22 | import org.apache.flink.connector.jdbc.core.database.JdbcFactory; 23 | import org.apache.flink.connector.jdbc.core.database.catalog.JdbcCatalog; 24 | import org.apache.flink.connector.jdbc.core.database.dialect.JdbcDialect; 25 | import org.apache.flink.connector.jdbc.trino.database.dialect.TrinoDialect; 26 | 27 | /** Factory for {@link TrinoDialect}. */ 28 | @Internal 29 | public class TrinoFactory implements JdbcFactory { 30 | @Override 31 | public boolean acceptsURL(String url) { 32 | return url.startsWith("jdbc:trino:"); 33 | } 34 | 35 | @Override 36 | public JdbcDialect createDialect() { 37 | return new TrinoDialect(); 38 | } 39 | 40 | @Override 41 | public JdbcCatalog createCatalog( 42 | ClassLoader classLoader, 43 | String catalogName, 44 | String defaultDatabase, 45 | String username, 46 | String pwd, 47 | String baseUrl) { 48 | throw new UnsupportedOperationException("Catalog for Trino is not supported yet."); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /flink-connector-jdbc-trino/src/main/resources/META-INF/services/org.apache.flink.connector.jdbc.core.database.JdbcFactory: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | org.apache.flink.connector.jdbc.trino.database.TrinoFactory 17 | -------------------------------------------------------------------------------- /flink-connector-jdbc-trino/src/test/java/org/apache/flink/connector/jdbc/trino/TrinoTestBase.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package org.apache.flink.connector.jdbc.trino; 20 | 21 | import org.apache.flink.connector.jdbc.testutils.DatabaseMetadata; 22 | import org.apache.flink.connector.jdbc.testutils.DatabaseTest; 23 | import org.apache.flink.connector.jdbc.trino.testutils.TrinoDatabase; 24 | 25 | import org.junit.jupiter.api.extension.ExtendWith; 26 | 27 | /** Base class for Trino testing. */ 28 | @ExtendWith(TrinoDatabase.class) 29 | public interface TrinoTestBase extends DatabaseTest { 30 | 31 | @Override 32 | default DatabaseMetadata getMetadata() { 33 | return TrinoDatabase.getMetadata(); 34 | } 35 | 36 | default DatabaseMetadata getMetadataDatabase() { 37 | return TrinoDatabase.getDatabaseMetadata(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /flink-connector-jdbc-trino/src/test/java/org/apache/flink/connector/jdbc/trino/testutils/TrinoImages.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package org.apache.flink.connector.jdbc.trino.testutils; 19 | 20 | /** Postgres docker images. */ 21 | public interface TrinoImages { 22 | String TRINO_IMAGE = "trinodb/trino:451"; 23 | } 24 | -------------------------------------------------------------------------------- /flink-connector-jdbc-trino/src/test/resources/log4j2-test.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.test.ref = TestLogger 23 | 24 | appender.testlogger.name = TestLogger 25 | appender.testlogger.type = CONSOLE 26 | appender.testlogger.target = SYSTEM_ERR 27 | appender.testlogger.layout.type = PatternLayout 28 | appender.testlogger.layout.pattern = %-4r [%t] %-5p %c %x - %m%n 29 | -------------------------------------------------------------------------------- /tools/ci/log4j.properties: -------------------------------------------------------------------------------- 1 | ################################################################################ 2 | # Licensed to the Apache Software Foundation (ASF) under one 3 | # or more contributor license agreements. See the NOTICE file 4 | # distributed with this work for additional information 5 | # regarding copyright ownership. The ASF licenses this file 6 | # to you under the Apache License, Version 2.0 (the 7 | # "License"); you may not use this file except in compliance 8 | # with the License. You may obtain a copy of the License at 9 | # 10 | # http://www.apache.org/licenses/LICENSE-2.0 11 | # 12 | # Unless required by applicable law or agreed to in writing, software 13 | # distributed under the License is distributed on an "AS IS" BASIS, 14 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | # See the License for the specific language governing permissions and 16 | # limitations under the License. 17 | ################################################################################ 18 | 19 | # Set root logger level to OFF to not flood build logs 20 | # set manually to INFO for debugging purposes 21 | rootLogger.level = OFF 22 | rootLogger.appenderRef.out.ref = ConsoleAppender 23 | 24 | # ----------------------------------------------------------------------------- 25 | # Console (use 'console') 26 | # ----------------------------------------------------------------------------- 27 | 28 | appender.console.name = ConsoleAppender 29 | appender.console.type = CONSOLE 30 | appender.console.layout.type = PatternLayout 31 | appender.console.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n 32 | 33 | # ----------------------------------------------------------------------------- 34 | # File (use 'file') 35 | # ----------------------------------------------------------------------------- 36 | appender.file.name = FileAppender 37 | appender.file.type = FILE 38 | appender.file.fileName = ${sys:log.dir}/mvn-${sys:mvn.forkNumber:-output}.log 39 | appender.file.layout.type = PatternLayout 40 | appender.file.layout.pattern = %d{HH:mm:ss,SSS} [%20t] %-5p %-60c %x - %m%n 41 | appender.file.createOnDemand = true 42 | 43 | # suppress the irrelevant (wrong) warnings from the netty channel handler 44 | logger.netty.name = org.jboss.netty.channel.DefaultChannelPipeline 45 | logger.netty.level = ERROR 46 | -------------------------------------------------------------------------------- /tools/maven/suppressions.xml: -------------------------------------------------------------------------------- 1 | 2 | 20 | 21 | 24 | 25 | 26 | 27 | --------------------------------------------------------------------------------