You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by jo...@apache.org on 2020/03/10 14:23:33 UTC

[incubator-dolphinscheduler] branch refactor-worker updated: let quartz use the same datasource (#2137)

This is an automated email from the ASF dual-hosted git repository.

journey pushed a commit to branch refactor-worker
in repository https://gitbox.apache.org/repos/asf/incubator-dolphinscheduler.git


The following commit(s) were added to refs/heads/refactor-worker by this push:
     new 7f72df4  let quartz use the same datasource (#2137)
7f72df4 is described below

commit 7f72df4ca26cbc4aa642fe107ca0288cba7eb227
Author: Tboy <gu...@immomo.com>
AuthorDate: Tue Mar 10 22:23:23 2020 +0800

    let quartz use the same datasource (#2137)
---
 .../dao/datasource/SpringConnectionFactory.java    |   3 +-
 .../master/runner/MasterSchedulerService.java      |  11 +-
 .../service/quartz/DruidConnectionProvider.java    | 190 +--------------------
 .../src/main/resources/quartz.properties           |  16 +-
 4 files changed, 22 insertions(+), 198 deletions(-)

diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SpringConnectionFactory.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SpringConnectionFactory.java
index 8eb1a2b..ad66d5a 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SpringConnectionFactory.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/datasource/SpringConnectionFactory.java
@@ -70,7 +70,7 @@ public class SpringConnectionFactory {
      * get the data source
      * @return druid dataSource
      */
-    @Bean
+    @Bean(destroyMethod="")
     public DruidDataSource dataSource() {
 
         DruidDataSource druidDataSource = new DruidDataSource();
@@ -98,7 +98,6 @@ public class SpringConnectionFactory {
         druidDataSource.setValidationQueryTimeout(conf.getInt(Constants.SPRING_DATASOURCE_VALIDATION_QUERY_TIMEOUT));
         //auto commit
         druidDataSource.setDefaultAutoCommit(conf.getBoolean(Constants.SPRING_DATASOURCE_DEFAULT_AUTO_COMMIT));
-
         return druidDataSource;
     }
 
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
index 07af6c5..c1925e0 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
@@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.stereotype.Service;
 
+import javax.annotation.PostConstruct;
 import java.util.concurrent.ThreadPoolExecutor;
 
 /**
@@ -59,9 +60,6 @@ public class MasterSchedulerService extends Thread {
     @Autowired
     private ZKMasterClient zkMasterClient;
 
-    @Autowired
-    private MasterConfig masterConfig;
-
     /**
      * master config
      */
@@ -71,17 +69,18 @@ public class MasterSchedulerService extends Thread {
     /**
      *  netty remoting client
      */
-    private final NettyRemotingClient nettyRemotingClient;
+    private NettyRemotingClient nettyRemotingClient;
 
     /**
      * master exec service
      */
-    private final ThreadPoolExecutor masterExecService;
+    private ThreadPoolExecutor masterExecService;
 
     /**
      * constructor of MasterSchedulerThread
      */
-    public MasterSchedulerService(){
+    @PostConstruct
+    public void init(){
         this.masterExecService = (ThreadPoolExecutor)ThreadUtils.newDaemonFixedThreadExecutor("Master-Exec-Thread", masterConfig.getMasterExecThreads());
         NettyClientConfig clientConfig = new NettyClientConfig();
         this.nettyRemotingClient = new NettyRemotingClient(clientConfig);
diff --git a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
index d51e8e8..3ac6cca 100644
--- a/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
+++ b/dolphinscheduler-service/src/main/java/org/apache/dolphinscheduler/service/quartz/DruidConnectionProvider.java
@@ -17,7 +17,7 @@
 package org.apache.dolphinscheduler.service.quartz;
 
 import com.alibaba.druid.pool.DruidDataSource;
-import org.quartz.SchedulerException;
+import org.apache.dolphinscheduler.service.bean.SpringApplicationContext;
 import org.quartz.utils.ConnectionProvider;
 
 import java.sql.Connection;
@@ -28,196 +28,24 @@ import java.sql.SQLException;
  */
 public class DruidConnectionProvider implements ConnectionProvider {
 
-    /**
-     * JDBC driver
-     */
-    public String driver;
+    private final DruidDataSource dataSource;
 
-    /**
-     * JDBC URL
-     */
-    public String URL;
-
-    /**
-     * Database user name
-     */
-    public String user;
-
-    /**
-     * Database password
-     */
-    public String password;
-
-    /**
-     * Maximum number of database connections
-     */
-    public int maxConnections;
-
-    /**
-     * The query that validates the database connection
-     */
-    public String validationQuery;
-
-    /**
-     * Whether the database sql query to validate connections should be executed every time
-     * a connection is retrieved from the pool to ensure that it is still valid.  If false,
-     * then validation will occur on check-in.  Default is false.
-     */
-    private boolean validateOnCheckout;
-
-    /**
-     * The number of seconds between tests of idle connections - only enabled
-     * if the validation query property is set.  Default is 50 seconds.
-     */
-    private int idleConnectionValidationSeconds;
-
-    /**
-     * The maximum number of prepared statements that will be cached per connection in the pool.
-     * Depending upon your JDBC Driver this may significantly help performance, or may slightly
-     * hinder performance.
-     * Default is 120, as Quartz uses over 100 unique statements. 0 disables the feature.
-     */
-    public String maxCachedStatementsPerConnection;
-
-    /**
-     * Discard connections after they have been idle this many seconds.  0 disables the feature. Default is 0.
-     */
-    private String discardIdleConnectionsSeconds;
-
-    /**
-     * Default maximum number of database connections in the pool.
-     */
-    public static final int DEFAULT_DB_MAX_CONNECTIONS = 10;
-
-    /**
-     * The maximum number of prepared statements that will be cached per connection in the pool.
-     */
-    public static final int DEFAULT_DB_MAX_CACHED_STATEMENTS_PER_CONNECTION = 120;
-
-    /**
-     * Druid connection pool
-     */
-    private DruidDataSource datasource;
+    public DruidConnectionProvider(){
+        this.dataSource = SpringApplicationContext.getBean(DruidDataSource.class);
+    }
 
-    /**
-     * get connection
-     * @return Connection
-     * @throws SQLException sql exception
-     */
     @Override
     public Connection getConnection() throws SQLException {
-        return datasource.getConnection();
+        return dataSource.getConnection();
     }
 
-    /**
-     * shutdown data source
-     * @throws SQLException sql exception
-     */
     @Override
     public void shutdown() throws SQLException {
-        datasource.close();
+        dataSource.close();
     }
 
-    /**
-     * data source initialize
-     * @throws SQLException sql exception
-     */
     @Override
-    public void initialize() throws SQLException{
-        if (this.URL == null) {
-            throw new SQLException("DBPool could not be created: DB URL cannot be null");
-        }
-        if (this.driver == null) {
-            throw new SQLException("DBPool driver could not be created: DB driver class name cannot be null!");
-        }
-        if (this.maxConnections < 0) {
-            throw new SQLException("DBPool maxConnectins could not be created: Max connections must be greater than zero!");
-        }
-        datasource = new DruidDataSource();
-        try{
-            datasource.setDriverClassName(this.driver);
-        } catch (Exception e) {
-            try {
-                throw new SchedulerException("Problem setting driver class name on datasource", e);
-            } catch (SchedulerException e1) {
-            }
-        }
-        datasource.setUrl(this.URL);
-        datasource.setUsername(this.user);
-        datasource.setPassword(this.password);
-        datasource.setMaxActive(this.maxConnections);
-        datasource.setMinIdle(1);
-        datasource.setMaxWait(0);
-        datasource.setMaxPoolPreparedStatementPerConnectionSize(DEFAULT_DB_MAX_CONNECTIONS);
-        if (this.validationQuery != null) {
-            datasource.setValidationQuery(this.validationQuery);
-            if(!this.validateOnCheckout){
-                datasource.setTestOnReturn(true);
-            } else {
-                datasource.setTestOnBorrow(true);
-            }
-            datasource.setValidationQueryTimeout(this.idleConnectionValidationSeconds);
-        }
-    }
-
-    public String getDriver() {
-        return driver;
-    }
-    public void setDriver(String driver) {
-        this.driver = driver;
-    }
-    public String getURL() {
-        return URL;
-    }
-    public void setURL(String URL) {
-        this.URL = URL;
-    }
-    public String getUser() {
-        return user;
-    }
-    public void setUser(String user) {
-        this.user = user;
-    }
-    public String getPassword() {
-        return password;
-    }
-    public void setPassword(String password) {
-        this.password = password;
-    }
-    public int getMaxConnections() {
-        return maxConnections;
-    }
-    public void setMaxConnections(int maxConnections) {
-        this.maxConnections = maxConnections;
-    }
-    public String getValidationQuery() {
-        return validationQuery;
-    }
-    public void setValidationQuery(String validationQuery) {
-        this.validationQuery = validationQuery;
-    }
-    public boolean isValidateOnCheckout() {
-        return validateOnCheckout;
-    }
-    public void setValidateOnCheckout(boolean validateOnCheckout) {
-        this.validateOnCheckout = validateOnCheckout;
-    }
-    public int getIdleConnectionValidationSeconds() {
-        return idleConnectionValidationSeconds;
-    }
-    public void setIdleConnectionValidationSeconds(int idleConnectionValidationSeconds) {
-        this.idleConnectionValidationSeconds = idleConnectionValidationSeconds;
-    }
-    public DruidDataSource getDatasource() {
-        return datasource;
-    }
-    public void setDatasource(DruidDataSource datasource) {
-        this.datasource = datasource;
-    }
-    public String getDiscardIdleConnectionsSeconds() {
-        return discardIdleConnectionsSeconds;
-    }
-    public void setDiscardIdleConnectionsSeconds(String discardIdleConnectionsSeconds) {
-        this.discardIdleConnectionsSeconds = discardIdleConnectionsSeconds;
+    public void initialize() throws SQLException {
+        //NOP
     }
 }
diff --git a/dolphinscheduler-service/src/main/resources/quartz.properties b/dolphinscheduler-service/src/main/resources/quartz.properties
index 9c8930b..60a0968 100644
--- a/dolphinscheduler-service/src/main/resources/quartz.properties
+++ b/dolphinscheduler-service/src/main/resources/quartz.properties
@@ -18,16 +18,16 @@
 #============================================================================
 # Configure Main Scheduler Properties
 #============================================================================
-#org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.StdJDBCDelegate
-org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate
+org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.StdJDBCDelegate
+#org.quartz.jobStore.driverDelegateClass = org.quartz.impl.jdbcjobstore.PostgreSQLDelegate
 # postgre
-org.quartz.dataSource.myDs.driver = org.postgresql.Driver
-org.quartz.dataSource.myDs.URL = jdbc:postgresql://localhost:5432/dolphinscheduler?characterEncoding=utf8
+#org.quartz.dataSource.myDs.driver = org.postgresql.Driver
+#org.quartz.dataSource.myDs.URL = jdbc:postgresql://localhost:5432/dolphinscheduler?characterEncoding=utf8
 # mysql
 #org.quartz.dataSource.myDs.driver = com.mysql.jdbc.Driver
 #org.quartz.dataSource.myDs.URL = jdbc:mysql://localhost:3306/dolphinscheduler?characterEncoding=utf8
-org.quartz.dataSource.myDs.user = test
-org.quartz.dataSource.myDs.password = test
+#org.quartz.dataSource.myDs.user = root
+#org.quartz.dataSource.myDs.password = 123456
 
 org.quartz.scheduler.instanceName = DolphinScheduler
 org.quartz.scheduler.instanceId = AUTO
@@ -59,6 +59,4 @@ org.quartz.jobStore.dataSource = myDs
 #============================================================================
 # Configure Datasources  
 #============================================================================
-org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.service.quartz.DruidConnectionProvider
-org.quartz.dataSource.myDs.maxConnections = 10
-org.quartz.dataSource.myDs.validationQuery = select 1
\ No newline at end of file
+org.quartz.dataSource.myDs.connectionProvider.class = org.apache.dolphinscheduler.service.quartz.DruidConnectionProvider
\ No newline at end of file