Browse Source

Use AdHoc datasource client in sqlTask (#14631)

* Use AdHoc datasource client in sqlTask

* Add method in DataSourceChannel to create PooledDataSourceClient and AdHocDataSourceClient
3.2.1-prepare
Wenjun Ruan 1 year ago committed by GitHub
parent
commit
4aab0b234f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 2
      .github/workflows/e2e.yml
  2. 2
      docs/docs/en/contribute/backend/spi/datasource.md
  3. 2
      docs/docs/zh/contribute/backend/spi/datasource.md
  4. 2
      dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataSourceServiceImpl.java
  5. 9
      dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataSourceServiceTest.java
  6. 51
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BaseAdHocDataSourceClient.java
  7. 87
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BasePooledDataSourceClient.java
  8. 123
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/CommonDataSourceClient.java
  9. 71
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourceClientProvider.java
  10. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourcePluginManager.java
  11. 22
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourceProcessorProvider.java
  12. 26
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/provider/JDBCDataSourceProvider.java
  13. 9
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/utils/CommonUtils.java
  14. 2
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/utils/DataSourceUtils.java
  15. 42
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/test/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BasePooledDataSourceClientTest.java
  16. 88
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/test/java/org/apache/dolphinscheduler/plugin/datasource/api/client/CommonDataSourceClientTest.java
  17. 6
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaAdHocDataSourceClient.java
  18. 13
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaDataSourceChannel.java
  19. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaPooledDataSourceClient.java
  20. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLAdHocDataSourceClient.java
  21. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLDataSourceChannel.java
  22. 41
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLPooledDataSourceClient.java
  23. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/SQLServerDataSourceChannelTest.java
  24. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseAdHocDataSourceClient.java
  25. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseDataSourceChannel.java
  26. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHousePooledDataSourceClient.java
  27. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/test/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseDataSourceChannelTest.java
  28. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengAdHocDataSourceClient.java
  29. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengDataSourceChannel.java
  30. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengPooledDataSourceClient.java
  31. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/test/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengDataSourceChannelTest.java
  32. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendAdHocDataSourceClient.java
  33. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendDataSourceChannel.java
  34. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendPooledDataSourceClient.java
  35. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/test/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendDataSourceChannelTest.java
  36. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2AdHocDataSourceClient.java
  37. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2DataSourceChannel.java
  38. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2PooledDataSourceClient.java
  39. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/test/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2DataSourceChannelTest.java
  40. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisAdHocDataSourceClient.java
  41. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisDataSourceChannel.java
  42. 28
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisPooledDataSourceClient.java
  43. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/DorisDataSourceChannelTest.java
  44. 14
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/provider/JDBCDataSourceProviderTest.java
  45. 11
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/utils/DataSourceUtilsTest.java
  46. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveAdHocDataSourceClient.java
  47. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveDataSourceChannel.java
  48. 45
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HivePooledDataSourceClient.java
  49. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/test/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveDataSourceChannelTest.java
  50. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiAdHocDataSourceClient.java
  51. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceChannel.java
  52. 75
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceClient.java
  53. 24
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiPooledDataSourceClient.java
  54. 1
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/param/KyuubiDataSourceProcessor.java
  55. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceChannelTest.java
  56. 73
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceClientTest.java
  57. 44
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiPooledDataSourceClientTest.java
  58. 14
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/provider/KyuubiJDBCDataSourceProviderTest.java
  59. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLAdHocDataSourceClient.java
  60. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLDataSourceChannel.java
  61. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLPooledDataSourceClient.java
  62. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLDataSourceChannelTest.java
  63. 14
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/provider/JDBCDataSourceProviderTest.java
  64. 9
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/utils/DataSourceUtilsTest.java
  65. 6
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBaseAdHocDataSourceClient.java
  66. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBaseDataSourceChannel.java
  67. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBasePooledDataSourceClient.java
  68. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleAdHocDataSourceClient.java
  69. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleDataSourceChannel.java
  70. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OraclePooledDataSourceClient.java
  71. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/test/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleDataSourceChannelTest.java
  72. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreAdHocSQLDataSourceClient.java
  73. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgrePooledSQLDataSourceClient.java
  74. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreSQLDataSourceChannel.java
  75. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreSQLDataSourceChannelTest.java
  76. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoAdHocDataSourceClient.java
  77. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoDataSourceChannel.java
  78. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoPooledDataSourceClient.java
  79. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/test/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoDataSourceChannelTest.java
  80. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftAdHocDataSourceClient.java
  81. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftDataSourceChannel.java
  82. 40
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftPooledDataSourceClient.java
  83. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeAdHocDataSourceClient.java
  84. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceChannel.java
  85. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakePooledDataSourceClient.java
  86. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceChannelTest.java
  87. 17
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakePooledDataSourceClientTest.java
  88. 14
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/provider/SnowflakeJDBCDataSourceProviderTest.java
  89. 29
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkAdHocDataSourceClient.java
  90. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkDataSourceChannel.java
  91. 6
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkPooledDataSourceClient.java
  92. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/test/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkDataSourceChannelTest.java
  93. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerAdHocDataSourceClient.java
  94. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerDataSourceChannel.java
  95. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerPooledDataSourceClient.java
  96. 8
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/test/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerDataSourceChannelTest.java
  97. 11
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-ssh/src/main/java/org/apache/dolphinscheduler/plugin/datasource/ssh/SSHDataSourceChannel.java
  98. 7
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksAdHocDataSourceClient.java
  99. 12
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksDataSourceChannel.java
  100. 30
      dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksPooledDataSourceClient.java
  101. Some files were not shown because too many files have changed in this diff Show More

2
.github/workflows/e2e.yml

@ -139,7 +139,7 @@ jobs:
path: /tmp path: /tmp
- name: Load Docker Images - name: Load Docker Images
run: | run: |
docker load -i /tmp/standalone-image.tar docker load -i /tmp/standalone-image.tar
- name: Run Test - name: Run Test
run: | run: |
./mvnw -B -f dolphinscheduler-e2e/pom.xml -am \ ./mvnw -B -f dolphinscheduler-e2e/pom.xml -am \

2
docs/docs/en/contribute/backend/spi/datasource.md

@ -10,7 +10,7 @@ If you are using MySQL or ORACLE data source, you need to place the correspondin
org.apache.dolphinscheduler.spi.datasource.DataSourceChannel org.apache.dolphinscheduler.spi.datasource.DataSourceChannel
org.apache.dolphinscheduler.spi.datasource.DataSourceChannelFactory org.apache.dolphinscheduler.spi.datasource.DataSourceChannelFactory
org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient org.apache.dolphinscheduler.spi.datasource.client.DataSourceClient
1. In the first step, the data source plug-in can implement the above interfaces and inherit the general client. For details, refer to the implementation of data source plug-ins such as sqlserver and mysql. The addition methods of all RDBMS plug-ins are the same. 1. In the first step, the data source plug-in can implement the above interfaces and inherit the general client. For details, refer to the implementation of data source plug-ins such as sqlserver and mysql. The addition methods of all RDBMS plug-ins are the same.

2
docs/docs/zh/contribute/backend/spi/datasource.md

@ -10,7 +10,7 @@
org.apache.dolphinscheduler.spi.datasource.DataSourceChannel org.apache.dolphinscheduler.spi.datasource.DataSourceChannel
org.apache.dolphinscheduler.spi.datasource.DataSourceChannelFactory org.apache.dolphinscheduler.spi.datasource.DataSourceChannelFactory
org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient org.apache.dolphinscheduler.spi.datasource.client.DataSourceClient
1. 第一步数据源插件实现以上接口和继承通用client即可,具体可以参考sqlserver、mysql等数据源插件实现,所有RDBMS插件的添加方式都是一样的。 1. 第一步数据源插件实现以上接口和继承通用client即可,具体可以参考sqlserver、mysql等数据源插件实现,所有RDBMS插件的添加方式都是一样的。
2. 在数据源插件pom.xml添加驱动配置 2. 在数据源插件pom.xml添加驱动配置

2
dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/DataSourceServiceImpl.java

@ -369,7 +369,7 @@ public class DataSourceServiceImpl extends BaseServiceImpl implements DataSource
} }
return result; return result;
} }
try (Connection connection = DataSourceClientProvider.getInstance().getConnection(type, connectionParam)) { try (Connection connection = DataSourceClientProvider.getAdHocConnection(type, connectionParam)) {
if (connection == null) { if (connection == null) {
log.error("Connection test to {} datasource failed, connectionParam:{}.", type.getDescp(), log.error("Connection test to {} datasource failed, connectionParam:{}.", type.getDescp(),
connectionParam); connectionParam);

9
dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/DataSourceServiceTest.java

@ -133,8 +133,6 @@ public class DataSourceServiceTest {
try ( try (
MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider = MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider =
Mockito.mockStatic(DataSourceClientProvider.class)) { Mockito.mockStatic(DataSourceClientProvider.class)) {
DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class);
mockedStaticDataSourceClientProvider.when(DataSourceClientProvider::getInstance).thenReturn(clientProvider);
Mockito.when(dataSourceMapper.queryDataSourceByName(dataSourceName.trim())).thenReturn(null); Mockito.when(dataSourceMapper.queryDataSourceByName(dataSourceName.trim())).thenReturn(null);
@ -199,9 +197,6 @@ public class DataSourceServiceTest {
MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider = MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider =
Mockito.mockStatic(DataSourceClientProvider.class)) { Mockito.mockStatic(DataSourceClientProvider.class)) {
// DATASOURCE_CONNECT_FAILED // DATASOURCE_CONNECT_FAILED
DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class);
mockedStaticDataSourceClientProvider.when(DataSourceClientProvider::getInstance).thenReturn(clientProvider);
Mockito.when(dataSourceMapper.queryDataSourceByName(postgreSqlDatasourceParam.getName())).thenReturn(null); Mockito.when(dataSourceMapper.queryDataSourceByName(postgreSqlDatasourceParam.getName())).thenReturn(null);
// SUCCESS // SUCCESS
@ -509,13 +504,13 @@ public class DataSourceServiceTest {
MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider = MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider =
Mockito.mockStatic(DataSourceClientProvider.class)) { Mockito.mockStatic(DataSourceClientProvider.class)) {
DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class); DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class);
mockedStaticDataSourceClientProvider.when(DataSourceClientProvider::getInstance).thenReturn(clientProvider);
Result result = dataSourceService.checkConnection(dataSourceType, connectionParam); Result result = dataSourceService.checkConnection(dataSourceType, connectionParam);
Assertions.assertEquals(Status.CONNECTION_TEST_FAILURE.getCode(), result.getCode().intValue()); Assertions.assertEquals(Status.CONNECTION_TEST_FAILURE.getCode(), result.getCode().intValue());
Connection connection = Mockito.mock(Connection.class); Connection connection = Mockito.mock(Connection.class);
Mockito.when(clientProvider.getConnection(Mockito.any(), Mockito.any())).thenReturn(connection); Mockito.when(DataSourceClientProvider.getAdHocConnection(Mockito.any(), Mockito.any()))
.thenReturn(connection);
result = dataSourceService.checkConnection(dataSourceType, connectionParam); result = dataSourceService.checkConnection(dataSourceType, connectionParam);
Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode().intValue()); Assertions.assertEquals(Status.SUCCESS.getCode(), result.getCode().intValue());
} }

51
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BaseAdHocDataSourceClient.java

@ -0,0 +1,51 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.api.client;
import org.apache.dolphinscheduler.plugin.datasource.api.plugin.DataSourceProcessorProvider;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection;
import java.sql.SQLException;
public abstract class BaseAdHocDataSourceClient implements AdHocDataSourceClient {
private final BaseConnectionParam baseConnectionParam;
private final DbType dbType;
protected BaseAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
this.baseConnectionParam = baseConnectionParam;
this.dbType = dbType;
}
@Override
public Connection getConnection() throws SQLException {
try {
return DataSourceProcessorProvider.getDataSourceProcessor(dbType).getConnection(baseConnectionParam);
} catch (Exception e) {
throw new SQLException("Create adhoc connection error", e);
}
}
@Override
public void close() {
// do nothing
}
}

87
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BasePooledDataSourceClient.java

@ -0,0 +1,87 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.api.client;
import static com.google.common.base.Preconditions.checkNotNull;
import org.apache.dolphinscheduler.common.constants.DataSourceConstants;
import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.utils.DataSourceUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.utils.PasswordUtils;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.commons.collections4.MapUtils;
import java.sql.Connection;
import java.sql.SQLException;
import lombok.extern.slf4j.Slf4j;
import com.zaxxer.hikari.HikariDataSource;
@Slf4j
public abstract class BasePooledDataSourceClient implements PooledDataSourceClient {
protected final BaseConnectionParam baseConnectionParam;
protected HikariDataSource dataSource;
public BasePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
this.baseConnectionParam = checkNotNull(baseConnectionParam, "baseConnectionParam is null");
this.dataSource = createDataSourcePool(baseConnectionParam, checkNotNull(dbType, "dbType is null"));
}
// todo: support multiple version databases
@Override
public HikariDataSource createDataSourcePool(BaseConnectionParam baseConnectionParam, DbType dbType) {
HikariDataSource dataSource = new HikariDataSource();
dataSource.setDriverClassName(baseConnectionParam.getDriverClassName());
dataSource.setJdbcUrl(DataSourceUtils.getJdbcUrl(dbType, baseConnectionParam));
dataSource.setUsername(baseConnectionParam.getUser());
dataSource.setPassword(PasswordUtils.decodePassword(baseConnectionParam.getPassword()));
dataSource.setMinimumIdle(PropertyUtils.getInt(DataSourceConstants.SPRING_DATASOURCE_MIN_IDLE, 5));
dataSource.setMaximumPoolSize(PropertyUtils.getInt(DataSourceConstants.SPRING_DATASOURCE_MAX_ACTIVE, 50));
dataSource.setConnectionTestQuery(baseConnectionParam.getValidationQuery());
if (MapUtils.isNotEmpty(baseConnectionParam.getOther())) {
baseConnectionParam.getOther().forEach(dataSource::addDataSourceProperty);
}
log.info("Creating HikariDataSource for {} success.", dbType.name());
return dataSource;
}
@Override
public Connection getConnection() throws SQLException {
return dataSource.getConnection();
}
@Override
public void close() {
log.info("do close dataSource {}.", baseConnectionParam.getDatabase());
try (HikariDataSource closedDatasource = dataSource) {
// only close the resource
}
}
}

123
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/client/CommonDataSourceClient.java

@ -1,123 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.api.client;
import org.apache.dolphinscheduler.plugin.datasource.api.provider.JDBCDataSourceProvider;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.commons.lang3.StringUtils;
import java.sql.Connection;
import java.sql.SQLException;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.springframework.jdbc.core.JdbcTemplate;
import com.google.common.base.Stopwatch;
import com.zaxxer.hikari.HikariDataSource;
@Slf4j
public class CommonDataSourceClient implements DataSourceClient {
public static final String COMMON_USER = "root";
public static final String COMMON_VALIDATION_QUERY = "select 1";
protected final BaseConnectionParam baseConnectionParam;
protected HikariDataSource dataSource;
protected JdbcTemplate jdbcTemplate;
public CommonDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
this.baseConnectionParam = baseConnectionParam;
preInit();
checkEnv(baseConnectionParam);
initClient(baseConnectionParam, dbType);
checkClient();
}
protected void preInit() {
log.info("preInit in CommonDataSourceClient");
}
protected void checkEnv(BaseConnectionParam baseConnectionParam) {
checkValidationQuery(baseConnectionParam);
checkUser(baseConnectionParam);
}
protected void initClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
this.dataSource = JDBCDataSourceProvider.createJdbcDataSource(baseConnectionParam, dbType);
this.jdbcTemplate = new JdbcTemplate(dataSource);
}
protected void checkUser(BaseConnectionParam baseConnectionParam) {
if (StringUtils.isBlank(baseConnectionParam.getUser())) {
setDefaultUsername(baseConnectionParam);
}
}
protected void setDefaultUsername(BaseConnectionParam baseConnectionParam) {
baseConnectionParam.setUser(COMMON_USER);
}
protected void checkValidationQuery(BaseConnectionParam baseConnectionParam) {
if (StringUtils.isBlank(baseConnectionParam.getValidationQuery())) {
setDefaultValidationQuery(baseConnectionParam);
}
}
protected void setDefaultValidationQuery(BaseConnectionParam baseConnectionParam) {
baseConnectionParam.setValidationQuery(COMMON_VALIDATION_QUERY);
}
@Override
public void checkClient() {
// Checking data source client
Stopwatch stopwatch = Stopwatch.createStarted();
try {
this.jdbcTemplate.execute(this.baseConnectionParam.getValidationQuery());
} catch (Exception e) {
throw new RuntimeException("JDBC connect failed", e);
} finally {
log.info("Time to execute check jdbc client with sql {} for {} ms ",
this.baseConnectionParam.getValidationQuery(), stopwatch.elapsed(TimeUnit.MILLISECONDS));
}
}
@Override
public Connection getConnection() {
try {
return this.dataSource.getConnection();
} catch (SQLException e) {
log.error("get druidDataSource Connection fail SQLException: {}", e.getMessage(), e);
return null;
}
}
@Override
public void close() {
log.info("do close dataSource {}.", baseConnectionParam.getDatabase());
try (HikariDataSource closedDatasource = dataSource) {
// only close the resource
}
this.jdbcTemplate = null;
}
}

71
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourceClientProvider.java

@ -20,13 +20,16 @@ package org.apache.dolphinscheduler.plugin.datasource.api.plugin;
import org.apache.dolphinscheduler.common.utils.PropertyUtils; import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.utils.DataSourceUtils; import org.apache.dolphinscheduler.plugin.datasource.api.utils.DataSourceUtils;
import org.apache.dolphinscheduler.plugin.task.api.TaskConstants; import org.apache.dolphinscheduler.plugin.task.api.TaskConstants;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.ConnectionParam; import org.apache.dolphinscheduler.spi.datasource.ConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.DataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -41,48 +44,58 @@ import com.google.common.cache.RemovalListener;
public class DataSourceClientProvider { public class DataSourceClientProvider {
private static final long duration = PropertyUtils.getLong(TaskConstants.KERBEROS_EXPIRE_TIME, 24); private static final long duration = PropertyUtils.getLong(TaskConstants.KERBEROS_EXPIRE_TIME, 24);
private static final Cache<String, DataSourceClient> uniqueId2dataSourceClientCache = CacheBuilder.newBuilder()
.expireAfterWrite(duration, TimeUnit.HOURS)
.removalListener((RemovalListener<String, DataSourceClient>) notification -> {
try (DataSourceClient closedClient = notification.getValue()) {
log.info("Datasource: {} is removed from cache due to expire", notification.getKey());
}
})
.maximumSize(100)
.build();
private DataSourcePluginManager dataSourcePluginManager;
private DataSourceClientProvider() { // We use the cache here to avoid creating a new datasource client every time,
initDataSourcePlugin(); // One DataSourceClient corresponds to one unique datasource.
} private static final Cache<String, PooledDataSourceClient> POOLED_DATASOURCE_CLIENT_CACHE =
CacheBuilder.newBuilder()
private static class DataSourceClientProviderHolder { .expireAfterWrite(duration, TimeUnit.HOURS)
.removalListener((RemovalListener<String, PooledDataSourceClient>) notification -> {
private static final DataSourceClientProvider INSTANCE = new DataSourceClientProvider(); try (PooledDataSourceClient closedClient = notification.getValue()) {
} log.info("Datasource: {} is removed from cache due to expire", notification.getKey());
} catch (Exception e) {
log.error("Close datasource client error", e);
}
})
.maximumSize(100)
.build();
private static final DataSourcePluginManager dataSourcePluginManager = new DataSourcePluginManager();
public static DataSourceClientProvider getInstance() { static {
return DataSourceClientProviderHolder.INSTANCE; dataSourcePluginManager.installPlugin();
} }
public Connection getConnection(DbType dbType, ConnectionParam connectionParam) throws ExecutionException { public static DataSourceClient getPooledDataSourceClient(DbType dbType,
ConnectionParam connectionParam) throws ExecutionException {
BaseConnectionParam baseConnectionParam = (BaseConnectionParam) connectionParam; BaseConnectionParam baseConnectionParam = (BaseConnectionParam) connectionParam;
String datasourceUniqueId = DataSourceUtils.getDatasourceUniqueId(baseConnectionParam, dbType); String datasourceUniqueId = DataSourceUtils.getDatasourceUniqueId(baseConnectionParam, dbType);
log.info("Get connection from datasource {}", datasourceUniqueId); return POOLED_DATASOURCE_CLIENT_CACHE.get(datasourceUniqueId, () -> {
DataSourceClient dataSourceClient = uniqueId2dataSourceClientCache.get(datasourceUniqueId, () -> {
Map<String, DataSourceChannel> dataSourceChannelMap = dataSourcePluginManager.getDataSourceChannelMap(); Map<String, DataSourceChannel> dataSourceChannelMap = dataSourcePluginManager.getDataSourceChannelMap();
DataSourceChannel dataSourceChannel = dataSourceChannelMap.get(dbType.getDescp()); DataSourceChannel dataSourceChannel = dataSourceChannelMap.get(dbType.getDescp());
if (null == dataSourceChannel) { if (null == dataSourceChannel) {
throw new RuntimeException(String.format("datasource plugin '%s' is not found", dbType.getDescp())); throw new RuntimeException(String.format("datasource plugin '%s' is not found", dbType.getDescp()));
} }
return dataSourceChannel.createDataSourceClient(baseConnectionParam, dbType); return dataSourceChannel.createPooledDataSourceClient(baseConnectionParam, dbType);
}); });
return dataSourceClient.getConnection();
} }
private void initDataSourcePlugin() { public static Connection getPooledConnection(DbType dbType,
dataSourcePluginManager = new DataSourcePluginManager(); ConnectionParam connectionParam) throws SQLException, ExecutionException {
dataSourcePluginManager.installPlugin(); return getPooledDataSourceClient(dbType, connectionParam).getConnection();
}
public static AdHocDataSourceClient getAdHocDataSourceClient(DbType dbType, ConnectionParam connectionParam) {
BaseConnectionParam baseConnectionParam = (BaseConnectionParam) connectionParam;
Map<String, DataSourceChannel> dataSourceChannelMap = dataSourcePluginManager.getDataSourceChannelMap();
DataSourceChannel dataSourceChannel = dataSourceChannelMap.get(dbType.getDescp());
if (null == dataSourceChannel) {
throw new RuntimeException(String.format("datasource plugin '%s' is not found", dbType.getDescp()));
}
return dataSourceChannel.createAdHocDataSourceClient(baseConnectionParam, dbType);
}
public static Connection getAdHocConnection(DbType dbType,
ConnectionParam connectionParam) throws SQLException, ExecutionException {
return getAdHocDataSourceClient(dbType, connectionParam).getConnection();
} }
} }

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourcePluginManager.java

@ -32,10 +32,10 @@ import lombok.extern.slf4j.Slf4j;
@Slf4j @Slf4j
public class DataSourcePluginManager { public class DataSourcePluginManager {
private final Map<String, DataSourceChannel> datasourceClientMap = new ConcurrentHashMap<>(); private final Map<String, DataSourceChannel> datasourceChannelMap = new ConcurrentHashMap<>();
public Map<String, DataSourceChannel> getDataSourceChannelMap() { public Map<String, DataSourceChannel> getDataSourceChannelMap() {
return Collections.unmodifiableMap(datasourceClientMap); return Collections.unmodifiableMap(datasourceChannelMap);
} }
public void installPlugin() { public void installPlugin() {
@ -48,7 +48,7 @@ public class DataSourcePluginManager {
log.info("Registering datasource plugin: {}", name); log.info("Registering datasource plugin: {}", name);
if (datasourceClientMap.containsKey(name)) { if (datasourceChannelMap.containsKey(name)) {
throw new IllegalStateException(format("Duplicate datasource plugins named '%s'", name)); throw new IllegalStateException(format("Duplicate datasource plugins named '%s'", name));
} }
@ -60,6 +60,6 @@ public class DataSourcePluginManager {
private void loadDatasourceClient(DataSourceChannelFactory datasourceChannelFactory) { private void loadDatasourceClient(DataSourceChannelFactory datasourceChannelFactory) {
DataSourceChannel datasourceChannel = datasourceChannelFactory.create(); DataSourceChannel datasourceChannel = datasourceChannelFactory.create();
datasourceClientMap.put(datasourceChannelFactory.getName(), datasourceChannel); datasourceChannelMap.put(datasourceChannelFactory.getName(), datasourceChannel);
} }
} }

22
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/plugin/DataSourceProcessorProvider.java

@ -28,31 +28,21 @@ import lombok.extern.slf4j.Slf4j;
@Slf4j @Slf4j
public class DataSourceProcessorProvider { public class DataSourceProcessorProvider {
private DataSourceProcessorManager dataSourcePluginManager; private static final DataSourceProcessorManager dataSourcePluginManager = new DataSourceProcessorManager();
private DataSourceProcessorProvider() { static {
initDataSourceProcessorPlugin(); dataSourcePluginManager.installProcessor();
}
private static class DataSourceClientProviderHolder {
private static final DataSourceProcessorProvider INSTANCE = new DataSourceProcessorProvider();
} }
public static DataSourceProcessorProvider getInstance() { private DataSourceProcessorProvider() {
return DataSourceClientProviderHolder.INSTANCE;
} }
public DataSourceProcessor getDataSourceProcessor(@NonNull DbType dbType) { public static DataSourceProcessor getDataSourceProcessor(@NonNull DbType dbType) {
return dataSourcePluginManager.getDataSourceProcessorMap().get(dbType.name()); return dataSourcePluginManager.getDataSourceProcessorMap().get(dbType.name());
} }
public Map<String, DataSourceProcessor> getDataSourceProcessorMap() { public static Map<String, DataSourceProcessor> getDataSourceProcessorMap() {
return dataSourcePluginManager.getDataSourceProcessorMap(); return dataSourcePluginManager.getDataSourceProcessorMap();
} }
private void initDataSourceProcessorPlugin() {
dataSourcePluginManager = new DataSourceProcessorManager();
dataSourcePluginManager.installProcessor();
}
} }

26
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/provider/JDBCDataSourceProvider.java

@ -40,32 +40,6 @@ import com.zaxxer.hikari.HikariDataSource;
@Slf4j @Slf4j
public class JDBCDataSourceProvider { public class JDBCDataSourceProvider {
public static HikariDataSource createJdbcDataSource(BaseConnectionParam properties, DbType dbType) {
log.info("Creating HikariDataSource pool for maxActive:{}",
PropertyUtils.getInt(DataSourceConstants.SPRING_DATASOURCE_MAX_ACTIVE, 50));
HikariDataSource dataSource = new HikariDataSource();
// TODO Support multiple versions of data sources
ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
loaderJdbcDriver(classLoader, properties, dbType);
dataSource.setDriverClassName(properties.getDriverClassName());
dataSource.setJdbcUrl(DataSourceUtils.getJdbcUrl(dbType, properties));
dataSource.setUsername(properties.getUser());
dataSource.setPassword(PasswordUtils.decodePassword(properties.getPassword()));
dataSource.setMinimumIdle(PropertyUtils.getInt(DataSourceConstants.SPRING_DATASOURCE_MIN_IDLE, 5));
dataSource.setMaximumPoolSize(PropertyUtils.getInt(DataSourceConstants.SPRING_DATASOURCE_MAX_ACTIVE, 50));
dataSource.setConnectionTestQuery(properties.getValidationQuery());
if (MapUtils.isNotEmpty(properties.getOther())) {
properties.getOther().forEach(dataSource::addDataSourceProperty);
}
log.info("Creating HikariDataSource pool success.");
return dataSource;
}
/** /**
* @return One Session Jdbc DataSource * @return One Session Jdbc DataSource
*/ */

9
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/utils/CommonUtils.java

@ -84,13 +84,14 @@ public class CommonUtils {
/** /**
* load kerberos configuration * load kerberos configuration
* *
* @param javaSecurityKrb5Conf javaSecurityKrb5Conf * @param javaSecurityKrb5Conf javaSecurityKrb5Conf
* @param loginUserKeytabUsername loginUserKeytabUsername * @param loginUserKeytabUsername loginUserKeytabUsername
* @param loginUserKeytabPath loginUserKeytabPath * @param loginUserKeytabPath loginUserKeytabPath
* @throws IOException errors * @throws IOException errors
*/ */
public static void loadKerberosConf(String javaSecurityKrb5Conf, String loginUserKeytabUsername, public static synchronized void loadKerberosConf(String javaSecurityKrb5Conf,
String loginUserKeytabPath) throws IOException { String loginUserKeytabUsername,
String loginUserKeytabPath) throws IOException {
Configuration configuration = new Configuration(); Configuration configuration = new Configuration();
configuration.setClassLoader(configuration.getClass().getClassLoader()); configuration.setClassLoader(configuration.getClass().getClassLoader());
loadKerberosConf(javaSecurityKrb5Conf, loginUserKeytabUsername, loginUserKeytabPath, configuration); loadKerberosConf(javaSecurityKrb5Conf, loginUserKeytabUsername, loginUserKeytabPath, configuration);

2
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/main/java/org/apache/dolphinscheduler/plugin/datasource/api/utils/DataSourceUtils.java

@ -84,7 +84,7 @@ public class DataSourceUtils {
public static DataSourceProcessor getDatasourceProcessor(DbType dbType) { public static DataSourceProcessor getDatasourceProcessor(DbType dbType) {
Map<String, DataSourceProcessor> dataSourceProcessorMap = Map<String, DataSourceProcessor> dataSourceProcessorMap =
DataSourceProcessorProvider.getInstance().getDataSourceProcessorMap(); DataSourceProcessorProvider.getDataSourceProcessorMap();
if (!dataSourceProcessorMap.containsKey(dbType.name())) { if (!dataSourceProcessorMap.containsKey(dbType.name())) {
throw new IllegalArgumentException("illegal datasource type"); throw new IllegalArgumentException("illegal datasource type");
} }

42
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/test/java/org/apache/dolphinscheduler/plugin/datasource/api/client/BasePooledDataSourceClientTest.java

@ -0,0 +1,42 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.api.client;
import java.sql.Connection;
import java.sql.SQLException;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class BasePooledDataSourceClientTest {
@Mock
private BasePooledDataSourceClient basePooledDataSourceClient;
@Test
public void testGetConnection() throws SQLException {
Connection connection = Mockito.mock(Connection.class);
Mockito.when(basePooledDataSourceClient.getConnection()).thenReturn(connection);
Assertions.assertNotNull(basePooledDataSourceClient.getConnection());
}
}

88
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-api/src/test/java/org/apache/dolphinscheduler/plugin/datasource/api/client/CommonDataSourceClientTest.java

@ -1,88 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.api.client;
import org.apache.dolphinscheduler.plugin.datasource.api.datasource.MySQLConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class CommonDataSourceClientTest {
@Mock
private CommonDataSourceClient commonDataSourceClient;
@Test
public void testPreInit() {
Mockito.doNothing().when(commonDataSourceClient).preInit();
commonDataSourceClient.preInit();
Mockito.verify(commonDataSourceClient).preInit();
}
@Test
public void testCheckEnv() {
BaseConnectionParam baseConnectionParam = new MySQLConnectionParam();
Mockito.doNothing().when(commonDataSourceClient).checkEnv(Mockito.any(BaseConnectionParam.class));
commonDataSourceClient.checkEnv(baseConnectionParam);
Mockito.verify(commonDataSourceClient).checkEnv(Mockito.any(BaseConnectionParam.class));
Mockito.doNothing().when(commonDataSourceClient).checkValidationQuery(Mockito.any(BaseConnectionParam.class));
commonDataSourceClient.checkValidationQuery(baseConnectionParam);
Mockito.verify(commonDataSourceClient).checkValidationQuery(Mockito.any(BaseConnectionParam.class));
Mockito.doNothing().when(commonDataSourceClient).checkUser(Mockito.any(BaseConnectionParam.class));
commonDataSourceClient.checkUser(baseConnectionParam);
Mockito.verify(commonDataSourceClient).checkUser(Mockito.any(BaseConnectionParam.class));
Mockito.doNothing().when(commonDataSourceClient).setDefaultUsername(Mockito.any(BaseConnectionParam.class));
commonDataSourceClient.setDefaultUsername(baseConnectionParam);
Mockito.verify(commonDataSourceClient).setDefaultUsername(Mockito.any(BaseConnectionParam.class));
}
@Test
public void testInitClient() {
BaseConnectionParam baseConnectionParam = new MySQLConnectionParam();
Mockito.doNothing().when(commonDataSourceClient).initClient(Mockito.any(BaseConnectionParam.class),
Mockito.any());
commonDataSourceClient.initClient(baseConnectionParam, DbType.MYSQL);
Mockito.verify(commonDataSourceClient).initClient(Mockito.any(BaseConnectionParam.class), Mockito.any());
}
@Test
public void testCheckClient() {
Mockito.doNothing().when(this.commonDataSourceClient).checkClient();
this.commonDataSourceClient.checkClient();
Mockito.verify(this.commonDataSourceClient).checkClient();
}
@Test
public void testGetConnection() {
Connection connection = Mockito.mock(Connection.class);
Mockito.when(commonDataSourceClient.getConnection()).thenReturn(connection);
Assertions.assertNotNull(commonDataSourceClient.getConnection());
}
}

6
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaAdHocDataSourceClient.java

@ -17,13 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.athena; package org.apache.dolphinscheduler.plugin.datasource.athena;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class AthenaDataSourceClient extends CommonDataSourceClient { public class AthenaAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public AthenaDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AthenaAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

13
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaDataSourceChannel.java

@ -17,15 +17,22 @@
package org.apache.dolphinscheduler.plugin.datasource.athena; package org.apache.dolphinscheduler.plugin.datasource.athena;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class AthenaDataSourceChannel implements DataSourceChannel { public class AthenaDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new AthenaDataSourceClient(baseConnectionParam, dbType); return new AthenaAdHocDataSourceClient(baseConnectionParam, dbType);
} }
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new AthenaPooledDataSourceClient(baseConnectionParam, dbType);
}
} }

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-athena/src/main/java/org/apache/dolphinscheduler/plugin/datasource/athena/AthenaPooledDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.athena;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class AthenaPooledDataSourceClient extends BasePooledDataSourceClient {
public AthenaPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.azuresql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class AzureSQLAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public AzureSQLAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.azuresql; package org.apache.dolphinscheduler.plugin.datasource.azuresql;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class AzureSQLDataSourceChannel implements DataSourceChannel { public class AzureSQLDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new AzureSQLDataSourceClient(baseConnectionParam, dbType); return new AzureSQLAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new AzureSQLPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

41
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/AzureSQLPooledDataSourceClient.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.plugin.datasource.azuresql; package org.apache.dolphinscheduler.plugin.datasource.azuresql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLAuthMode; import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLAuthMode;
import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLConnectionParam; import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLConnectionParam;
import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLDataSourceProcessor; import org.apache.dolphinscheduler.plugin.datasource.azuresql.param.AzureSQLDataSourceProcessor;
@ -26,22 +26,18 @@ import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException; import java.sql.SQLException;
import java.sql.Statement;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
import com.google.common.base.Stopwatch;
@Slf4j @Slf4j
public class AzureSQLDataSourceClient extends CommonDataSourceClient { public class AzureSQLPooledDataSourceClient extends BasePooledDataSourceClient {
public AzureSQLDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AzureSQLPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
@Override @Override
public Connection getConnection() { public Connection getConnection() throws SQLException {
AzureSQLConnectionParam connectionParam = (AzureSQLConnectionParam) this.baseConnectionParam; AzureSQLConnectionParam connectionParam = (AzureSQLConnectionParam) this.baseConnectionParam;
if (!connectionParam.getMode().equals(AzureSQLAuthMode.ACCESSTOKEN)) { if (!connectionParam.getMode().equals(AzureSQLAuthMode.ACCESSTOKEN)) {
return super.getConnection(); return super.getConnection();
@ -49,33 +45,4 @@ public class AzureSQLDataSourceClient extends CommonDataSourceClient {
return AzureSQLDataSourceProcessor.tokenGetConnection(connectionParam); return AzureSQLDataSourceProcessor.tokenGetConnection(connectionParam);
} }
@Override
public void checkClient() {
AzureSQLConnectionParam connectionParam = (AzureSQLConnectionParam) this.baseConnectionParam;
Stopwatch stopwatch = Stopwatch.createStarted();
String validationQuery = this.baseConnectionParam.getValidationQuery();
if (!connectionParam.getMode().equals(AzureSQLAuthMode.ACCESSTOKEN)) {
// Checking data source client
try {
this.jdbcTemplate.execute(validationQuery);
} catch (Exception e) {
throw new RuntimeException("JDBC connect failed", e);
} finally {
log.info("Time to execute check jdbc client with sql {} for {} ms ",
this.baseConnectionParam.getValidationQuery(), stopwatch.elapsed(TimeUnit.MILLISECONDS));
}
} else {
try (Statement statement = getConnection().createStatement()) {
if (!statement.execute(validationQuery)) {
throw new SQLException("execute check azure sql token client failed : " + validationQuery);
}
} catch (SQLException e) {
throw new RuntimeException(e);
} finally {
log.info("Time to execute check azure sql token client with sql {} for {} ms ",
this.baseConnectionParam.getValidationQuery(), stopwatch.elapsed(TimeUnit.MILLISECONDS));
}
}
}
} }

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-azure-sql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/azuresql/SQLServerDataSourceChannelTest.java

@ -32,9 +32,11 @@ public class SQLServerDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
AzureSQLDataSourceChannel sourceChannel = Mockito.mock(AzureSQLDataSourceChannel.class); AzureSQLDataSourceChannel sourceChannel = Mockito.mock(AzureSQLDataSourceChannel.class);
AzureSQLDataSourceClient dataSourceClient = Mockito.mock(AzureSQLDataSourceClient.class); AzureSQLPooledDataSourceClient dataSourceClient = Mockito.mock(AzureSQLPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions Assertions
.assertNotNull(sourceChannel.createDataSourceClient(new AzureSQLConnectionParam(), DbType.AZURESQL)); .assertNotNull(
sourceChannel.createPooledDataSourceClient(new AzureSQLConnectionParam(), DbType.AZURESQL));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.clickhouse; package org.apache.dolphinscheduler.plugin.datasource.clickhouse;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class ClickHouseDataSourceClient extends CommonDataSourceClient { public class ClickHouseAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public ClickHouseDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public ClickHouseAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.clickhouse; package org.apache.dolphinscheduler.plugin.datasource.clickhouse;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class ClickHouseDataSourceChannel implements DataSourceChannel { public class ClickHouseDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new ClickHouseDataSourceClient(baseConnectionParam, dbType); return new ClickHouseAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new ClickHousePooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/main/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHousePooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.clickhouse;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class ClickHousePooledDataSourceClient extends BasePooledDataSourceClient {
public ClickHousePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-clickhouse/src/test/java/org/apache/dolphinscheduler/plugin/datasource/clickhouse/ClickHouseDataSourceChannelTest.java

@ -32,9 +32,10 @@ public class ClickHouseDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
ClickHouseDataSourceChannel sourceChannel = Mockito.mock(ClickHouseDataSourceChannel.class); ClickHouseDataSourceChannel sourceChannel = Mockito.mock(ClickHouseDataSourceChannel.class);
ClickHouseDataSourceClient dataSourceClient = Mockito.mock(ClickHouseDataSourceClient.class); ClickHousePooledDataSourceClient dataSourceClient = Mockito.mock(ClickHousePooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions.assertNotNull( Assertions.assertNotNull(
sourceChannel.createDataSourceClient(new ClickHouseConnectionParam(), DbType.CLICKHOUSE)); sourceChannel.createPooledDataSourceClient(new ClickHouseConnectionParam(), DbType.CLICKHOUSE));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.dameng; package org.apache.dolphinscheduler.plugin.datasource.dameng;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DamengDataSourceClient extends CommonDataSourceClient { public class DamengAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public DamengDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public DamengAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.dameng; package org.apache.dolphinscheduler.plugin.datasource.dameng;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DamengDataSourceChannel implements DataSourceChannel { public class DamengDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DamengDataSourceClient(baseConnectionParam, dbType); return new DamengAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DamengPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/main/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.dameng;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class DamengPooledDataSourceClient extends BasePooledDataSourceClient {
public DamengPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-dameng/src/test/java/org/apache/dolphinscheduler/plugin/datasource/dameng/DamengDataSourceChannelTest.java

@ -32,9 +32,11 @@ public class DamengDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
DamengDataSourceChannel sourceChannel = Mockito.mock(DamengDataSourceChannel.class); DamengDataSourceChannel sourceChannel = Mockito.mock(DamengDataSourceChannel.class);
DamengDataSourceClient dataSourceClient = Mockito.mock(DamengDataSourceClient.class); DamengPooledDataSourceClient dataSourceClient = Mockito.mock(DamengPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new DamengConnectionParam(), DbType.DAMENG)); .thenReturn(dataSourceClient);
Assertions
.assertNotNull(sourceChannel.createPooledDataSourceClient(new DamengConnectionParam(), DbType.DAMENG));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.databend; package org.apache.dolphinscheduler.plugin.datasource.databend;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DatabendDataSourceClient extends CommonDataSourceClient { public class DatabendAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public DatabendDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public DatabendAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.databend; package org.apache.dolphinscheduler.plugin.datasource.databend;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DatabendDataSourceChannel implements DataSourceChannel { public class DatabendDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DatabendDataSourceClient(baseConnectionParam, dbType); return new DatabendAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DatabendPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/main/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.databend;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class DatabendPooledDataSourceClient extends BasePooledDataSourceClient {
public DatabendPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-databend/src/test/java/org/apache/dolphinscheduler/plugin/datasource/databend/DatabendDataSourceChannelTest.java

@ -32,9 +32,10 @@ public class DatabendDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
DatabendDataSourceChannel sourceChannel = Mockito.mock(DatabendDataSourceChannel.class); DatabendDataSourceChannel sourceChannel = Mockito.mock(DatabendDataSourceChannel.class);
DatabendDataSourceClient dataSourceClient = Mockito.mock(DatabendDataSourceClient.class); DatabendPooledDataSourceClient dataSourceClient = Mockito.mock(DatabendPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions.assertNotNull( Assertions.assertNotNull(
sourceChannel.createDataSourceClient(new DatabendConnectionParam(), DbType.DATABEND)); sourceChannel.createPooledDataSourceClient(new DatabendConnectionParam(), DbType.DATABEND));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2DataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2AdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.db2; package org.apache.dolphinscheduler.plugin.datasource.db2;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DB2DataSourceClient extends CommonDataSourceClient { public class DB2AdHocDataSourceClient extends BaseAdHocDataSourceClient {
public DB2DataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public DB2AdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2DataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.db2; package org.apache.dolphinscheduler.plugin.datasource.db2;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DB2DataSourceChannel implements DataSourceChannel { public class DB2DataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DB2DataSourceClient(baseConnectionParam, dbType); return new DB2AdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DB2PooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/main/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2PooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.db2;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class DB2PooledDataSourceClient extends BasePooledDataSourceClient {
public DB2PooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-db2/src/test/java/org/apache/dolphinscheduler/plugin/datasource/db2/DB2DataSourceChannelTest.java

@ -32,8 +32,9 @@ public class DB2DataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
DB2DataSourceChannel sourceChannel = Mockito.mock(DB2DataSourceChannel.class); DB2DataSourceChannel sourceChannel = Mockito.mock(DB2DataSourceChannel.class);
DB2DataSourceClient dataSourceClient = Mockito.mock(DB2DataSourceClient.class); DB2PooledDataSourceClient dataSourceClient = Mockito.mock(DB2PooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new Db2ConnectionParam(), DbType.DB2)); .thenReturn(dataSourceClient);
Assertions.assertNotNull(sourceChannel.createPooledDataSourceClient(new Db2ConnectionParam(), DbType.DB2));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisAdHocDataSourceClient.java

@ -14,15 +14,16 @@
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.plugin.doris; package org.apache.dolphinscheduler.plugin.doris;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DorisDataSourceClient extends CommonDataSourceClient { public class DorisAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public DorisDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public DorisAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisDataSourceChannel.java

@ -16,15 +16,21 @@
*/ */
package org.apache.dolphinscheduler.plugin.doris; package org.apache.dolphinscheduler.plugin.doris;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class DorisDataSourceChannel implements DataSourceChannel { public class DorisDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return null; return new DorisAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new DorisPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

28
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/main/java/org/apache/dolphinscheduler/plugin/doris/DorisPooledDataSourceClient.java

@ -0,0 +1,28 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.doris;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class DorisPooledDataSourceClient extends BasePooledDataSourceClient {
public DorisPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/DorisDataSourceChannelTest.java

@ -32,8 +32,9 @@ public class DorisDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
DorisDataSourceChannel sourceChannel = Mockito.mock(DorisDataSourceChannel.class); DorisDataSourceChannel sourceChannel = Mockito.mock(DorisDataSourceChannel.class);
DorisDataSourceClient dataSourceClient = Mockito.mock(DorisDataSourceClient.class); DorisPooledDataSourceClient dataSourceClient = Mockito.mock(DorisPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new DorisConnectionParam(), DbType.DORIS)); .thenReturn(dataSourceClient);
Assertions.assertNotNull(sourceChannel.createPooledDataSourceClient(new DorisConnectionParam(), DbType.DORIS));
} }
} }

14
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/provider/JDBCDataSourceProviderTest.java

@ -33,20 +33,6 @@ import com.zaxxer.hikari.HikariDataSource;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
public class JDBCDataSourceProviderTest { public class JDBCDataSourceProviderTest {
@Test
public void testCreateJdbcDataSource() {
try (
MockedStatic<JDBCDataSourceProvider> mockedJDBCDataSourceProvider =
Mockito.mockStatic(JDBCDataSourceProvider.class)) {
HikariDataSource dataSource = Mockito.mock(HikariDataSource.class);
mockedJDBCDataSourceProvider
.when(() -> JDBCDataSourceProvider.createJdbcDataSource(Mockito.any(), Mockito.any()))
.thenReturn(dataSource);
Assertions.assertNotNull(
JDBCDataSourceProvider.createJdbcDataSource(new DorisConnectionParam(), DbType.DORIS));
}
}
@Test @Test
public void testCreateOneSessionJdbcDataSource() { public void testCreateOneSessionJdbcDataSource() {
try ( try (

11
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-doris/src/test/java/org/apache/dolphinscheduler/plugin/doris/utils/DataSourceUtilsTest.java

@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.spi.datasource.ConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -92,22 +93,20 @@ public class DataSourceUtilsTest {
} }
@Test @Test
public void testGetConnection() throws ExecutionException { public void testGetConnection() throws ExecutionException, SQLException {
try ( try (
MockedStatic<PropertyUtils> mockedStaticPropertyUtils = Mockito.mockStatic(PropertyUtils.class); MockedStatic<PropertyUtils> mockedStaticPropertyUtils = Mockito.mockStatic(PropertyUtils.class);
MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider = MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider =
Mockito.mockStatic(DataSourceClientProvider.class)) { Mockito.mockStatic(DataSourceClientProvider.class)) {
mockedStaticPropertyUtils.when(() -> PropertyUtils.getLong("kerberos.expire.time", 24L)).thenReturn(24L);
DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class);
mockedStaticDataSourceClientProvider.when(DataSourceClientProvider::getInstance).thenReturn(clientProvider);
Connection connection = Mockito.mock(Connection.class); Connection connection = Mockito.mock(Connection.class);
Mockito.when(clientProvider.getConnection(Mockito.any(), Mockito.any())).thenReturn(connection); Mockito.when(DataSourceClientProvider.getAdHocConnection(Mockito.any(), Mockito.any()))
.thenReturn(connection);
DorisConnectionParam connectionParam = new DorisConnectionParam(); DorisConnectionParam connectionParam = new DorisConnectionParam();
connectionParam.setUser("root"); connectionParam.setUser("root");
connectionParam.setPassword("123456"); connectionParam.setPassword("123456");
connection = DataSourceClientProvider.getInstance().getConnection(DbType.DORIS, connectionParam); connection = DataSourceClientProvider.getAdHocConnection(DbType.DORIS, connectionParam);
Assertions.assertNotNull(connection); Assertions.assertNotNull(connection);
} }

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.hive;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class HiveAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public HiveAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.hive; package org.apache.dolphinscheduler.plugin.datasource.hive;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class HiveDataSourceChannel implements DataSourceChannel { public class HiveDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new HiveDataSourceClient(baseConnectionParam, dbType); return new HiveAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new HivePooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

45
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/main/java/org/apache/dolphinscheduler/plugin/datasource/hive/HivePooledDataSourceClient.java

@ -22,8 +22,7 @@ import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.JAVA_SEC
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.JAVA_SECURITY_KRB5_CONF_PATH; import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.JAVA_SECURITY_KRB5_CONF_PATH;
import org.apache.dolphinscheduler.common.utils.PropertyUtils; import org.apache.dolphinscheduler.common.utils.PropertyUtils;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.plugin.datasource.api.provider.JDBCDataSourceProvider;
import org.apache.dolphinscheduler.plugin.datasource.hive.security.UserGroupInformationFactory; import org.apache.dolphinscheduler.plugin.datasource.hive.security.UserGroupInformationFactory;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
@ -38,37 +37,22 @@ import java.sql.SQLException;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
import org.springframework.jdbc.core.JdbcTemplate; import com.zaxxer.hikari.HikariDataSource;
@Slf4j @Slf4j
public class HiveDataSourceClient extends CommonDataSourceClient { public class HivePooledDataSourceClient extends BasePooledDataSourceClient {
public HiveDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public HivePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
@Override public HikariDataSource createDataSourcePool(BaseConnectionParam baseConnectionParam, DbType dbType) {
protected void preInit() {
log.info("PreInit in {}", getClass().getName());
}
@Override
protected void initClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
log.info("Create UserGroupInformation.");
UserGroupInformationFactory.login(baseConnectionParam.getUser());
log.info("Create ugi success.");
this.dataSource = JDBCDataSourceProvider.createOneSessionJdbcDataSource(baseConnectionParam, dbType);
this.jdbcTemplate = new JdbcTemplate(dataSource);
log.info("Init {} success.", getClass().getName());
}
@Override
protected void checkEnv(BaseConnectionParam baseConnectionParam) {
super.checkEnv(baseConnectionParam);
checkKerberosEnv(); checkKerberosEnv();
UserGroupInformationFactory.login(baseConnectionParam.getUser());
return super.createDataSourcePool(baseConnectionParam, dbType);
} }
// used in constructor
private void checkKerberosEnv() { private void checkKerberosEnv() {
String krb5File = PropertyUtils.getString(JAVA_SECURITY_KRB5_CONF_PATH); String krb5File = PropertyUtils.getString(JAVA_SECURITY_KRB5_CONF_PATH);
Boolean kerberosStartupState = PropertyUtils.getBoolean(HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false); Boolean kerberosStartupState = PropertyUtils.getBoolean(HADOOP_SECURITY_AUTHENTICATION_STARTUP_STATE, false);
@ -87,17 +71,16 @@ public class HiveDataSourceClient extends CommonDataSourceClient {
} }
@Override @Override
public Connection getConnection() { public Connection getConnection() throws SQLException {
Connection connection = null; try {
while (connection == null) { return dataSource.getConnection();
try { } catch (SQLException e) {
connection = dataSource.getConnection(); synchronized (HikariDataSource.class) {
} catch (SQLException e) {
UserGroupInformationFactory.logout(baseConnectionParam.getUser()); UserGroupInformationFactory.logout(baseConnectionParam.getUser());
UserGroupInformationFactory.login(baseConnectionParam.getUser()); UserGroupInformationFactory.login(baseConnectionParam.getUser());
return dataSource.getConnection();
} }
} }
return connection;
} }
@Override @Override

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-hive/src/test/java/org/apache/dolphinscheduler/plugin/datasource/hive/HiveDataSourceChannelTest.java

@ -32,8 +32,9 @@ public class HiveDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
HiveDataSourceChannel sourceChannel = Mockito.mock(HiveDataSourceChannel.class); HiveDataSourceChannel sourceChannel = Mockito.mock(HiveDataSourceChannel.class);
HiveDataSourceClient dataSourceClient = Mockito.mock(HiveDataSourceClient.class); HivePooledDataSourceClient dataSourceClient = Mockito.mock(HivePooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new HiveConnectionParam(), DbType.HIVE)); .thenReturn(dataSourceClient);
Assertions.assertNotNull(sourceChannel.createPooledDataSourceClient(new HiveConnectionParam(), DbType.HIVE));
} }
} }

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class KyuubiAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public KyuubiAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.kyuubi; package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class KyuubiDataSourceChannel implements DataSourceChannel { public class KyuubiDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new KyuubiDataSourceClient(baseConnectionParam, dbType); return new KyuubiAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new KyuubiPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

75
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceClient.java

@ -1,75 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient;
import org.apache.dolphinscheduler.plugin.datasource.api.provider.JDBCDataSourceProvider;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection;
import java.sql.SQLException;
import lombok.extern.slf4j.Slf4j;
import org.springframework.jdbc.core.JdbcTemplate;
@Slf4j
public class KyuubiDataSourceClient extends CommonDataSourceClient {
public KyuubiDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
@Override
protected void preInit() {
log.info("PreInit in {}", getClass().getName());
}
@Override
protected void initClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
this.dataSource = JDBCDataSourceProvider.createOneSessionJdbcDataSource(baseConnectionParam, dbType);
this.jdbcTemplate = new JdbcTemplate(dataSource);
log.info("Init {} success.", getClass().getName());
}
@Override
protected void checkEnv(BaseConnectionParam baseConnectionParam) {
super.checkEnv(baseConnectionParam);
}
@Override
public Connection getConnection() {
Connection connection = null;
while (connection == null) {
try {
connection = dataSource.getConnection();
} catch (SQLException e) {
log.error("Failed to get Kyuubi Connection.", e);
}
}
return connection;
}
@Override
public void close() {
super.close();
log.info("Closed Kyuubi datasource client.");
}
}

24
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiPooledDataSourceClient.java

@ -15,22 +15,32 @@
* limitations under the License. * limitations under the License.
*/ */
package org.apache.dolphinscheduler.plugin.datasource.oracle; package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import org.apache.dolphinscheduler.common.constants.DataSourceConstants; import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class OracleDataSourceClient extends CommonDataSourceClient { import java.sql.Connection;
import java.sql.SQLException;
public OracleDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { import lombok.extern.slf4j.Slf4j;
@Slf4j
public class KyuubiPooledDataSourceClient extends BasePooledDataSourceClient {
public KyuubiPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
@Override @Override
protected void setDefaultValidationQuery(BaseConnectionParam baseConnectionParam) { public Connection getConnection() throws SQLException {
baseConnectionParam.setValidationQuery(DataSourceConstants.ORACLE_VALIDATION_QUERY); return dataSource.getConnection();
} }
@Override
public void close() {
super.close();
log.info("Closed Kyuubi datasource client.");
}
} }

1
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/main/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/param/KyuubiDataSourceProcessor.java

@ -121,6 +121,7 @@ public class KyuubiDataSourceProcessor extends AbstractDataSourceProcessor {
public Connection getConnection(ConnectionParam connectionParam) throws ClassNotFoundException, SQLException { public Connection getConnection(ConnectionParam connectionParam) throws ClassNotFoundException, SQLException {
KyuubiConnectionParam kyuubiConnectionParam = (KyuubiConnectionParam) connectionParam; KyuubiConnectionParam kyuubiConnectionParam = (KyuubiConnectionParam) connectionParam;
Class.forName(getDatasourceDriver()); Class.forName(getDatasourceDriver());
// todo:
return DriverManager.getConnection(getJdbcUrl(connectionParam), return DriverManager.getConnection(getJdbcUrl(connectionParam),
kyuubiConnectionParam.getUser(), PasswordUtils.decodePassword(kyuubiConnectionParam.getPassword())); kyuubiConnectionParam.getUser(), PasswordUtils.decodePassword(kyuubiConnectionParam.getPassword()));
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceChannelTest.java

@ -32,9 +32,10 @@ public class KyuubiDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
KyuubiDataSourceChannel sourceChannel = Mockito.mock(KyuubiDataSourceChannel.class); KyuubiDataSourceChannel sourceChannel = Mockito.mock(KyuubiDataSourceChannel.class);
KyuubiDataSourceClient dataSourceClient = Mockito.mock(KyuubiDataSourceClient.class); KyuubiPooledDataSourceClient dataSourceClient = Mockito.mock(KyuubiPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions Assertions
.assertNotNull(sourceChannel.createDataSourceClient(new KyuubiConnectionParam(), DbType.KYUUBI)); .assertNotNull(sourceChannel.createPooledDataSourceClient(new KyuubiConnectionParam(), DbType.KYUUBI));
} }
} }

73
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiDataSourceClientTest.java

@ -1,73 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import org.apache.dolphinscheduler.plugin.datasource.kyuubi.param.KyuubiConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class KyuubiDataSourceClientTest {
@Mock
private KyuubiDataSourceClient kyuubiDataSourceClient;
@Test
public void testPreInit() {
kyuubiDataSourceClient.preInit();
Mockito.verify(kyuubiDataSourceClient).preInit();
}
@Test
public void testCheckEnv() {
KyuubiConnectionParam kyuubiConnectionParam = new KyuubiConnectionParam();
kyuubiDataSourceClient.checkEnv(kyuubiConnectionParam);
Mockito.verify(kyuubiDataSourceClient).checkEnv(kyuubiConnectionParam);
}
@Test
public void testInitClient() {
KyuubiConnectionParam kyuubiConnectionParam = new KyuubiConnectionParam();
kyuubiDataSourceClient.initClient(kyuubiConnectionParam, DbType.KYUUBI);
Mockito.verify(kyuubiDataSourceClient).initClient(kyuubiConnectionParam, DbType.KYUUBI);
}
@Test
public void testCheckClient() {
kyuubiDataSourceClient.checkClient();
Mockito.verify(kyuubiDataSourceClient).checkClient();
}
@Test
public void testGetConnection() {
Connection connection = Mockito.mock(Connection.class);
Mockito.when(kyuubiDataSourceClient.getConnection()).thenReturn(connection);
Assertions.assertNotNull(kyuubiDataSourceClient.getConnection());
}
}

44
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/KyuubiPooledDataSourceClientTest.java

@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.kyuubi;
import java.sql.Connection;
import java.sql.SQLException;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class)
public class KyuubiPooledDataSourceClientTest {
@Mock
private KyuubiPooledDataSourceClient kyuubiPooledDataSourceClient;
@Test
public void testGetConnection() throws SQLException {
Connection connection = Mockito.mock(Connection.class);
Mockito.when(kyuubiPooledDataSourceClient.getConnection()).thenReturn(connection);
Assertions.assertNotNull(kyuubiPooledDataSourceClient.getConnection());
}
}

14
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-kyuubi/src/test/java/org/apache/dolphinscheduler/plugin/datasource/kyuubi/provider/KyuubiJDBCDataSourceProviderTest.java

@ -30,20 +30,6 @@ import com.zaxxer.hikari.HikariDataSource;
public class KyuubiJDBCDataSourceProviderTest { public class KyuubiJDBCDataSourceProviderTest {
@Test
public void testCreateJdbcDataSource() {
try (
MockedStatic<JDBCDataSourceProvider> mockedJDBCDataSourceProvider =
Mockito.mockStatic(JDBCDataSourceProvider.class)) {
HikariDataSource dataSource = Mockito.mock(HikariDataSource.class);
mockedJDBCDataSourceProvider
.when(() -> JDBCDataSourceProvider.createJdbcDataSource(Mockito.any(), Mockito.any()))
.thenReturn(dataSource);
Assertions.assertNotNull(
JDBCDataSourceProvider.createJdbcDataSource(new KyuubiConnectionParam(), DbType.KYUUBI));
}
}
@Test @Test
public void testCreateOneSessionJdbcDataSource() { public void testCreateOneSessionJdbcDataSource() {
try ( try (

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.mysql; package org.apache.dolphinscheduler.plugin.datasource.mysql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class MySQLDataSourceClient extends CommonDataSourceClient { public class MySQLAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public MySQLDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public MySQLAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.mysql; package org.apache.dolphinscheduler.plugin.datasource.mysql;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class MySQLDataSourceChannel implements DataSourceChannel { public class MySQLDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new MySQLDataSourceClient(baseConnectionParam, dbType); return new MySQLAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new MySQLPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.mysql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class MySQLPooledDataSourceClient extends BasePooledDataSourceClient {
public MySQLPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/MySQLDataSourceChannelTest.java

@ -32,8 +32,9 @@ public class MySQLDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
MySQLDataSourceChannel sourceChannel = Mockito.mock(MySQLDataSourceChannel.class); MySQLDataSourceChannel sourceChannel = Mockito.mock(MySQLDataSourceChannel.class);
MySQLDataSourceClient dataSourceClient = Mockito.mock(MySQLDataSourceClient.class); MySQLPooledDataSourceClient dataSourceClient = Mockito.mock(MySQLPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new MySQLConnectionParam(), DbType.MYSQL)); .thenReturn(dataSourceClient);
Assertions.assertNotNull(sourceChannel.createPooledDataSourceClient(new MySQLConnectionParam(), DbType.MYSQL));
} }
} }

14
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/provider/JDBCDataSourceProviderTest.java

@ -33,20 +33,6 @@ import com.zaxxer.hikari.HikariDataSource;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
public class JDBCDataSourceProviderTest { public class JDBCDataSourceProviderTest {
@Test
public void testCreateJdbcDataSource() {
try (
MockedStatic<JDBCDataSourceProvider> mockedJDBCDataSourceProvider =
Mockito.mockStatic(JDBCDataSourceProvider.class)) {
HikariDataSource dataSource = Mockito.mock(HikariDataSource.class);
mockedJDBCDataSourceProvider
.when(() -> JDBCDataSourceProvider.createJdbcDataSource(Mockito.any(), Mockito.any()))
.thenReturn(dataSource);
Assertions.assertNotNull(
JDBCDataSourceProvider.createJdbcDataSource(new MySQLConnectionParam(), DbType.MYSQL));
}
}
@Test @Test
public void testCreateOneSessionJdbcDataSource() { public void testCreateOneSessionJdbcDataSource() {
try ( try (

9
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-mysql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/mysql/utils/DataSourceUtilsTest.java

@ -29,6 +29,7 @@ import org.apache.dolphinscheduler.spi.datasource.ConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -91,22 +92,20 @@ public class DataSourceUtilsTest {
} }
@Test @Test
public void testGetConnection() throws ExecutionException { public void testGetConnection() throws ExecutionException, SQLException {
try ( try (
MockedStatic<PropertyUtils> mockedStaticPropertyUtils = Mockito.mockStatic(PropertyUtils.class); MockedStatic<PropertyUtils> mockedStaticPropertyUtils = Mockito.mockStatic(PropertyUtils.class);
MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider = MockedStatic<DataSourceClientProvider> mockedStaticDataSourceClientProvider =
Mockito.mockStatic(DataSourceClientProvider.class)) { Mockito.mockStatic(DataSourceClientProvider.class)) {
mockedStaticPropertyUtils.when(() -> PropertyUtils.getLong("kerberos.expire.time", 24L)).thenReturn(24L); mockedStaticPropertyUtils.when(() -> PropertyUtils.getLong("kerberos.expire.time", 24L)).thenReturn(24L);
DataSourceClientProvider clientProvider = Mockito.mock(DataSourceClientProvider.class);
mockedStaticDataSourceClientProvider.when(DataSourceClientProvider::getInstance).thenReturn(clientProvider);
Connection connection = Mockito.mock(Connection.class); Connection connection = Mockito.mock(Connection.class);
Mockito.when(clientProvider.getConnection(Mockito.any(), Mockito.any())).thenReturn(connection); Mockito.when(DataSourceClientProvider.getAdHocConnection(Mockito.any(), Mockito.any()))
.thenReturn(connection);
MySQLConnectionParam connectionParam = new MySQLConnectionParam(); MySQLConnectionParam connectionParam = new MySQLConnectionParam();
connectionParam.setUser("root"); connectionParam.setUser("root");
connectionParam.setPassword("123456"); connectionParam.setPassword("123456");
connection = DataSourceClientProvider.getInstance().getConnection(DbType.MYSQL, connectionParam);
Assertions.assertNotNull(connection); Assertions.assertNotNull(connection);
} }

6
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBaseDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBaseAdHocDataSourceClient.java

@ -17,13 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.oceanbase; package org.apache.dolphinscheduler.plugin.datasource.oceanbase;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class OceanBaseDataSourceClient extends CommonDataSourceClient { public class OceanBaseAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public OceanBaseDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public OceanBaseAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBaseDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.oceanbase; package org.apache.dolphinscheduler.plugin.datasource.oceanbase;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class OceanBaseDataSourceChannel implements DataSourceChannel { public class OceanBaseDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new OceanBaseDataSourceClient(baseConnectionParam, dbType); return new OceanBaseAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new OceanBasePooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oceanbase/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oceanbase/OceanBasePooledDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.oceanbase;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class OceanBasePooledDataSourceClient extends BasePooledDataSourceClient {
public OceanBasePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.oracle;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class OracleAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public OracleAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.oracle; package org.apache.dolphinscheduler.plugin.datasource.oracle;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class OracleDataSourceChannel implements DataSourceChannel { public class OracleDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new OracleDataSourceClient(baseConnectionParam, dbType); return new OracleAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new OraclePooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/main/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OraclePooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.oracle;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class OraclePooledDataSourceClient extends BasePooledDataSourceClient {
public OraclePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-oracle/src/test/java/org/apache/dolphinscheduler/plugin/datasource/oracle/OracleDataSourceChannelTest.java

@ -32,8 +32,10 @@ public class OracleDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
OracleDataSourceChannel sourceChannel = Mockito.mock(OracleDataSourceChannel.class); OracleDataSourceChannel sourceChannel = Mockito.mock(OracleDataSourceChannel.class);
OracleDataSourceClient dataSourceClient = Mockito.mock(OracleDataSourceClient.class); OraclePooledDataSourceClient dataSourceClient = Mockito.mock(OraclePooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new OracleConnectionParam(), DbType.ORACLE)); .thenReturn(dataSourceClient);
Assertions
.assertNotNull(sourceChannel.createPooledDataSourceClient(new OracleConnectionParam(), DbType.ORACLE));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreSQLDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreAdHocSQLDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.postgresql; package org.apache.dolphinscheduler.plugin.datasource.postgresql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class PostgreSQLDataSourceClient extends CommonDataSourceClient { public class PostgreAdHocSQLDataSourceClient extends BaseAdHocDataSourceClient {
public PostgreSQLDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public PostgreAdHocSQLDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgrePooledSQLDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.postgresql;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class PostgrePooledSQLDataSourceClient extends BasePooledDataSourceClient {
public PostgrePooledSQLDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/main/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreSQLDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.postgresql; package org.apache.dolphinscheduler.plugin.datasource.postgresql;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class PostgreSQLDataSourceChannel implements DataSourceChannel { public class PostgreSQLDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new PostgreSQLDataSourceClient(baseConnectionParam, dbType); return new PostgreAdHocSQLDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new PostgrePooledSQLDataSourceClient(baseConnectionParam, dbType);
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-postgresql/src/test/java/org/apache/dolphinscheduler/plugin/datasource/postgresql/PostgreSQLDataSourceChannelTest.java

@ -32,9 +32,10 @@ public class PostgreSQLDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
PostgreSQLDataSourceChannel sourceChannel = Mockito.mock(PostgreSQLDataSourceChannel.class); PostgreSQLDataSourceChannel sourceChannel = Mockito.mock(PostgreSQLDataSourceChannel.class);
PostgreSQLDataSourceClient dataSourceClient = Mockito.mock(PostgreSQLDataSourceClient.class); PostgrePooledSQLDataSourceClient dataSourceClient = Mockito.mock(PostgrePooledSQLDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions.assertNotNull( Assertions.assertNotNull(
sourceChannel.createDataSourceClient(new PostgreSQLConnectionParam(), DbType.POSTGRESQL)); sourceChannel.createPooledDataSourceClient(new PostgreSQLConnectionParam(), DbType.POSTGRESQL));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.presto; package org.apache.dolphinscheduler.plugin.datasource.presto;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class PrestoDataSourceClient extends CommonDataSourceClient { public class PrestoAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public PrestoDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public PrestoAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.presto; package org.apache.dolphinscheduler.plugin.datasource.presto;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class PrestoDataSourceChannel implements DataSourceChannel { public class PrestoDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new PrestoDataSourceClient(baseConnectionParam, dbType); return new PrestoAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new PrestoPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/main/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.presto;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class PrestoPooledDataSourceClient extends BasePooledDataSourceClient {
public PrestoPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-presto/src/test/java/org/apache/dolphinscheduler/plugin/datasource/presto/PrestoDataSourceChannelTest.java

@ -32,8 +32,10 @@ public class PrestoDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
PrestoDataSourceChannel sourceChannel = Mockito.mock(PrestoDataSourceChannel.class); PrestoDataSourceChannel sourceChannel = Mockito.mock(PrestoDataSourceChannel.class);
PrestoDataSourceClient dataSourceClient = Mockito.mock(PrestoDataSourceClient.class); PrestoPooledDataSourceClient dataSourceClient = Mockito.mock(PrestoPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new PrestoConnectionParam(), DbType.PRESTO)); .thenReturn(dataSourceClient);
Assertions
.assertNotNull(sourceChannel.createPooledDataSourceClient(new PrestoConnectionParam(), DbType.PRESTO));
} }
} }

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.redshift;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class RedshiftAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public RedshiftAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.redshift; package org.apache.dolphinscheduler.plugin.datasource.redshift;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class RedshiftDataSourceChannel implements DataSourceChannel { public class RedshiftDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new RedshiftDataSourceClient(baseConnectionParam, dbType); return new RedshiftAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new RedshiftPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

40
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-redshift/src/main/java/org/apache/dolphinscheduler/plugin/datasource/redshift/RedshiftPooledDataSourceClient.java

@ -17,7 +17,7 @@
package org.apache.dolphinscheduler.plugin.datasource.redshift; package org.apache.dolphinscheduler.plugin.datasource.redshift;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftAuthMode; import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftAuthMode;
import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftConnectionParam; import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftConnectionParam;
import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftDataSourceProcessor; import org.apache.dolphinscheduler.plugin.datasource.redshift.param.RedshiftDataSourceProcessor;
@ -26,22 +26,18 @@ import org.apache.dolphinscheduler.spi.enums.DbType;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException; import java.sql.SQLException;
import java.sql.Statement;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j; import lombok.extern.slf4j.Slf4j;
import com.google.common.base.Stopwatch;
@Slf4j @Slf4j
public class RedshiftDataSourceClient extends CommonDataSourceClient { public class RedshiftPooledDataSourceClient extends BasePooledDataSourceClient {
public RedshiftDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public RedshiftPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
@Override @Override
public Connection getConnection() { public Connection getConnection() throws SQLException {
RedshiftConnectionParam connectionParam = (RedshiftConnectionParam) this.baseConnectionParam; RedshiftConnectionParam connectionParam = (RedshiftConnectionParam) this.baseConnectionParam;
if (connectionParam.getMode().equals(RedshiftAuthMode.PASSWORD)) { if (connectionParam.getMode().equals(RedshiftAuthMode.PASSWORD)) {
return super.getConnection(); return super.getConnection();
@ -49,32 +45,4 @@ public class RedshiftDataSourceClient extends CommonDataSourceClient {
return RedshiftDataSourceProcessor.getConnectionByIAM(connectionParam); return RedshiftDataSourceProcessor.getConnectionByIAM(connectionParam);
} }
@Override
public void checkClient() {
RedshiftConnectionParam connectionParam = (RedshiftConnectionParam) this.baseConnectionParam;
Stopwatch stopwatch = Stopwatch.createStarted();
String validationQuery = this.baseConnectionParam.getValidationQuery();
if (connectionParam.getMode().equals(RedshiftAuthMode.PASSWORD)) {
// Checking data source client
try {
this.jdbcTemplate.execute(validationQuery);
} catch (Exception e) {
throw new RuntimeException("JDBC connect failed", e);
} finally {
log.info("Time to execute check jdbc client with sql {} for {} ms ",
this.baseConnectionParam.getValidationQuery(), stopwatch.elapsed(TimeUnit.MILLISECONDS));
}
} else {
try (Statement statement = getConnection().createStatement()) {
if (!statement.execute(validationQuery)) {
throw new SQLException("execute check redshift access key failed : " + validationQuery);
}
} catch (SQLException e) {
throw new RuntimeException(e);
} finally {
log.info("Time to execute check redshift access key with sql {} for {} ms ",
this.baseConnectionParam.getValidationQuery(), stopwatch.elapsed(TimeUnit.MILLISECONDS));
}
}
}
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.snowflake; package org.apache.dolphinscheduler.plugin.datasource.snowflake;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SnowflakeDataSourceClient extends CommonDataSourceClient { public class SnowflakeAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public SnowflakeDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public SnowflakeAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.snowflake; package org.apache.dolphinscheduler.plugin.datasource.snowflake;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SnowflakeDataSourceChannel implements DataSourceChannel { public class SnowflakeDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SnowflakeDataSourceClient(baseConnectionParam, dbType); return new SnowflakeAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SnowflakePooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/main/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakePooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.snowflake;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class SnowflakePooledDataSourceClient extends BasePooledDataSourceClient {
public SnowflakePooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceChannelTest.java

@ -32,9 +32,11 @@ public class SnowflakeDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
SnowflakeDataSourceChannel sourceChannel = Mockito.mock(SnowflakeDataSourceChannel.class); SnowflakeDataSourceChannel sourceChannel = Mockito.mock(SnowflakeDataSourceChannel.class);
SnowflakeDataSourceClient dataSourceClient = Mockito.mock(SnowflakeDataSourceClient.class); SnowflakePooledDataSourceClient dataSourceClient = Mockito.mock(SnowflakePooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions Assertions
.assertNotNull(sourceChannel.createDataSourceClient(new SnowflakeConnectionParam(), DbType.SNOWFLAKE)); .assertNotNull(
sourceChannel.createPooledDataSourceClient(new SnowflakeConnectionParam(), DbType.SNOWFLAKE));
} }
} }

17
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakeDataSourceClientTest.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/SnowflakePooledDataSourceClientTest.java

@ -18,6 +18,7 @@
package org.apache.dolphinscheduler.plugin.datasource.snowflake; package org.apache.dolphinscheduler.plugin.datasource.snowflake;
import java.sql.Connection; import java.sql.Connection;
import java.sql.SQLException;
import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
@ -27,22 +28,16 @@ import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoExtension;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
public class SnowflakeDataSourceClientTest { public class SnowflakePooledDataSourceClientTest {
@Mock @Mock
private SnowflakeDataSourceClient snowflakeDataSourceClient; private SnowflakePooledDataSourceClient snowflakePooledDataSourceClient;
@Test @Test
public void testCheckClient() { public void testGetConnection() throws SQLException {
snowflakeDataSourceClient.checkClient();
Mockito.verify(snowflakeDataSourceClient).checkClient();
}
@Test
public void testGetConnection() {
Connection connection = Mockito.mock(Connection.class); Connection connection = Mockito.mock(Connection.class);
Mockito.when(snowflakeDataSourceClient.getConnection()).thenReturn(connection); Mockito.when(snowflakePooledDataSourceClient.getConnection()).thenReturn(connection);
Assertions.assertNotNull(snowflakeDataSourceClient.getConnection()); Assertions.assertNotNull(snowflakePooledDataSourceClient.getConnection());
} }

14
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-snowflake/src/test/java/org/apache/dolphinscheduler/plugin/datasource/snowflake/provider/SnowflakeJDBCDataSourceProviderTest.java

@ -33,20 +33,6 @@ import com.zaxxer.hikari.HikariDataSource;
@ExtendWith(MockitoExtension.class) @ExtendWith(MockitoExtension.class)
public class SnowflakeJDBCDataSourceProviderTest { public class SnowflakeJDBCDataSourceProviderTest {
@Test
public void testCreateJdbcDataSource() {
try (
MockedStatic<JDBCDataSourceProvider> mockedJDBCDataSourceProvider =
Mockito.mockStatic(JDBCDataSourceProvider.class)) {
HikariDataSource dataSource = Mockito.mock(HikariDataSource.class);
mockedJDBCDataSourceProvider
.when(() -> JDBCDataSourceProvider.createJdbcDataSource(Mockito.any(), Mockito.any()))
.thenReturn(dataSource);
Assertions.assertNotNull(
JDBCDataSourceProvider.createJdbcDataSource(new SnowflakeConnectionParam(), DbType.SNOWFLAKE));
}
}
@Test @Test
public void testCreateOneSessionJdbcDataSource() { public void testCreateOneSessionJdbcDataSource() {
try ( try (

29
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkAdHocDataSourceClient.java

@ -0,0 +1,29 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.spark;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class SparkAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public SparkAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.spark; package org.apache.dolphinscheduler.plugin.datasource.spark;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SparkDataSourceChannel implements DataSourceChannel { public class SparkDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SparkDataSourceClient(baseConnectionParam, dbType); return new SparkAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SparkPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

6
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/main/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkPooledDataSourceClient.java

@ -17,13 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.spark; package org.apache.dolphinscheduler.plugin.datasource.spark;
import org.apache.dolphinscheduler.plugin.datasource.hive.HiveDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.hive.HivePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SparkDataSourceClient extends HiveDataSourceClient { public class SparkPooledDataSourceClient extends HivePooledDataSourceClient {
public SparkDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public SparkPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-spark/src/test/java/org/apache/dolphinscheduler/plugin/datasource/spark/SparkDataSourceChannelTest.java

@ -32,8 +32,9 @@ public class SparkDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
SparkDataSourceChannel sourceChannel = Mockito.mock(SparkDataSourceChannel.class); SparkDataSourceChannel sourceChannel = Mockito.mock(SparkDataSourceChannel.class);
SparkDataSourceClient dataSourceClient = Mockito.mock(SparkDataSourceClient.class); SparkPooledDataSourceClient dataSourceClient = Mockito.mock(SparkPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
Assertions.assertNotNull(sourceChannel.createDataSourceClient(new SparkConnectionParam(), DbType.SPARK)); .thenReturn(dataSourceClient);
Assertions.assertNotNull(sourceChannel.createPooledDataSourceClient(new SparkConnectionParam(), DbType.SPARK));
} }
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.sqlserver; package org.apache.dolphinscheduler.plugin.datasource.sqlserver;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SQLServerDataSourceClient extends CommonDataSourceClient { public class SQLServerAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public SQLServerDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public SQLServerAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.sqlserver; package org.apache.dolphinscheduler.plugin.datasource.sqlserver;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SQLServerDataSourceChannel implements DataSourceChannel { public class SQLServerDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SQLServerDataSourceClient(baseConnectionParam, dbType); return new SQLServerAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SQLServerPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/main/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.sqlserver;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class SQLServerPooledDataSourceClient extends BasePooledDataSourceClient {
public SQLServerPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

8
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-sqlserver/src/test/java/org/apache/dolphinscheduler/plugin/datasource/sqlserver/SQLServerDataSourceChannelTest.java

@ -32,9 +32,11 @@ public class SQLServerDataSourceChannelTest {
@Test @Test
public void testCreateDataSourceClient() { public void testCreateDataSourceClient() {
SQLServerDataSourceChannel sourceChannel = Mockito.mock(SQLServerDataSourceChannel.class); SQLServerDataSourceChannel sourceChannel = Mockito.mock(SQLServerDataSourceChannel.class);
SQLServerDataSourceClient dataSourceClient = Mockito.mock(SQLServerDataSourceClient.class); SQLServerPooledDataSourceClient dataSourceClient = Mockito.mock(SQLServerPooledDataSourceClient.class);
Mockito.when(sourceChannel.createDataSourceClient(Mockito.any(), Mockito.any())).thenReturn(dataSourceClient); Mockito.when(sourceChannel.createPooledDataSourceClient(Mockito.any(), Mockito.any()))
.thenReturn(dataSourceClient);
Assertions Assertions
.assertNotNull(sourceChannel.createDataSourceClient(new SQLServerConnectionParam(), DbType.SQLSERVER)); .assertNotNull(
sourceChannel.createPooledDataSourceClient(new SQLServerConnectionParam(), DbType.SQLSERVER));
} }
} }

11
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-ssh/src/main/java/org/apache/dolphinscheduler/plugin/datasource/ssh/SSHDataSourceChannel.java

@ -17,16 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.ssh; package org.apache.dolphinscheduler.plugin.datasource.ssh;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class SSHDataSourceChannel implements DataSourceChannel { public class SSHDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new SSHDataSourceClient(baseConnectionParam, dbType); throw new UnsupportedOperationException("SSH AdHocDataSourceClient is not supported");
} }
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
throw new UnsupportedOperationException("SSH PooledDataSourceClient is not supported");
}
} }

7
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksDataSourceClient.java → dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksAdHocDataSourceClient.java

@ -17,14 +17,13 @@
package org.apache.dolphinscheduler.plugin.datasource.starrocks; package org.apache.dolphinscheduler.plugin.datasource.starrocks;
import org.apache.dolphinscheduler.plugin.datasource.api.client.CommonDataSourceClient; import org.apache.dolphinscheduler.plugin.datasource.api.client.BaseAdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class StarRocksDataSourceClient extends CommonDataSourceClient { public class StarRocksAdHocDataSourceClient extends BaseAdHocDataSourceClient {
public StarRocksDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public StarRocksAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType); super(baseConnectionParam, dbType);
} }
} }

12
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksDataSourceChannel.java

@ -17,15 +17,21 @@
package org.apache.dolphinscheduler.plugin.datasource.starrocks; package org.apache.dolphinscheduler.plugin.datasource.starrocks;
import org.apache.dolphinscheduler.spi.datasource.AdHocDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam; import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel; import org.apache.dolphinscheduler.spi.datasource.DataSourceChannel;
import org.apache.dolphinscheduler.spi.datasource.DataSourceClient; import org.apache.dolphinscheduler.spi.datasource.PooledDataSourceClient;
import org.apache.dolphinscheduler.spi.enums.DbType; import org.apache.dolphinscheduler.spi.enums.DbType;
public class StarRocksDataSourceChannel implements DataSourceChannel { public class StarRocksDataSourceChannel implements DataSourceChannel {
@Override @Override
public DataSourceClient createDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) { public AdHocDataSourceClient createAdHocDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new StarRocksDataSourceClient(baseConnectionParam, dbType); return new StarRocksAdHocDataSourceClient(baseConnectionParam, dbType);
}
@Override
public PooledDataSourceClient createPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
return new StarRocksPooledDataSourceClient(baseConnectionParam, dbType);
} }
} }

30
dolphinscheduler-datasource-plugin/dolphinscheduler-datasource-starrocks/src/main/java/org/apache/dolphinscheduler/plugin/datasource/starrocks/StarRocksPooledDataSourceClient.java

@ -0,0 +1,30 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.dolphinscheduler.plugin.datasource.starrocks;
import org.apache.dolphinscheduler.plugin.datasource.api.client.BasePooledDataSourceClient;
import org.apache.dolphinscheduler.spi.datasource.BaseConnectionParam;
import org.apache.dolphinscheduler.spi.enums.DbType;
public class StarRocksPooledDataSourceClient extends BasePooledDataSourceClient {
public StarRocksPooledDataSourceClient(BaseConnectionParam baseConnectionParam, DbType dbType) {
super(baseConnectionParam, dbType);
}
}

Some files were not shown because too many files have changed in this diff Show More

Loading…
Cancel
Save