diff --git a/.github/workflows/cluster-test/mysql_with_mysql_registry/deploy.sh b/.github/workflows/cluster-test/mysql_with_mysql_registry/deploy.sh
index 72b2a630fa..722a4b849f 100644
--- a/.github/workflows/cluster-test/mysql_with_mysql_registry/deploy.sh
+++ b/.github/workflows/cluster-test/mysql_with_mysql_registry/deploy.sh
@@ -35,7 +35,6 @@ service ssh start
#Init schema
/bin/bash $DOLPHINSCHEDULER_HOME/tools/bin/upgrade-schema.sh
-/bin/bash $DOLPHINSCHEDULER_HOME/tools/bin/initialize-jdbc-registry.sh
#Start Cluster
/bin/bash $DOLPHINSCHEDULER_HOME/bin/start-all.sh
diff --git a/.github/workflows/cluster-test/mysql_with_mysql_registry/dolphinscheduler_env.sh b/.github/workflows/cluster-test/mysql_with_mysql_registry/dolphinscheduler_env.sh
index 8536eb0905..5c02c5f5ff 100755
--- a/.github/workflows/cluster-test/mysql_with_mysql_registry/dolphinscheduler_env.sh
+++ b/.github/workflows/cluster-test/mysql_with_mysql_registry/dolphinscheduler_env.sh
@@ -46,8 +46,12 @@ export DATAX_LAUNCHER=${DATAX_LAUNCHER:-/opt/soft/datax/bin/datax.py}
export PATH=$HADOOP_HOME/bin:$SPARK_HOME/bin:$PYTHON_LAUNCHER:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_LAUNCHER:$PATH
-export MASTER_RESERVED_MEMORY=0.01
-export WORKER_RESERVED_MEMORY=0.01
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
# applicationId auto collection related configuration, the following configurations are unnecessary if setting appId.collect=log
#export HADOOP_CLASSPATH=`hadoop classpath`:${DOLPHINSCHEDULER_HOME}/tools/libs/*
diff --git a/.github/workflows/cluster-test/mysql_with_zookeeper_registry/dolphinscheduler_env.sh b/.github/workflows/cluster-test/mysql_with_zookeeper_registry/dolphinscheduler_env.sh
index f64e59b768..e3c2891295 100755
--- a/.github/workflows/cluster-test/mysql_with_zookeeper_registry/dolphinscheduler_env.sh
+++ b/.github/workflows/cluster-test/mysql_with_zookeeper_registry/dolphinscheduler_env.sh
@@ -45,8 +45,12 @@ export DATAX_LAUNCHER=${DATAX_LAUNCHER:-/opt/soft/datax/bin/datax.py}
export PATH=$HADOOP_HOME/bin:$SPARK_HOME/bin:$PYTHON_LAUNCHER:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_LAUNCHER:$PATH
-export MASTER_RESERVED_MEMORY=0.01
-export WORKER_RESERVED_MEMORY=0.01
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
# applicationId auto collection related configuration, the following configurations are unnecessary if setting appId.collect=log
#export HADOOP_CLASSPATH=`hadoop classpath`:${DOLPHINSCHEDULER_HOME}/tools/libs/*
diff --git a/.github/workflows/cluster-test/postgresql_with_postgresql_registry/deploy.sh b/.github/workflows/cluster-test/postgresql_with_postgresql_registry/deploy.sh
index 37bf3433c0..f7547d40b4 100644
--- a/.github/workflows/cluster-test/postgresql_with_postgresql_registry/deploy.sh
+++ b/.github/workflows/cluster-test/postgresql_with_postgresql_registry/deploy.sh
@@ -32,7 +32,6 @@ service ssh start
#Init schema
/bin/bash $DOLPHINSCHEDULER_HOME/tools/bin/upgrade-schema.sh
-/bin/bash $DOLPHINSCHEDULER_HOME/tools/bin/initialize-jdbc-registry.sh
#Start Cluster
/bin/bash $DOLPHINSCHEDULER_HOME/bin/start-all.sh
diff --git a/.github/workflows/cluster-test/postgresql_with_postgresql_registry/dolphinscheduler_env.sh b/.github/workflows/cluster-test/postgresql_with_postgresql_registry/dolphinscheduler_env.sh
index 29f8570319..ea2447c224 100644
--- a/.github/workflows/cluster-test/postgresql_with_postgresql_registry/dolphinscheduler_env.sh
+++ b/.github/workflows/cluster-test/postgresql_with_postgresql_registry/dolphinscheduler_env.sh
@@ -46,8 +46,12 @@ export DATAX_LAUNCHER=${DATAX_LAUNCHER:-/opt/soft/datax/bin/datax.py}
export PATH=$HADOOP_HOME/bin:$SPARK_HOME/bin:$PYTHON_LAUNCHER:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_LAUNCHER:$PATH
-export MASTER_RESERVED_MEMORY=0.01
-export WORKER_RESERVED_MEMORY=0.01
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
# applicationId auto collection related configuration, the following configurations are unnecessary if setting appId.collect=log
#export HADOOP_CLASSPATH=`hadoop classpath`:${DOLPHINSCHEDULER_HOME}/tools/libs/*
diff --git a/.github/workflows/cluster-test/postgresql_with_zookeeper_registry/dolphinscheduler_env.sh b/.github/workflows/cluster-test/postgresql_with_zookeeper_registry/dolphinscheduler_env.sh
index 6851716058..021f1319d2 100644
--- a/.github/workflows/cluster-test/postgresql_with_zookeeper_registry/dolphinscheduler_env.sh
+++ b/.github/workflows/cluster-test/postgresql_with_zookeeper_registry/dolphinscheduler_env.sh
@@ -45,8 +45,12 @@ export DATAX_LAUNCHER=${DATAX_LAUNCHER:-/opt/soft/datax/bin/datax.py}
export PATH=$HADOOP_HOME/bin:$SPARK_HOME/bin:$PYTHON_LAUNCHER:$JAVA_HOME/bin:$HIVE_HOME/bin:$FLINK_HOME/bin:$DATAX_LAUNCHER:$PATH
-export MASTER_RESERVED_MEMORY=0.01
-export WORKER_RESERVED_MEMORY=0.01
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export MASTER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_JVM_CPU_USAGE_PERCENTAGE_THRESHOLDS=0.7
+export WORKER_SERVER_LOAD_PROTECTION_MAX_SYSTEM_MEMORY_USAGE_PERCENTAGE_THRESHOLDS=0.7
# applicationId auto collection related configuration, the following configurations are unnecessary if setting appId.collect=log
#export HADOOP_CLASSPATH=`hadoop classpath`:${DOLPHINSCHEDULER_HOME}/tools/libs/*
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
index c4271919b5..a24f4ac615 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/thread/ThreadUtils.java
@@ -39,6 +39,10 @@ public class ThreadUtils {
return Executors.newSingleThreadScheduledExecutor(newDaemonThreadFactory(threadName));
}
+ public static ScheduledExecutorService newDaemonScheduledExecutorService(String threadName, int threadsNum) {
+ return Executors.newScheduledThreadPool(threadsNum, newDaemonThreadFactory(threadName));
+ }
+
public static ThreadFactory newDaemonThreadFactory(String threadName) {
return new ThreadFactoryBuilder()
.setDaemon(true)
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
index 6069284bbd..64c07c5410 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_h2.sql
@@ -2207,3 +2207,64 @@ CREATE TABLE t_ds_process_task_lineage
KEY `idx_task_code_version` (`task_definition_code`,`task_definition_version`),
KEY `idx_dept_code` (`dept_project_code`,`dept_process_definition_code`,`dept_task_definition_code`)
);
+
+
+-- ----------------------------
+-- Table structure for jdbc registry
+-- ----------------------------
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data`;
+CREATE TABLE `t_ds_jdbc_registry_data`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',
+ `data_value` text NOT NULL COMMENT 'data, like zookeeper node value',
+ `data_type` varchar(64) NOT NULL COMMENT 'EPHEMERAL, PERSISTENT',
+ `client_id` bigint NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL default current_timestamp COMMENT 'create time',
+ `last_update_time` timestamp NOT NULL default current_timestamp COMMENT 'last update time',
+ PRIMARY KEY (`id`),
+ unique KEY `uk_t_ds_jdbc_registry_dataKey`(`data_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_lock`;
+CREATE TABLE `t_ds_jdbc_registry_lock`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `lock_key` varchar(256) NOT NULL COMMENT 'lock path',
+ `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',
+ `client_id` bigint NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL default current_timestamp COMMENT 'create time',
+ PRIMARY KEY (`id`),
+ unique KEY `uk_t_ds_jdbc_registry_lockKey`(`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_ds_jdbc_registry_client_heartbeat`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `client_name` varchar(256) NOT NULL COMMENT 'client name, ip_processId',
+ `last_heartbeat_time` bigint NOT NULL COMMENT 'last heartbeat timestamp',
+ `connection_config` text NOT NULL COMMENT 'connection config',
+ `create_time` timestamp NOT NULL default current_timestamp COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data_change_event`;
+CREATE TABLE `t_ds_jdbc_registry_data_change_event`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `event_type` varchar(64) NOT NULL COMMENT 'ADD, UPDATE, DELETE',
+ `jdbc_registry_data` text NOT NULL COMMENT 'jdbc registry data',
+ `create_time` timestamp NOT NULL default current_timestamp COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+-- ----------------------------
+-- Table structure for jdbc registry
+-- ----------------------------
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
index f60d78bf5f..82f166bb55 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_mysql.sql
@@ -2178,3 +2178,55 @@ CREATE TABLE `t_ds_process_task_lineage` (
KEY `idx_task_code_version` (`task_definition_code`,`task_definition_version`),
KEY `idx_dept_code` (`dept_project_code`,`dept_process_definition_code`,`dept_task_definition_code`)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data`;
+CREATE TABLE `t_ds_jdbc_registry_data`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',
+ `data_value` text NOT NULL COMMENT 'data, like zookeeper node value',
+ `data_type` varchar(64) NOT NULL COMMENT 'EPHEMERAL, PERSISTENT',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'last update time',
+ PRIMARY KEY (`id`),
+ unique Key `uk_t_ds_jdbc_registry_dataKey` (`data_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_lock`;
+CREATE TABLE `t_ds_jdbc_registry_lock`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `lock_key` varchar(256) NOT NULL COMMENT 'lock path',
+ `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`),
+ unique Key `uk_t_ds_jdbc_registry_lockKey` (`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_ds_jdbc_registry_client_heartbeat`
+(
+ `id` bigint(11) NOT NULL COMMENT 'primary key',
+ `client_name` varchar(256) NOT NULL COMMENT 'client name, ip_processId',
+ `last_heartbeat_time` bigint(11) NOT NULL COMMENT 'last heartbeat timestamp',
+ `connection_config` text NOT NULL COMMENT 'connection config',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data_change_event`;
+CREATE TABLE `t_ds_jdbc_registry_data_change_event`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `event_type` varchar(64) NOT NULL COMMENT 'ADD, UPDATE, DELETE',
+ `jdbc_registry_data` text NOT NULL COMMENT 'jdbc registry data',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
diff --git a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
index cbaa38f039..af1802b603 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/dolphinscheduler_postgresql.sql
@@ -2171,3 +2171,52 @@ CREATE TABLE t_ds_process_task_lineage (
create index idx_process_code_version on t_ds_process_task_lineage (process_definition_code,process_definition_version);
create index idx_task_code_version on t_ds_process_task_lineage (task_definition_code,task_definition_version);
create index idx_dept_code on t_ds_process_task_lineage (dept_project_code,dept_process_definition_code,dept_task_definition_code);
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_data;
+create table t_ds_jdbc_registry_data
+(
+ id bigserial not null,
+ data_key varchar not null,
+ data_value text not null,
+ data_type varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ last_update_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+create unique index uk_t_ds_jdbc_registry_dataKey on t_ds_jdbc_registry_data (data_key);
+
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_lock;
+create table t_ds_jdbc_registry_lock
+(
+ id bigserial not null,
+ lock_key varchar not null,
+ lock_owner varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+create unique index uk_t_ds_jdbc_registry_lockKey on t_ds_jdbc_registry_lock (lock_key);
+
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_client_heartbeat;
+create table t_ds_jdbc_registry_client_heartbeat
+(
+ id bigint not null,
+ client_name varchar not null,
+ last_heartbeat_time bigint not null,
+ connection_config text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_data_change_event;
+create table t_ds_jdbc_registry_data_change_event
+(
+ id bigserial not null,
+ event_type varchar not null,
+ jdbc_registry_data text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql
index e90459d209..d77f639a63 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/mysql/dolphinscheduler_ddl.sql
@@ -33,3 +33,55 @@ CREATE TABLE `t_ds_process_task_lineage`
KEY `idx_task_code_version` (`task_definition_code`,`task_definition_version`),
KEY `idx_dept_code` (`dept_project_code`,`dept_process_definition_code`,`dept_task_definition_code`)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data`;
+CREATE TABLE `t_ds_jdbc_registry_data`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',
+ `data_value` text NOT NULL COMMENT 'data, like zookeeper node value',
+ `data_type` varchar(64) NOT NULL COMMENT 'EPHEMERAL, PERSISTENT',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'last update time',
+ PRIMARY KEY (`id`),
+ unique Key `uk_t_ds_jdbc_registry_dataKey` (`data_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_lock`;
+CREATE TABLE `t_ds_jdbc_registry_lock`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `lock_key` varchar(256) NOT NULL COMMENT 'lock path',
+ `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`),
+ unique Key `uk_t_ds_jdbc_registry_lockKey` (`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_ds_jdbc_registry_client_heartbeat`
+(
+ `id` bigint(11) NOT NULL COMMENT 'primary key',
+ `client_name` varchar(256) NOT NULL COMMENT 'client name, ip_processId',
+ `last_heartbeat_time` bigint(11) NOT NULL COMMENT 'last heartbeat timestamp',
+ `connection_config` text NOT NULL COMMENT 'connection config',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data_change_event`;
+CREATE TABLE `t_ds_jdbc_registry_data_change_event`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `event_type` varchar(64) NOT NULL COMMENT 'ADD, UPDATE, DELETE',
+ `jdbc_registry_data` text NOT NULL COMMENT 'jdbc registry data',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
\ No newline at end of file
diff --git a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql
index 593d9998fd..c775636d4b 100644
--- a/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql
+++ b/dolphinscheduler-dao/src/main/resources/sql/upgrade/3.3.0_schema/postgresql/dolphinscheduler_ddl.sql
@@ -33,3 +33,52 @@ CREATE TABLE t_ds_process_task_lineage (
create index idx_process_code_version on t_ds_process_task_lineage (process_definition_code,process_definition_version);
create index idx_task_code_version on t_ds_process_task_lineage (task_definition_code,task_definition_version);
create index idx_dept_code on t_ds_process_task_lineage (dept_project_code,dept_process_definition_code,dept_task_definition_code);
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_data;
+create table t_ds_jdbc_registry_data
+(
+ id bigserial not null,
+ data_key varchar not null,
+ data_value text not null,
+ data_type varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ last_update_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+create unique index uk_t_ds_jdbc_registry_dataKey on t_ds_jdbc_registry_data (data_key);
+
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_lock;
+create table t_ds_jdbc_registry_lock
+(
+ id bigserial not null,
+ lock_key varchar not null,
+ lock_owner varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+create unique index uk_t_ds_jdbc_registry_lockKey on t_ds_jdbc_registry_lock (lock_key);
+
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_client_heartbeat;
+create table t_ds_jdbc_registry_client_heartbeat
+(
+ id bigint not null,
+ client_name varchar not null,
+ last_heartbeat_time bigint not null,
+ connection_config text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_data_change_event;
+create table t_ds_jdbc_registry_data_change_event
+(
+ id bigserial not null,
+ event_type varchar not null,
+ jdbc_registry_data text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
\ No newline at end of file
diff --git a/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/metrics/DefaultMetricsProvider.java b/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/metrics/DefaultMetricsProvider.java
index d0187a467e..d227679446 100644
--- a/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/metrics/DefaultMetricsProvider.java
+++ b/dolphinscheduler-meter/src/main/java/org/apache/dolphinscheduler/meter/metrics/DefaultMetricsProvider.java
@@ -19,8 +19,10 @@ package org.apache.dolphinscheduler.meter.metrics;
import org.apache.dolphinscheduler.common.utils.OSUtils;
+import lombok.extern.slf4j.Slf4j;
import io.micrometer.core.instrument.MeterRegistry;
+@Slf4j
public class DefaultMetricsProvider implements MetricsProvider {
private final MeterRegistry meterRegistry;
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/Registry.java b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/Registry.java
index f90ef1ea32..50e3d84374 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/Registry.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-api/src/main/java/org/apache/dolphinscheduler/registry/api/Registry.java
@@ -52,17 +52,14 @@ public interface Registry extends Closeable {
/**
* Subscribe the path, when the path has expose {@link Event}, the listener will be triggered.
+ *
+ * The sub path will also be watched, if the sub path has event, the listener will be triggered.
*
* @param path the path to subscribe
* @param listener the listener to be triggered
*/
void subscribe(String path, SubscribeListener listener);
- /**
- * Remove the path from the subscribe list.
- */
- void unsubscribe(String path);
-
/**
* Add a connection listener to collection.
*/
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java
index 80279775ff..54a270f047 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-etcd/src/main/java/org/apache/dolphinscheduler/plugin/registry/etcd/EtcdRegistry.java
@@ -166,20 +166,6 @@ public class EtcdRegistry implements Registry {
}
}
- /**
- * @param path The prefix of the key being listened to
- * @throws throws an exception if the unsubscribe path does not exist
- */
- @Override
- public void unsubscribe(String path) {
- try {
- watcherMap.get(path).close();
- watcherMap.remove(path);
- } catch (Exception e) {
- throw new RegistryException("Failed to unsubscribe listener for key: " + path, e);
- }
- }
-
@Override
public void addConnectionStateListener(ConnectionListener listener) {
etcdConnectionStateListener.addConnectionListener(listener);
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-it/src/test/java/org/apache/dolphinscheduler/plugin/registry/RegistryTestCase.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-it/src/test/java/org/apache/dolphinscheduler/plugin/registry/RegistryTestCase.java
index 8fbd6bc5c0..1edb0327f8 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-it/src/test/java/org/apache/dolphinscheduler/plugin/registry/RegistryTestCase.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-it/src/test/java/org/apache/dolphinscheduler/plugin/registry/RegistryTestCase.java
@@ -112,41 +112,6 @@ public abstract class RegistryTestCase {
});
}
- @SneakyThrows
- @Test
- public void testUnsubscribe() {
- registry.start();
-
- final AtomicBoolean subscribeAdded = new AtomicBoolean(false);
- final AtomicBoolean subscribeRemoved = new AtomicBoolean(false);
- final AtomicBoolean subscribeUpdated = new AtomicBoolean(false);
-
- SubscribeListener subscribeListener = event -> {
- if (event.type() == Event.Type.ADD) {
- subscribeAdded.compareAndSet(false, true);
- }
- if (event.type() == Event.Type.REMOVE) {
- subscribeRemoved.compareAndSet(false, true);
- }
- if (event.type() == Event.Type.UPDATE) {
- subscribeUpdated.compareAndSet(false, true);
- }
- };
- String key = "/nodes/master" + System.nanoTime();
- String value = "127.0.0.1:8080";
- registry.subscribe(key, subscribeListener);
- registry.unsubscribe(key);
- registry.put(key, value, true);
- registry.put(key, value, true);
- registry.delete(key);
-
- Thread.sleep(2000);
- Assertions.assertFalse(subscribeAdded.get());
- Assertions.assertFalse(subscribeRemoved.get());
- Assertions.assertFalse(subscribeUpdated.get());
-
- }
-
@SneakyThrows
@Test
public void testAddConnectionStateListener() {
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/README.md b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/README.md
index 554c375218..094b727eb2 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/README.md
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/README.md
@@ -34,11 +34,18 @@ Database.
```yaml
registry:
type: jdbc
- # Used to schedule refresh the ephemeral data/ lock.
- term-refresh-interval: 2s
- # Used to calculate the expire time,
- # e.g. if you set 2, and latest two refresh error, then the ephemeral data/lock will be expire.
- term-expire-times: 3
+ # Used to schedule refresh the heartbeat.
+ heartbeat-refresh-interval: 3s
+ # Once the client's heartbeat is not refresh in this time, the server will consider the client is offline.
+ session-timeout: 60s
+ # The hikari configuration, default will use the same datasource pool as DolphinScheduler.
+ hikari-config:
+ jdbc-url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler
+ username: root
+ password: root
+ maximum-pool-size: 5
+ connection-timeout: 9000
+ idle-timeout: 600000
```
## Use different database configuration for jdbc registry center
@@ -50,8 +57,8 @@ You need to set the registry properties in master/worker/api's application.yml
```yaml
registry:
type: jdbc
- term-refresh-interval: 2s
- term-expire-times: 3
+ heartbeat-refresh-interval: 3s
+ session-timeout: 60s
hikari-config:
jdbc-url: jdbc:mysql://127.0.0.1:3306/dolphinscheduler
username: root
@@ -66,8 +73,8 @@ registry:
```yaml
registry:
type: jdbc
- term-refresh-interval: 2s
- term-expire-times: 3
+ heartbeat-refresh-interval: 3s
+ session-timeout: 60s
hikari-config:
jdbc-url: jdbc:postgresql://localhost:5432/dolphinscheduler
username: root
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/EphemeralDateManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/EphemeralDateManager.java
deleted file mode 100644
index 7c601b91a1..0000000000
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/EphemeralDateManager.java
+++ /dev/null
@@ -1,164 +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.registry.jdbc;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import org.apache.dolphinscheduler.registry.api.ConnectionListener;
-import org.apache.dolphinscheduler.registry.api.ConnectionState;
-
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import lombok.extern.slf4j.Slf4j;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * This thread is used to check the connect state to jdbc.
- */
-@Slf4j
-class EphemeralDateManager implements AutoCloseable {
-
- private ConnectionState connectionState;
- private final JdbcOperator jdbcOperator;
- private final JdbcRegistryProperties registryProperties;
- private final List connectionListeners = Collections.synchronizedList(new ArrayList<>());
- private final Set ephemeralDateIds = Collections.synchronizedSet(new HashSet<>());
- private final ScheduledExecutorService scheduledExecutorService;
-
- EphemeralDateManager(JdbcRegistryProperties registryProperties, JdbcOperator jdbcOperator) {
- this.registryProperties = registryProperties;
- this.jdbcOperator = checkNotNull(jdbcOperator);
- this.scheduledExecutorService = Executors.newScheduledThreadPool(
- 1,
- new ThreadFactoryBuilder().setNameFormat("EphemeralDateTermRefreshThread").setDaemon(true).build());
- }
-
- public void start() {
- this.scheduledExecutorService.scheduleWithFixedDelay(
- new EphemeralDateTermRefreshTask(jdbcOperator, connectionListeners, ephemeralDateIds),
- registryProperties.getTermRefreshInterval().toMillis(),
- registryProperties.getTermRefreshInterval().toMillis(),
- TimeUnit.MILLISECONDS);
- }
-
- public void addConnectionListener(ConnectionListener connectionListener) {
- connectionListeners.add(connectionListener);
- }
-
- public long insertOrUpdateEphemeralData(String key, String value) throws SQLException {
- long ephemeralId = jdbcOperator.insertOrUpdateEphemeralData(key, value);
- ephemeralDateIds.add(ephemeralId);
- return ephemeralId;
- }
-
- public ConnectionState getConnectionState() {
- return connectionState;
- }
-
- @Override
- public void close() throws SQLException {
- for (Long ephemeralDateId : ephemeralDateIds) {
- jdbcOperator.deleteDataById(ephemeralDateId);
- }
- ephemeralDateIds.clear();
- connectionListeners.clear();
- scheduledExecutorService.shutdownNow();
- }
-
- // Use this task to refresh ephemeral term and check the connect state.
- class EphemeralDateTermRefreshTask implements Runnable {
-
- private final List connectionListeners;
- private final Set ephemeralDateIds;
- private final JdbcOperator jdbcOperator;
-
- private EphemeralDateTermRefreshTask(JdbcOperator jdbcOperator,
- List connectionListeners,
- Set ephemeralDateIds) {
- this.jdbcOperator = checkNotNull(jdbcOperator);
- this.connectionListeners = checkNotNull(connectionListeners);
- this.ephemeralDateIds = checkNotNull(ephemeralDateIds);
- }
-
- @Override
- public void run() {
- try {
- ConnectionState currentConnectionState = getConnectionState();
- if (currentConnectionState == connectionState) {
- // no state change
- return;
- }
-
- if (connectionState == ConnectionState.CONNECTED) {
- if (currentConnectionState == ConnectionState.DISCONNECTED) {
- connectionState = ConnectionState.DISCONNECTED;
- triggerListener(ConnectionState.DISCONNECTED);
- }
- } else if (connectionState == ConnectionState.DISCONNECTED) {
- if (currentConnectionState == ConnectionState.CONNECTED) {
- connectionState = ConnectionState.CONNECTED;
- triggerListener(ConnectionState.RECONNECTED);
- }
- } else if (connectionState == null) {
- connectionState = currentConnectionState;
- triggerListener(connectionState);
- }
- } catch (Exception e) {
- log.error("Jdbc Registry connect state check task execute failed", e);
- connectionState = ConnectionState.DISCONNECTED;
- triggerListener(ConnectionState.DISCONNECTED);
- }
- }
-
- private ConnectionState getConnectionState() {
- try {
- if (ephemeralDateIds.isEmpty()) {
- jdbcOperator.healthCheck();
- } else {
- updateEphemeralDateTerm();
- }
- jdbcOperator.clearExpireEphemeralDate();
- return ConnectionState.CONNECTED;
- } catch (Exception ex) {
- log.error("Get connection state error, meet an unknown exception", ex);
- return ConnectionState.DISCONNECTED;
- }
- }
-
- private void updateEphemeralDateTerm() {
- if (!jdbcOperator.updateEphemeralDataTerm(ephemeralDateIds)) {
- log.warn("Update jdbc registry ephemeral data: {} term error", ephemeralDateIds);
- }
- }
-
- private void triggerListener(ConnectionState connectionState) {
- for (ConnectionListener connectionListener : connectionListeners) {
- connectionListener.onUpdate(connectionState);
- }
- }
- }
-}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcOperator.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcOperator.java
deleted file mode 100644
index 46ba84db8a..0000000000
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcOperator.java
+++ /dev/null
@@ -1,184 +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.registry.jdbc;
-
-import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryDataMapper;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryLockMapper;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DataType;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryData;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryLock;
-
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
-
-import java.sql.SQLException;
-import java.util.Collection;
-import java.util.Date;
-import java.util.List;
-import java.util.stream.Collectors;
-
-import org.springframework.dao.DuplicateKeyException;
-
-public final class JdbcOperator {
-
- private final JdbcRegistryDataMapper jdbcRegistryDataMapper;
- private final JdbcRegistryLockMapper jdbcRegistryLockMapper;
- private final long expireTimeWindow;
-
- JdbcOperator(JdbcRegistryProperties registryProperties,
- JdbcRegistryDataMapper jdbcRegistryDataMapper,
- JdbcRegistryLockMapper jdbcRegistryLockMapper) {
- this.expireTimeWindow =
- registryProperties.getTermExpireTimes() * registryProperties.getTermRefreshInterval().toMillis();
- this.jdbcRegistryDataMapper = jdbcRegistryDataMapper;
- this.jdbcRegistryLockMapper = jdbcRegistryLockMapper;
- }
-
- public void healthCheck() {
- jdbcRegistryLockMapper.countAll();
- }
-
- public List queryAllJdbcRegistryData() {
- return jdbcRegistryDataMapper.selectAll();
- }
-
- public Long insertOrUpdateEphemeralData(String key, String value) throws SQLException {
- JdbcRegistryData jdbcRegistryData = jdbcRegistryDataMapper.selectByKey(key);
- if (jdbcRegistryData != null) {
- jdbcRegistryData.setDataValue(value);
- jdbcRegistryData.setLastUpdateTime(new Date());
- jdbcRegistryData.setLastTerm(System.currentTimeMillis());
- if (jdbcRegistryDataMapper.updateById(jdbcRegistryData) <= 0) {
- throw new SQLException(String.format("update registry value failed, key: %s, value: %s", key, value));
- }
- return jdbcRegistryData.getId();
- }
- jdbcRegistryData = JdbcRegistryData.builder()
- .dataKey(key)
- .dataValue(value)
- .dataType(DataType.EPHEMERAL.getTypeValue())
- .lastTerm(System.currentTimeMillis())
- .lastUpdateTime(new Date())
- .createTime(new Date())
- .build();
- jdbcRegistryDataMapper.insert(jdbcRegistryData);
- return jdbcRegistryData.getId();
- }
-
- public long insertOrUpdatePersistentData(String key, String value) throws SQLException {
- JdbcRegistryData jdbcRegistryData = jdbcRegistryDataMapper.selectByKey(key);
- if (jdbcRegistryData != null) {
- jdbcRegistryData.setDataValue(value);
- jdbcRegistryData.setLastUpdateTime(new Date());
- jdbcRegistryData.setLastTerm(System.currentTimeMillis());
- if (jdbcRegistryDataMapper.updateById(jdbcRegistryData) <= 0) {
- throw new SQLException(String.format("update registry value failed, key: %s, value: %s", key, value));
- }
- return jdbcRegistryData.getId();
- }
- jdbcRegistryData = JdbcRegistryData.builder()
- .dataKey(key)
- .dataValue(value)
- .dataType(DataType.PERSISTENT.getTypeValue())
- .lastTerm(System.currentTimeMillis())
- .lastUpdateTime(new Date())
- .createTime(new Date())
- .build();
- jdbcRegistryDataMapper.insert(jdbcRegistryData);
- return jdbcRegistryData.getId();
- }
-
- public void deleteDataByKey(String key) {
- jdbcRegistryDataMapper.deleteByKey(key);
- }
-
- public void deleteDataById(long id) {
- jdbcRegistryDataMapper.deleteById(id);
- }
-
- public void clearExpireLock() {
- jdbcRegistryLockMapper.clearExpireLock(System.currentTimeMillis() - expireTimeWindow);
- }
-
- public void clearExpireEphemeralDate() {
- jdbcRegistryDataMapper.clearExpireEphemeralDate(System.currentTimeMillis() - expireTimeWindow,
- DataType.EPHEMERAL.getTypeValue());
- }
-
- public JdbcRegistryData getData(String key) throws SQLException {
- return jdbcRegistryDataMapper.selectByKey(key);
- }
-
- public List getChildren(String key) throws SQLException {
- return jdbcRegistryDataMapper.fuzzyQueryByKey(key)
- .stream()
- .map(JdbcRegistryData::getDataKey)
- .filter(fullPath -> fullPath.length() > key.length())
- .map(fullPath -> StringUtils.substringBefore(fullPath.substring(key.length() + 1), "/"))
- .collect(Collectors.toList());
- }
-
- public boolean existKey(String key) {
- JdbcRegistryData jdbcRegistryData = jdbcRegistryDataMapper.selectByKey(key);
- return jdbcRegistryData != null;
- }
-
- /**
- * Try to acquire the target Lock, if cannot acquire, return null.
- */
- public JdbcRegistryLock tryToAcquireLock(String key) {
- JdbcRegistryLock jdbcRegistryLock = JdbcRegistryLock.builder()
- .lockKey(key)
- .lockOwner(LockUtils.getLockOwner())
- .lastTerm(System.currentTimeMillis())
- .lastUpdateTime(new Date())
- .build();
- try {
- jdbcRegistryLockMapper.insert(jdbcRegistryLock);
- return jdbcRegistryLock;
- } catch (Exception e) {
- if (e instanceof DuplicateKeyException) {
- return null;
- }
- throw e;
- }
- }
-
- public JdbcRegistryLock getLockById(long lockId) {
- return jdbcRegistryLockMapper.selectById(lockId);
- }
-
- public boolean releaseLock(long lockId) throws SQLException {
- return jdbcRegistryLockMapper.deleteById(lockId) > 0;
- }
-
- public boolean updateEphemeralDataTerm(Collection ephemeralDateIds) {
- if (CollectionUtils.isEmpty(ephemeralDateIds)) {
- return true;
- }
- return jdbcRegistryDataMapper.updateTermByIds(ephemeralDateIds, System.currentTimeMillis()) > 0;
- }
-
- public boolean updateLockTerm(List lockIds) {
- if (CollectionUtils.isEmpty(lockIds)) {
- return true;
- }
- return jdbcRegistryLockMapper.updateTermByIds(lockIds, System.currentTimeMillis()) > 0;
- }
-
-}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistry.java
index 2b7993c87b..c8f205dafd 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistry.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistry.java
@@ -17,16 +17,28 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryData;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.client.JdbcRegistryClient;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryDataChangeListener;
import org.apache.dolphinscheduler.registry.api.ConnectionListener;
import org.apache.dolphinscheduler.registry.api.ConnectionState;
+import org.apache.dolphinscheduler.registry.api.Event;
import org.apache.dolphinscheduler.registry.api.Registry;
import org.apache.dolphinscheduler.registry.api.RegistryException;
import org.apache.dolphinscheduler.registry.api.SubscribeListener;
-import java.sql.SQLException;
+import org.apache.commons.lang3.StringUtils;
+
import java.time.Duration;
import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@@ -39,37 +51,28 @@ import lombok.extern.slf4j.Slf4j;
public final class JdbcRegistry implements Registry {
private final JdbcRegistryProperties jdbcRegistryProperties;
- private final EphemeralDateManager ephemeralDateManager;
- private final SubscribeDataManager subscribeDataManager;
- private final RegistryLockManager registryLockManager;
- private final JdbcOperator jdbcOperator;
-
- JdbcRegistry(JdbcRegistryProperties jdbcRegistryProperties,
- JdbcOperator jdbcOperator) {
- this.jdbcOperator = jdbcOperator;
- jdbcOperator.clearExpireLock();
- jdbcOperator.clearExpireEphemeralDate();
+ private final JdbcRegistryClient jdbcRegistryClient;
+
+ private final IJdbcRegistryServer jdbcRegistryServer;
+
+ JdbcRegistry(JdbcRegistryProperties jdbcRegistryProperties, IJdbcRegistryServer jdbcRegistryServer) {
this.jdbcRegistryProperties = jdbcRegistryProperties;
- this.ephemeralDateManager = new EphemeralDateManager(jdbcRegistryProperties, jdbcOperator);
- this.subscribeDataManager = new SubscribeDataManager(jdbcRegistryProperties, jdbcOperator);
- this.registryLockManager = new RegistryLockManager(jdbcRegistryProperties, jdbcOperator);
+ this.jdbcRegistryServer = jdbcRegistryServer;
+ this.jdbcRegistryClient = new JdbcRegistryClient(jdbcRegistryProperties, jdbcRegistryServer);
log.info("Initialize Jdbc Registry...");
}
@Override
public void start() {
log.info("Starting Jdbc Registry...");
- // start a jdbc connect check
- ephemeralDateManager.start();
- subscribeDataManager.start();
- registryLockManager.start();
+ jdbcRegistryServer.start();
+ jdbcRegistryClient.start();
log.info("Started Jdbc Registry...");
}
@Override
public boolean isConnected() {
- jdbcOperator.healthCheck();
- return true;
+ return jdbcRegistryClient.isConnectivity();
}
@Override
@@ -81,11 +84,11 @@ public final class JdbcRegistry implements Registry {
throw new RegistryException(
String.format("Cannot connect to jdbc registry in %s s", timeout.getSeconds()));
}
- if (ephemeralDateManager.getConnectionState() == ConnectionState.CONNECTED) {
+ if (jdbcRegistryClient.isConnectivity()) {
return;
}
try {
- Thread.sleep(jdbcRegistryProperties.getTermRefreshInterval().toMillis());
+ Thread.sleep(jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis());
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
throw new RegistryException("Cannot connect to jdbc registry due to interrupted exception", e);
@@ -95,30 +98,85 @@ public final class JdbcRegistry implements Registry {
@Override
public void subscribe(String path, SubscribeListener listener) {
- // new a schedule thread to query the path, if the path
- subscribeDataManager.addListener(path, listener);
- }
+ checkNotNull(path);
+ checkNotNull(listener);
+ jdbcRegistryClient.subscribeJdbcRegistryDataChange(new JdbcRegistryDataChangeListener() {
- @Override
- public void unsubscribe(String path) {
- subscribeDataManager.removeListener(path);
+ @Override
+ public void onJdbcRegistryDataChanged(String key, String value) {
+ if (!key.startsWith(path)) {
+ return;
+ }
+ Event event = Event.builder()
+ .key(key)
+ .path(path)
+ .data(value)
+ .type(Event.Type.UPDATE)
+ .build();
+ listener.notify(event);
+ }
+
+ @Override
+ public void onJdbcRegistryDataDeleted(String key) {
+ if (!key.startsWith(path)) {
+ return;
+ }
+ Event event = Event.builder()
+ .key(key)
+ .path(key)
+ .type(Event.Type.REMOVE)
+ .build();
+ listener.notify(event);
+ }
+
+ @Override
+ public void onJdbcRegistryDataAdded(String key, String value) {
+ if (!key.startsWith(path)) {
+ return;
+ }
+ Event event = Event.builder()
+ .key(key)
+ .path(key)
+ .data(value)
+ .type(Event.Type.ADD)
+ .build();
+ listener.notify(event);
+ }
+ });
}
@Override
public void addConnectionStateListener(ConnectionListener listener) {
- // check the current connection
- ephemeralDateManager.addConnectionListener(listener);
+ checkNotNull(listener);
+ jdbcRegistryClient.subscribeConnectionStateChange(new ConnectionStateListener() {
+
+ @Override
+ public void onConnected() {
+ listener.onUpdate(ConnectionState.CONNECTED);
+ }
+
+ @Override
+ public void onDisConnected() {
+ listener.onUpdate(ConnectionState.DISCONNECTED);
+ }
+
+ @Override
+ public void onReconnected() {
+ listener.onUpdate(ConnectionState.RECONNECTED);
+ }
+ });
}
@Override
public String get(String key) {
try {
// get the key value
- JdbcRegistryData data = jdbcOperator.getData(key);
- if (data == null) {
+ // Directly get from the db?
+ Optional jdbcRegistryDataOptional = jdbcRegistryClient.getJdbcRegistryDataByKey(key);
+ if (!jdbcRegistryDataOptional.isPresent()) {
throw new RegistryException("key: " + key + " not exist");
}
- return data.getDataValue();
+ return jdbcRegistryDataOptional.get().getDataValue();
} catch (RegistryException registryException) {
throw registryException;
} catch (Exception e) {
@@ -129,12 +187,8 @@ public final class JdbcRegistry implements Registry {
@Override
public void put(String key, String value, boolean deleteOnDisconnect) {
try {
- if (deleteOnDisconnect) {
- // when put a ephemeralData will new a scheduler thread to update it
- ephemeralDateManager.insertOrUpdateEphemeralData(key, value);
- } else {
- jdbcOperator.insertOrUpdatePersistentData(key, value);
- }
+ DataType dataType = deleteOnDisconnect ? DataType.EPHEMERAL : DataType.PERSISTENT;
+ jdbcRegistryClient.putJdbcRegistryData(key, value, dataType);
} catch (Exception ex) {
throw new RegistryException(String.format("put key:%s, value:%s error", key, value), ex);
}
@@ -143,7 +197,7 @@ public final class JdbcRegistry implements Registry {
@Override
public void delete(String key) {
try {
- jdbcOperator.deleteDataByKey(key);
+ jdbcRegistryClient.deleteJdbcRegistryDataByKey(key);
} catch (Exception e) {
throw new RegistryException(String.format("Delete key: %s error", key), e);
}
@@ -152,8 +206,14 @@ public final class JdbcRegistry implements Registry {
@Override
public Collection children(String key) {
try {
- return jdbcOperator.getChildren(key);
- } catch (SQLException e) {
+ List children = jdbcRegistryClient.listJdbcRegistryDataChildren(key);
+ return children
+ .stream()
+ .map(JdbcRegistryDataDTO::getDataKey)
+ .filter(fullPath -> fullPath.length() > key.length())
+ .map(fullPath -> StringUtils.substringBefore(fullPath.substring(key.length() + 1), "/"))
+ .collect(Collectors.toList());
+ } catch (Exception e) {
throw new RegistryException(String.format("Get key: %s children error", key), e);
}
}
@@ -161,7 +221,7 @@ public final class JdbcRegistry implements Registry {
@Override
public boolean exists(String key) {
try {
- return jdbcOperator.existKey(key);
+ return jdbcRegistryClient.existJdbcRegistryDataKey(key);
} catch (Exception e) {
throw new RegistryException(String.format("Check key: %s exist error", key), e);
}
@@ -170,7 +230,7 @@ public final class JdbcRegistry implements Registry {
@Override
public boolean acquireLock(String key) {
try {
- registryLockManager.acquireLock(key);
+ jdbcRegistryClient.acquireJdbcRegistryLock(key);
return true;
} catch (RegistryException e) {
throw e;
@@ -182,7 +242,7 @@ public final class JdbcRegistry implements Registry {
@Override
public boolean acquireLock(String key, long timeout) {
try {
- return registryLockManager.acquireLock(key, timeout);
+ return jdbcRegistryClient.acquireJdbcRegistryLock(key, timeout);
} catch (RegistryException e) {
throw e;
} catch (Exception e) {
@@ -192,7 +252,7 @@ public final class JdbcRegistry implements Registry {
@Override
public boolean releaseLock(String key) {
- registryLockManager.releaseLock(key);
+ jdbcRegistryClient.releaseJdbcRegistryLock(key);
return true;
}
@@ -200,10 +260,7 @@ public final class JdbcRegistry implements Registry {
public void close() {
log.info("Closing Jdbc Registry...");
// remove the current Ephemeral node, if can connect to jdbc
- try (
- EphemeralDateManager closed1 = ephemeralDateManager;
- SubscribeDataManager close2 = subscribeDataManager;
- RegistryLockManager close3 = registryLockManager) {
+ try (JdbcRegistryClient closed1 = jdbcRegistryClient) {
} catch (Exception e) {
log.error("Close Jdbc Registry error", e);
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java
index 603a476322..0bdf2f15f2 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java
@@ -17,8 +17,16 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryClientHeartbeatMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryDataChanceEventMapper;
import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryDataMapper;
import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryLockMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryClientRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryServer;
import org.apache.ibatis.session.SqlSessionFactory;
@@ -50,15 +58,25 @@ public class JdbcRegistryAutoConfiguration {
}
@Bean
- public JdbcOperator jdbcOperator(JdbcRegistryProperties jdbcRegistryProperties,
- JdbcRegistryDataMapper jdbcRegistryDataMapper,
- JdbcRegistryLockMapper jdbcRegistryLockMapper) {
- return new JdbcOperator(jdbcRegistryProperties, jdbcRegistryDataMapper, jdbcRegistryLockMapper);
+ public IJdbcRegistryServer jdbcRegistryServer(JdbcRegistryDataRepository jdbcRegistryDataRepository,
+ JdbcRegistryLockRepository jdbcRegistryLockRepository,
+ JdbcRegistryClientRepository jdbcRegistryClientRepository,
+ JdbcRegistryDataChanceEventRepository jdbcRegistryDataChanceEventRepository,
+ JdbcRegistryProperties jdbcRegistryProperties) {
+ return new JdbcRegistryServer(
+ jdbcRegistryDataRepository,
+ jdbcRegistryLockRepository,
+ jdbcRegistryClientRepository,
+ jdbcRegistryDataChanceEventRepository,
+ jdbcRegistryProperties);
}
@Bean
- public JdbcRegistry jdbcRegistry(JdbcRegistryProperties jdbcRegistryProperties, JdbcOperator jdbcOperator) {
- return new JdbcRegistry(jdbcRegistryProperties, jdbcOperator);
+ public JdbcRegistry jdbcRegistry(JdbcRegistryProperties jdbcRegistryProperties,
+ IJdbcRegistryServer jdbcRegistryServer) {
+ JdbcRegistry jdbcRegistry = new JdbcRegistry(jdbcRegistryProperties, jdbcRegistryServer);
+ jdbcRegistry.start();
+ return jdbcRegistry;
}
@Bean
@@ -89,4 +107,16 @@ public class JdbcRegistryAutoConfiguration {
return jdbcRegistrySqlSessionTemplate.getMapper(JdbcRegistryLockMapper.class);
}
+ @Bean
+ public JdbcRegistryDataChanceEventMapper jdbcRegistryDataChanceEventMapper(SqlSessionTemplate jdbcRegistrySqlSessionTemplate) {
+ jdbcRegistrySqlSessionTemplate.getConfiguration().addMapper(JdbcRegistryDataChanceEventMapper.class);
+ return jdbcRegistrySqlSessionTemplate.getMapper(JdbcRegistryDataChanceEventMapper.class);
+ }
+
+ @Bean
+ public JdbcRegistryClientHeartbeatMapper jdbcRegistryClientHeartbeatMapper(SqlSessionTemplate jdbcRegistrySqlSessionTemplate) {
+ jdbcRegistrySqlSessionTemplate.getConfiguration().addMapper(JdbcRegistryClientHeartbeatMapper.class);
+ return jdbcRegistrySqlSessionTemplate.getMapper(JdbcRegistryClientHeartbeatMapper.class);
+ }
+
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryProperties.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryProperties.java
index eb8d5c8807..de98ec3184 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryProperties.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryProperties.java
@@ -17,31 +17,77 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
+import org.apache.dolphinscheduler.common.utils.NetUtils;
+
+import org.apache.commons.lang3.StringUtils;
+
import java.time.Duration;
import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Value;
import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
import org.springframework.boot.context.properties.ConfigurationProperties;
import org.springframework.context.annotation.Configuration;
+import org.springframework.validation.Errors;
+import org.springframework.validation.Validator;
+import org.springframework.validation.annotation.Validated;
import com.zaxxer.hikari.HikariConfig;
@Data
+@Slf4j
+@Validated
@Configuration
@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "jdbc")
@ConfigurationProperties(prefix = "registry")
-public class JdbcRegistryProperties {
-
- /**
- * Used to schedule refresh the ephemeral data/ lock.
- */
- private Duration termRefreshInterval = Duration.ofSeconds(2);
- /**
- * Used to calculate the expire time,
- * e.g. if you set 2, and latest two refresh error, then the ephemeral data/lock will be expire.
- */
- private int termExpireTimes = 3;
+public class JdbcRegistryProperties implements Validator {
+
+ private static final Duration MIN_HEARTBEAT_REFRESH_INTERVAL = Duration.ofSeconds(1);
+
+ @Value("${server.port:8080}")
+ private int serverPort;
+
+ private String jdbcRegistryClientName;
+
+ private Duration heartbeatRefreshInterval = Duration.ofSeconds(3);
+ private Duration sessionTimeout = Duration.ofSeconds(60);
private HikariConfig hikariConfig;
+ @Override
+ public boolean supports(Class> clazz) {
+ return JdbcRegistryProperties.class.isAssignableFrom(clazz);
+ }
+
+ @Override
+ public void validate(Object target, Errors errors) {
+ JdbcRegistryProperties jdbcRegistryProperties = (JdbcRegistryProperties) target;
+ if (jdbcRegistryProperties.getHeartbeatRefreshInterval().compareTo(MIN_HEARTBEAT_REFRESH_INTERVAL) < 0) {
+ errors.rejectValue("heartbeatRefreshInterval", "heartbeatRefreshInterval",
+ "heartbeatRefreshInterval must be greater than 1s");
+ }
+
+ if (jdbcRegistryProperties.getSessionTimeout().toMillis() < 3
+ * jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis()) {
+ errors.rejectValue("sessionTimeout", "sessionTimeout",
+ "sessionTimeout must be greater than 3 * heartbeatRefreshInterval");
+ }
+ if (StringUtils.isEmpty(jdbcRegistryClientName)) {
+ jdbcRegistryClientName = NetUtils.getHost() + ":" + serverPort;
+ }
+ print();
+
+ }
+
+ private void print() {
+ String config =
+ "\n****************************JdbcRegistryProperties**************************************" +
+ "\n jdbcRegistryClientName -> " + jdbcRegistryClientName +
+ "\n heartbeatRefreshInterval -> " + heartbeatRefreshInterval +
+ "\n sessionTimeout -> " + sessionTimeout +
+ "\n hikariConfig -> " + hikariConfig +
+ "\n****************************JdbcRegistryProperties**************************************";
+ log.info(config);
+ }
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactory.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactory.java
new file mode 100644
index 0000000000..92638b2959
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactory.java
@@ -0,0 +1,31 @@
+/*
+ * 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.registry.jdbc;
+
+import org.apache.dolphinscheduler.common.thread.ThreadUtils;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+public class JdbcRegistryThreadFactory {
+
+ public static ScheduledExecutorService getDefaultSchedulerThreadExecutor() {
+ return ThreadUtils.newDaemonScheduledExecutorService("jdbc-registry-default-scheduler-thread-pool",
+ Runtime.getRuntime().availableProcessors());
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/RegistryLockManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/RegistryLockManager.java
deleted file mode 100644
index 6c519685ff..0000000000
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/RegistryLockManager.java
+++ /dev/null
@@ -1,161 +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.registry.jdbc;
-
-import org.apache.dolphinscheduler.common.thread.ThreadUtils;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryLock;
-import org.apache.dolphinscheduler.registry.api.RegistryException;
-
-import java.sql.SQLException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.stream.Collectors;
-
-import lombok.AccessLevel;
-import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-@Slf4j
-class RegistryLockManager implements AutoCloseable {
-
- private final JdbcOperator jdbcOperator;
- private final JdbcRegistryProperties registryProperties;
- // lock owner -> lock
- private final Map lockHoldMap;
- private final ScheduledExecutorService lockTermUpdateThreadPool;
-
- RegistryLockManager(JdbcRegistryProperties registryProperties, JdbcOperator jdbcOperator) {
- this.registryProperties = registryProperties;
- this.jdbcOperator = jdbcOperator;
- this.lockHoldMap = new ConcurrentHashMap<>();
- this.lockTermUpdateThreadPool = Executors.newSingleThreadScheduledExecutor(
- new ThreadFactoryBuilder().setNameFormat("JdbcRegistryLockTermRefreshThread").setDaemon(true).build());
- }
-
- public void start() {
- lockTermUpdateThreadPool.scheduleWithFixedDelay(
- new LockTermRefreshTask(lockHoldMap, jdbcOperator),
- registryProperties.getTermRefreshInterval().toMillis(),
- registryProperties.getTermRefreshInterval().toMillis(),
- TimeUnit.MILLISECONDS);
- }
-
- /**
- * Acquire the lock, if cannot get the lock will await.
- */
- public void acquireLock(String lockKey) throws RegistryException {
- try {
- while (true) {
- JdbcRegistryLock jdbcRegistryLock = lockHoldMap.get(lockKey);
- if (jdbcRegistryLock != null && LockUtils.getLockOwner().equals(jdbcRegistryLock.getLockOwner())) {
- return;
- }
- jdbcRegistryLock = jdbcOperator.tryToAcquireLock(lockKey);
- if (jdbcRegistryLock != null) {
- lockHoldMap.put(lockKey, jdbcRegistryLock);
- return;
- }
- log.debug("Acquire the lock {} failed try again", lockKey);
- // acquire failed, wait and try again
- ThreadUtils.sleep(JdbcRegistryConstant.LOCK_ACQUIRE_INTERVAL);
- }
- } catch (Exception ex) {
- throw new RegistryException("Acquire the lock: " + lockKey + " error", ex);
- }
- }
-
- /**
- * Acquire the lock, if cannot get the lock will await.
- */
- public boolean acquireLock(String lockKey, long timeout) throws RegistryException {
- long startTime = System.currentTimeMillis();
- try {
- while (System.currentTimeMillis() - startTime < timeout) {
- JdbcRegistryLock jdbcRegistryLock = lockHoldMap.get(lockKey);
- if (jdbcRegistryLock != null && LockUtils.getLockOwner().equals(jdbcRegistryLock.getLockOwner())) {
- return true;
- }
- jdbcRegistryLock = jdbcOperator.tryToAcquireLock(lockKey);
- if (jdbcRegistryLock != null) {
- lockHoldMap.put(lockKey, jdbcRegistryLock);
- return true;
- }
- log.debug("Acquire the lock {} failed try again", lockKey);
- ThreadUtils.sleep(JdbcRegistryConstant.LOCK_ACQUIRE_INTERVAL);
- }
- } catch (Exception e) {
- throw new RegistryException("Acquire the lock: " + lockKey + " error", e);
- }
- return false;
- }
-
- public void releaseLock(String lockKey) {
- JdbcRegistryLock jdbcRegistryLock = lockHoldMap.get(lockKey);
- if (jdbcRegistryLock != null) {
- try {
- // the lock is unExit
- jdbcOperator.releaseLock(jdbcRegistryLock.getId());
- lockHoldMap.remove(lockKey);
- } catch (SQLException e) {
- lockHoldMap.remove(lockKey);
- throw new RegistryException(String.format("Release lock: %s error", lockKey), e);
- }
- }
- }
-
- @Override
- public void close() {
- lockTermUpdateThreadPool.shutdownNow();
- for (Map.Entry lockEntry : lockHoldMap.entrySet()) {
- releaseLock(lockEntry.getKey());
- }
- }
-
- /**
- * This task is used to refresh the lock held by the current server.
- */
- @RequiredArgsConstructor(access = AccessLevel.PRIVATE)
- static class LockTermRefreshTask implements Runnable {
-
- private final Map lockHoldMap;
- private final JdbcOperator jdbcOperator;
-
- public void run() {
- try {
- if (lockHoldMap.isEmpty()) {
- return;
- }
- List lockIds = lockHoldMap.values()
- .stream()
- .map(JdbcRegistryLock::getId)
- .collect(Collectors.toList());
- if (!jdbcOperator.updateLockTerm(lockIds)) {
- log.warn("Update the lock: {} term failed.", lockIds);
- }
- } catch (Exception e) {
- log.error("Update lock term error", e);
- }
- }
- }
-}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/SubscribeDataManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/SubscribeDataManager.java
deleted file mode 100644
index e86dc4b155..0000000000
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/SubscribeDataManager.java
+++ /dev/null
@@ -1,150 +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.registry.jdbc;
-
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryData;
-import org.apache.dolphinscheduler.registry.api.Event;
-import org.apache.dolphinscheduler.registry.api.SubscribeListener;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import lombok.RequiredArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-
-/**
- * Used to refresh if the subscribe path has been changed.
- */
-@Slf4j
-class SubscribeDataManager implements AutoCloseable {
-
- private final JdbcOperator jdbcOperator;
- private final JdbcRegistryProperties registryProperties;
- private final Map> dataSubScribeMap = new ConcurrentHashMap<>();
- private final ScheduledExecutorService dataSubscribeCheckThreadPool;
- private final Map jdbcRegistryDataMap = new ConcurrentHashMap<>();
-
- SubscribeDataManager(JdbcRegistryProperties registryProperties, JdbcOperator jdbcOperator) {
- this.registryProperties = registryProperties;
- this.jdbcOperator = jdbcOperator;
- this.dataSubscribeCheckThreadPool = Executors.newScheduledThreadPool(
- 1,
- new ThreadFactoryBuilder().setNameFormat("JdbcRegistrySubscribeDataCheckThread").setDaemon(true)
- .build());
- }
-
- public void start() {
- dataSubscribeCheckThreadPool.scheduleWithFixedDelay(
- new RegistrySubscribeDataCheckTask(dataSubScribeMap, jdbcOperator, jdbcRegistryDataMap),
- registryProperties.getTermRefreshInterval().toMillis(),
- registryProperties.getTermRefreshInterval().toMillis(),
- TimeUnit.MILLISECONDS);
- }
-
- public void addListener(String path, SubscribeListener subscribeListener) {
- dataSubScribeMap.computeIfAbsent(path, k -> new ArrayList<>()).add(subscribeListener);
- }
-
- public void removeListener(String path) {
- dataSubScribeMap.remove(path);
- }
-
- public JdbcRegistryData getData(String path) {
- return jdbcRegistryDataMap.get(path);
- }
-
- @Override
- public void close() {
- dataSubscribeCheckThreadPool.shutdownNow();
- dataSubScribeMap.clear();
- }
-
- @RequiredArgsConstructor
- static class RegistrySubscribeDataCheckTask implements Runnable {
-
- private final Map> dataSubScribeMap;
- private final JdbcOperator jdbcOperator;
- private final Map jdbcRegistryDataMap;
-
- @Override
- public void run() {
- // query the full data from database, and update the jdbcRegistryDataMap
- try {
- Map currentJdbcDataMap = jdbcOperator.queryAllJdbcRegistryData()
- .stream()
- .collect(Collectors.toMap(JdbcRegistryData::getDataKey, Function.identity()));
- // find the different
- List addedData = new ArrayList<>();
- List deletedData = new ArrayList<>();
- List updatedData = new ArrayList<>();
-
- for (Map.Entry entry : currentJdbcDataMap.entrySet()) {
- JdbcRegistryData newData = entry.getValue();
- JdbcRegistryData oldData = jdbcRegistryDataMap.get(entry.getKey());
- if (oldData == null) {
- addedData.add(newData);
- } else {
- if (!entry.getValue().getLastUpdateTime().equals(oldData.getLastUpdateTime())) {
- updatedData.add(newData);
- }
- }
- }
-
- for (Map.Entry entry : jdbcRegistryDataMap.entrySet()) {
- if (!currentJdbcDataMap.containsKey(entry.getKey())) {
- deletedData.add(entry.getValue());
- }
- }
- jdbcRegistryDataMap.clear();
- jdbcRegistryDataMap.putAll(currentJdbcDataMap);
- // trigger listener
- for (Map.Entry> entry : dataSubScribeMap.entrySet()) {
- String subscribeKey = entry.getKey();
- List subscribeListeners = entry.getValue();
- triggerListener(addedData, subscribeKey, subscribeListeners, Event.Type.ADD);
- triggerListener(deletedData, subscribeKey, subscribeListeners, Event.Type.REMOVE);
- triggerListener(updatedData, subscribeKey, subscribeListeners, Event.Type.UPDATE);
- }
- } catch (Exception e) {
- log.error("Query data from jdbc registry error");
- }
- }
-
- private void triggerListener(List dataList,
- String subscribeKey,
- List subscribeListeners,
- Event.Type type) {
- for (JdbcRegistryData data : dataList) {
- if (data.getDataKey().startsWith(subscribeKey)) {
- subscribeListeners.forEach(subscribeListener -> subscribeListener
- .notify(new Event(data.getDataKey(), data.getDataKey(), data.getDataValue(), type)));
- }
- }
- }
-
- }
-}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/IJdbcRegistryClient.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/IJdbcRegistryClient.java
new file mode 100644
index 0000000000..f517c2bf05
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/IJdbcRegistryClient.java
@@ -0,0 +1,106 @@
+/*
+ * 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.registry.jdbc.client;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryDataChangeListener;
+
+import java.util.List;
+import java.util.Optional;
+
+public interface IJdbcRegistryClient extends AutoCloseable {
+
+ /**
+ * Start the jdbc registry client, once started, the client will connect to the jdbc registry server, and then it can be used.
+ */
+ void start();
+
+ /**
+ * Get identify of the client.
+ */
+ JdbcRegistryClientIdentify getJdbcRegistryClientIdentify();
+
+ /**
+ * Check the connectivity of the client.
+ */
+ boolean isConnectivity();
+
+ /**
+ * Subscribe the jdbc registry connection state change event.
+ */
+ void subscribeConnectionStateChange(ConnectionStateListener connectionStateListener);
+
+ /**
+ * Subscribe the {@link JdbcRegistryDataDTO} change event.
+ */
+ void subscribeJdbcRegistryDataChange(JdbcRegistryDataChangeListener jdbcRegistryDataChangeListener);
+
+ /**
+ * Get the {@link JdbcRegistryDataDTO} by key.
+ */
+ Optional getJdbcRegistryDataByKey(String key);
+
+ /**
+ * Put the {@link JdbcRegistryDataDTO} to the jdbc registry server.
+ *
+ * If the key is already exist, then update the {@link JdbcRegistryDataDTO}. If the key is not exist, then insert a new {@link JdbcRegistryDataDTO}.
+ */
+ void putJdbcRegistryData(String key, String value, DataType dataType);
+
+ /**
+ * Delete the {@link JdbcRegistryDataDTO} by key.
+ */
+ void deleteJdbcRegistryDataByKey(String key);
+
+ /**
+ * List all the {@link JdbcRegistryDataDTO} children by key.
+ *
+ * e.g. key = "/dolphinscheduler/master", and data exist in db is "/dolphinscheduler/master/master1", "/dolphinscheduler/master/master2"
+ *
+ * then the return value will be ["master1", "master2"]
+ */
+ List listJdbcRegistryDataChildren(String key);
+
+ /**
+ * Check the key exist in the jdbc registry server.
+ */
+ boolean existJdbcRegistryDataKey(String key);
+
+ /**
+ * Acquire the jdbc registry lock by key. this is a blocking method. if you want to stop the blocking, you can use interrupt the thread.
+ */
+ void acquireJdbcRegistryLock(String lockKey) throws IllegalArgumentException;
+
+ /**
+ * Acquire the jdbc registry lock by key until timeout.
+ */
+ boolean acquireJdbcRegistryLock(String lockKey, long timeout);
+
+ /**
+ * Release the jdbc registry lock by key, if the lockKey is not exist will do nothing.
+ */
+ void releaseJdbcRegistryLock(String lockKey);
+
+ /**
+ * Close the jdbc registry client, once the client been closed, it cannot work anymore.
+ */
+ @Override
+ void close();
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClient.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClient.java
new file mode 100644
index 0000000000..b00424c8e4
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClient.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.registry.jdbc.client;
+
+import org.apache.dolphinscheduler.common.utils.CodeGenerateUtils;
+import org.apache.dolphinscheduler.common.utils.NetUtils;
+import org.apache.dolphinscheduler.common.utils.OSUtils;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryDataChangeListener;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryServerState;
+
+import java.util.List;
+import java.util.Optional;
+
+import lombok.extern.slf4j.Slf4j;
+
+/**
+ * The client of jdbc registry, used to interact with the {@link org.apache.dolphinscheduler.plugin.registry.jdbc.server.JdbcRegistryServer}.
+ */
+@Slf4j
+public class JdbcRegistryClient implements IJdbcRegistryClient {
+
+ private static final String DEFAULT_CLIENT_NAME = NetUtils.getHost() + "_" + OSUtils.getProcessID();
+
+ private final JdbcRegistryProperties jdbcRegistryProperties;
+
+ private final JdbcRegistryClientIdentify jdbcRegistryClientIdentify;
+
+ private final IJdbcRegistryServer jdbcRegistryServer;
+
+ public JdbcRegistryClient(JdbcRegistryProperties jdbcRegistryProperties, IJdbcRegistryServer jdbcRegistryServer) {
+ this.jdbcRegistryProperties = jdbcRegistryProperties;
+ this.jdbcRegistryServer = jdbcRegistryServer;
+ this.jdbcRegistryClientIdentify =
+ new JdbcRegistryClientIdentify(CodeGenerateUtils.genCode(), DEFAULT_CLIENT_NAME);
+ }
+
+ @Override
+ public void start() {
+ jdbcRegistryServer.registerClient(this);
+ }
+
+ @Override
+ public JdbcRegistryClientIdentify getJdbcRegistryClientIdentify() {
+ return jdbcRegistryClientIdentify;
+ }
+
+ @Override
+ public void subscribeConnectionStateChange(ConnectionStateListener connectionStateListener) {
+ jdbcRegistryServer.subscribeConnectionStateChange(connectionStateListener);
+ }
+
+ @Override
+ public void subscribeJdbcRegistryDataChange(JdbcRegistryDataChangeListener jdbcRegistryDataChangeListener) {
+ jdbcRegistryServer.subscribeJdbcRegistryDataChange(jdbcRegistryDataChangeListener);
+ }
+
+ @Override
+ public Optional getJdbcRegistryDataByKey(String key) {
+ return jdbcRegistryServer.getJdbcRegistryDataByKey(key);
+ }
+
+ @Override
+ public void putJdbcRegistryData(String key, String value, DataType dataType) {
+ jdbcRegistryServer.putJdbcRegistryData(jdbcRegistryClientIdentify.getClientId(), key, value, dataType);
+ }
+
+ @Override
+ public void deleteJdbcRegistryDataByKey(String key) {
+ jdbcRegistryServer.deleteJdbcRegistryDataByKey(key);
+ }
+
+ @Override
+ public List listJdbcRegistryDataChildren(String key) {
+ return jdbcRegistryServer.listJdbcRegistryDataChildren(key);
+ }
+
+ @Override
+ public boolean existJdbcRegistryDataKey(String key) {
+ return jdbcRegistryServer.existJdbcRegistryDataKey(key);
+ }
+
+ @Override
+ public void acquireJdbcRegistryLock(String key) {
+ jdbcRegistryServer.acquireJdbcRegistryLock(jdbcRegistryClientIdentify.getClientId(), key);
+ }
+
+ @Override
+ public boolean acquireJdbcRegistryLock(String key, long timeout) {
+ return jdbcRegistryServer.acquireJdbcRegistryLock(jdbcRegistryClientIdentify.getClientId(), key, timeout);
+ }
+
+ @Override
+ public void releaseJdbcRegistryLock(String key) {
+ jdbcRegistryServer.releaseJdbcRegistryLock(jdbcRegistryClientIdentify.getClientId(), key);
+ }
+
+ @Override
+ public void close() {
+ jdbcRegistryServer.deregisterClient(this);
+ log.info("Closed JdbcRegistryClient: {}", jdbcRegistryClientIdentify);
+ }
+
+ @Override
+ public boolean isConnectivity() {
+ return jdbcRegistryServer.getServerState() == JdbcRegistryServerState.STARTED;
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClientIdentify.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClientIdentify.java
new file mode 100644
index 0000000000..b6c8e0986e
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/client/JdbcRegistryClientIdentify.java
@@ -0,0 +1,33 @@
+/*
+ * 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.registry.jdbc.client;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+import lombok.ToString;
+
+@ToString
+@Getter
+@AllArgsConstructor
+public class JdbcRegistryClientIdentify {
+
+ private final Long clientId;
+
+ private final String clientName;
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryClientHeartbeatMapper.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryClientHeartbeatMapper.java
new file mode 100644
index 0000000000..2b8499bb48
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryClientHeartbeatMapper.java
@@ -0,0 +1,33 @@
+/*
+ * 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.registry.jdbc.mapper;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+
+import org.apache.ibatis.annotations.Select;
+
+import java.util.List;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+public interface JdbcRegistryClientHeartbeatMapper extends BaseMapper {
+
+ @Select("select * from t_ds_jdbc_registry_client_heartbeat")
+ List selectAll();
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataChanceEventMapper.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataChanceEventMapper.java
new file mode 100644
index 0000000000..0ef1513ea0
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataChanceEventMapper.java
@@ -0,0 +1,41 @@
+/*
+ * 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.registry.jdbc.mapper;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryDataChanceEvent;
+
+import org.apache.ibatis.annotations.Delete;
+import org.apache.ibatis.annotations.Param;
+import org.apache.ibatis.annotations.Select;
+
+import java.util.Date;
+import java.util.List;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+public interface JdbcRegistryDataChanceEventMapper extends BaseMapper {
+
+ @Select("select max(id) from t_ds_jdbc_registry_data_change_event")
+ Long getMaxId();
+
+ @Select("select * from t_ds_jdbc_registry_data_change_event where id > #{id} order by id asc limit 1000")
+ List selectJdbcRegistryDataChangeEventWhereIdAfter(@Param("id") long id);
+
+ @Delete("delete from t_ds_jdbc_registry_data_change_event where create_time > #{createTime}")
+ void deleteJdbcRegistryDataChangeEventBeforeCreateTime(@Param("createTime") Date createTime);
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java
index e1d27bbf0b..261bfae4fe 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java
@@ -17,14 +17,12 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc.mapper;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryData;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryData;
import org.apache.ibatis.annotations.Delete;
import org.apache.ibatis.annotations.Param;
import org.apache.ibatis.annotations.Select;
-import org.apache.ibatis.annotations.Update;
-import java.util.Collection;
import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
@@ -37,22 +35,17 @@ public interface JdbcRegistryDataMapper extends BaseMapper {
@Select("select * from t_ds_jdbc_registry_data where data_key = #{key}")
JdbcRegistryData selectByKey(@Param("key") String key);
- @Select("select * from t_ds_jdbc_registry_data where data_key like CONCAT (#{key}, '%')")
- List fuzzyQueryByKey(@Param("key") String key);
-
@Delete("delete from t_ds_jdbc_registry_data where data_key = #{key}")
void deleteByKey(@Param("key") String key);
- @Delete("delete from t_ds_jdbc_registry_data where last_term < #{term} and data_type = #{type}")
- void clearExpireEphemeralDate(@Param("term") long term, @Param("type") int type);
-
- @Update({""})
- int updateTermByIds(@Param("ids") Collection ids, @Param("term") long term);
+ void deleteByClientIds(@Param("clientIds") List clientIds, @Param("dataType") String dataType);
+
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java
index 2d11c90a24..0639dfbc20 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java
@@ -17,32 +17,23 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc.mapper;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryLock;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
import org.apache.ibatis.annotations.Delete;
import org.apache.ibatis.annotations.Param;
-import org.apache.ibatis.annotations.Select;
-import org.apache.ibatis.annotations.Update;
-import java.util.Collection;
+import java.util.List;
import com.baomidou.mybatisplus.core.mapper.BaseMapper;
public interface JdbcRegistryLockMapper extends BaseMapper {
- @Select("select count(1) from t_ds_jdbc_registry_lock")
- int countAll();
-
- @Delete("delete from t_ds_jdbc_registry_lock where last_term < #{term}")
- void clearExpireLock(@Param("term") long term);
-
- @Update({""})
- int updateTermByIds(@Param("ids") Collection ids, @Param("term") long term);
+ void deleteByClientIds(@Param("clientIds") List clientIds);
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java
new file mode 100644
index 0000000000..cce8a488a3
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java
@@ -0,0 +1,45 @@
+/*
+ * 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.registry.jdbc.model.DO;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+
+@TableName(value = "t_ds_jdbc_registry_client_heartbeat")
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryClientHeartbeat {
+
+ @TableId(value = "id", type = IdType.INPUT)
+ private Long id;
+ private String clientName;
+ private Long lastHeartbeatTime;
+ private String connectionConfig;
+ private Date createTime;
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryData.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryData.java
similarity index 92%
rename from dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryData.java
rename to dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryData.java
index 565e613151..44ba0e620f 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryData.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryData.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.plugin.registry.jdbc.model;
+package org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO;
import java.util.Date;
@@ -39,8 +39,8 @@ public class JdbcRegistryData {
private Long id;
private String dataKey;
private String dataValue;
- private int dataType;
- private long lastTerm;
+ private String dataType;
+ private long clientId;
private Date createTime;
private Date lastUpdateTime;
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.java
new file mode 100644
index 0000000000..860eff3267
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.java
@@ -0,0 +1,47 @@
+/*
+ * 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.registry.jdbc.model.DO;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+
+@TableName(value = "t_ds_jdbc_registry_data_change_event")
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryDataChanceEvent {
+
+ @TableId(value = "id", type = IdType.AUTO)
+ private Long id;
+
+ private String eventType;
+
+ private String jdbcRegistryData;
+
+ private Date createTime;
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryLock.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryLock.java
similarity index 77%
rename from dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryLock.java
rename to dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryLock.java
index 17f840c37f..2b4a84fe7c 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/JdbcRegistryLock.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DO/JdbcRegistryLock.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.plugin.registry.jdbc.model;
+package org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO;
import java.util.Date;
@@ -37,24 +37,8 @@ public class JdbcRegistryLock {
@TableId(value = "id", type = IdType.AUTO)
private Long id;
- /**
- * The lock key.
- */
private String lockKey;
- /**
- * acquire lock host.
- */
private String lockOwner;
- /**
- * The last term, if the (currentTime - lastTerm) > termExpire time, the lock will be expired.
- */
- private Long lastTerm;
- /**
- * The lock last update time.
- */
- private Date lastUpdateTime;
- /**
- * The lock create time.
- */
+ private Long clientId;
private Date createTime;
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DataType.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/DataType.java
similarity index 75%
rename from dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DataType.java
rename to dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/DataType.java
index 88cc5415eb..7b7e9a8515 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DataType.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/DataType.java
@@ -15,20 +15,11 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.plugin.registry.jdbc.model;
+package org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO;
public enum DataType {
- EPHEMERAL(1),
- PERSISTENT(2),
+ EPHEMERAL(),
+ PERSISTENT(),
;
- private final int typeValue;
-
- DataType(int typeValue) {
- this.typeValue = typeValue;
- }
-
- public int getTypeValue() {
- return typeValue;
- }
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryClientHeartbeatDTO.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryClientHeartbeatDTO.java
new file mode 100644
index 0000000000..1f006291f5
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryClientHeartbeatDTO.java
@@ -0,0 +1,97 @@
+/*
+ * 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.registry.jdbc.model.DTO;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.SneakyThrows;
+
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryClientHeartbeatDTO {
+
+ private Long id;
+
+ // clientName
+ private String clientName;
+
+ private Long lastHeartbeatTime;
+
+ private ClientConfig clientConfig;
+
+ private Date createTime;
+
+ public static JdbcRegistryClientHeartbeatDTO fromJdbcRegistryClientHeartbeat(JdbcRegistryClientHeartbeat jdbcRegistryClientHeartbeat) {
+ return JdbcRegistryClientHeartbeatDTO.builder()
+ .id(jdbcRegistryClientHeartbeat.getId())
+ .clientName(jdbcRegistryClientHeartbeat.getClientName())
+ .lastHeartbeatTime(jdbcRegistryClientHeartbeat.getLastHeartbeatTime())
+ .clientConfig(
+ JSONUtils.parseObject(jdbcRegistryClientHeartbeat.getConnectionConfig(), ClientConfig.class))
+ .createTime(jdbcRegistryClientHeartbeat.getCreateTime())
+ .build();
+ }
+
+ public static JdbcRegistryClientHeartbeat toJdbcRegistryClientHeartbeat(JdbcRegistryClientHeartbeatDTO jdbcRegistryClientHeartbeatDTO) {
+ return JdbcRegistryClientHeartbeat.builder()
+ .id(jdbcRegistryClientHeartbeatDTO.getId())
+ .clientName(jdbcRegistryClientHeartbeatDTO.getClientName())
+ .lastHeartbeatTime(jdbcRegistryClientHeartbeatDTO.getLastHeartbeatTime())
+ .connectionConfig(JSONUtils.toJsonString(jdbcRegistryClientHeartbeatDTO.getClientConfig()))
+ .createTime(jdbcRegistryClientHeartbeatDTO.getCreateTime())
+ .build();
+ }
+
+ public boolean isDead() {
+ // check if the client connection is expired.
+ return System.currentTimeMillis() - lastHeartbeatTime > clientConfig.getSessionTimeout();
+ }
+
+ @SneakyThrows
+ @Override
+ public JdbcRegistryClientHeartbeatDTO clone() {
+ return JdbcRegistryClientHeartbeatDTO.builder()
+ .id(id)
+ .clientName(clientName)
+ .lastHeartbeatTime(lastHeartbeatTime)
+ .clientConfig(clientConfig)
+ .createTime(createTime)
+ .build();
+ }
+
+ @Data
+ @Builder
+ @NoArgsConstructor
+ @AllArgsConstructor
+ public static class ClientConfig {
+
+ @Builder.Default
+ private long sessionTimeout = 60 * 1000L;
+
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataChanceEventDTO.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataChanceEventDTO.java
new file mode 100644
index 0000000000..5735505c28
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataChanceEventDTO.java
@@ -0,0 +1,76 @@
+/*
+ * 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.registry.jdbc.model.DTO;
+
+import org.apache.dolphinscheduler.common.utils.JSONUtils;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryDataChanceEvent;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryDataChanceEventDTO {
+
+ private Long id;
+
+ private EventType eventType;
+
+ private JdbcRegistryDataDTO jdbcRegistryData;
+
+ private Date createTime;
+
+ public enum EventType {
+ ADD,
+ UPDATE,
+ DELETE;
+
+ }
+
+ public static JdbcRegistryDataChanceEventDTO fromJdbcRegistryDataChanceEvent(JdbcRegistryDataChanceEvent jdbcRegistryDataChanceEvent) {
+ JdbcRegistryData jdbcRegistryData =
+ JSONUtils.parseObject(jdbcRegistryDataChanceEvent.getJdbcRegistryData(), JdbcRegistryData.class);
+ if (jdbcRegistryData == null) {
+ throw new IllegalArgumentException(
+ "jdbcRegistryData: " + jdbcRegistryDataChanceEvent.getJdbcRegistryData() + " is invalidated");
+ }
+ return JdbcRegistryDataChanceEventDTO.builder()
+ .id(jdbcRegistryDataChanceEvent.getId())
+ .jdbcRegistryData(JdbcRegistryDataDTO.fromJdbcRegistryData(jdbcRegistryData))
+ .eventType(EventType.valueOf(jdbcRegistryDataChanceEvent.getEventType()))
+ .createTime(jdbcRegistryDataChanceEvent.getCreateTime())
+ .build();
+ }
+
+ public static JdbcRegistryDataChanceEvent toJdbcRegistryDataChanceEvent(JdbcRegistryDataChanceEventDTO jdbcRegistryDataChanceEvent) {
+ return JdbcRegistryDataChanceEvent.builder()
+ .id(jdbcRegistryDataChanceEvent.getId())
+ .jdbcRegistryData(JSONUtils.toJsonString(jdbcRegistryDataChanceEvent.getJdbcRegistryData()))
+ .eventType(jdbcRegistryDataChanceEvent.getEventType().name())
+ .createTime(jdbcRegistryDataChanceEvent.getCreateTime())
+ .build();
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataDTO.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataDTO.java
new file mode 100644
index 0000000000..a017faa261
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryDataDTO.java
@@ -0,0 +1,67 @@
+/*
+ * 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.registry.jdbc.model.DTO;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryDataDTO {
+
+ private Long id;
+ private String dataKey;
+ private String dataValue;
+ private String dataType;
+ private Long clientId;
+ private Date createTime;
+ private Date lastUpdateTime;
+
+ public static JdbcRegistryDataDTO fromJdbcRegistryData(JdbcRegistryData jdbcRegistryData) {
+ return JdbcRegistryDataDTO.builder()
+ .id(jdbcRegistryData.getId())
+ .dataKey(jdbcRegistryData.getDataKey())
+ .dataValue(jdbcRegistryData.getDataValue())
+ .dataType(jdbcRegistryData.getDataType())
+ .clientId(jdbcRegistryData.getClientId())
+ .createTime(jdbcRegistryData.getCreateTime())
+ .lastUpdateTime(jdbcRegistryData.getLastUpdateTime())
+ .build();
+ }
+
+ public static JdbcRegistryData toJdbcRegistryData(JdbcRegistryDataDTO jdbcRegistryData) {
+ return JdbcRegistryData.builder()
+ .id(jdbcRegistryData.getId())
+ .dataKey(jdbcRegistryData.getDataKey())
+ .dataValue(jdbcRegistryData.getDataValue())
+ .dataType(jdbcRegistryData.getDataType())
+ .clientId(jdbcRegistryData.getClientId())
+ .createTime(jdbcRegistryData.getCreateTime())
+ .lastUpdateTime(jdbcRegistryData.getLastUpdateTime())
+ .build();
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryLockDTO.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryLockDTO.java
new file mode 100644
index 0000000000..59b3c871c2
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/model/DTO/JdbcRegistryLockDTO.java
@@ -0,0 +1,61 @@
+/*
+ * 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.registry.jdbc.model.DTO;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
+
+import java.util.Date;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryLockDTO {
+
+ private Long id;
+ private String lockKey;
+ private String lockOwner;
+ private Long clientId;
+ private Date createTime;
+
+ public static JdbcRegistryLockDTO fromJdbcRegistryLock(JdbcRegistryLock jdbcRegistryLock) {
+ return JdbcRegistryLockDTO.builder()
+ .id(jdbcRegistryLock.getId())
+ .lockKey(jdbcRegistryLock.getLockKey())
+ .lockOwner(jdbcRegistryLock.getLockOwner())
+ .clientId(jdbcRegistryLock.getClientId())
+ .createTime(jdbcRegistryLock.getCreateTime())
+ .build();
+ }
+
+ public static JdbcRegistryLock toJdbcRegistryLock(JdbcRegistryLockDTO jdbcRegistryLock) {
+ return JdbcRegistryLock.builder()
+ .id(jdbcRegistryLock.getId())
+ .lockKey(jdbcRegistryLock.getLockKey())
+ .lockOwner(jdbcRegistryLock.getLockOwner())
+ .clientId(jdbcRegistryLock.getClientId())
+ .createTime(jdbcRegistryLock.getCreateTime())
+ .build();
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java
new file mode 100644
index 0000000000..f9ae2c6ef1
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java
@@ -0,0 +1,67 @@
+/*
+ * 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.registry.jdbc.repository;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryClientHeartbeatMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryClientHeartbeatDTO;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+
+@Repository
+public class JdbcRegistryClientRepository {
+
+ @Autowired
+ private JdbcRegistryClientHeartbeatMapper jdbcRegistryClientHeartbeatMapper;
+
+ public List queryAll() {
+ return jdbcRegistryClientHeartbeatMapper.selectAll()
+ .stream()
+ .map(JdbcRegistryClientHeartbeatDTO::fromJdbcRegistryClientHeartbeat)
+ .collect(Collectors.toList());
+ }
+
+ public void deleteByIds(List clientIds) {
+ if (CollectionUtils.isEmpty(clientIds)) {
+ return;
+ }
+ jdbcRegistryClientHeartbeatMapper.deleteBatchIds(clientIds);
+ }
+
+ public boolean updateById(JdbcRegistryClientHeartbeatDTO jdbcRegistryClientHeartbeatDTO) {
+ JdbcRegistryClientHeartbeat jdbcRegistryClientHeartbeat =
+ JdbcRegistryClientHeartbeatDTO.toJdbcRegistryClientHeartbeat(jdbcRegistryClientHeartbeatDTO);
+ return jdbcRegistryClientHeartbeatMapper.updateById(jdbcRegistryClientHeartbeat) == 1;
+ }
+
+ public void insert(JdbcRegistryClientHeartbeatDTO jdbcRegistryClient) {
+ checkNotNull(jdbcRegistryClient.getId());
+ JdbcRegistryClientHeartbeat jdbcRegistryClientHeartbeat =
+ JdbcRegistryClientHeartbeatDTO.toJdbcRegistryClientHeartbeat(jdbcRegistryClient);
+ jdbcRegistryClientHeartbeatMapper.insert(jdbcRegistryClientHeartbeat);
+
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataChanceEventRepository.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataChanceEventRepository.java
new file mode 100644
index 0000000000..ed30128bb3
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataChanceEventRepository.java
@@ -0,0 +1,65 @@
+/*
+ * 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.registry.jdbc.repository;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryDataChanceEventMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryDataChanceEvent;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataChanceEventDTO;
+
+import java.util.Date;
+import java.util.List;
+import java.util.stream.Collectors;
+
+import org.springframework.stereotype.Repository;
+
+@Repository
+public class JdbcRegistryDataChanceEventRepository {
+
+ private final JdbcRegistryDataChanceEventMapper jdbcRegistryDataChanceEventMapper;
+
+ public JdbcRegistryDataChanceEventRepository(JdbcRegistryDataChanceEventMapper jdbcRegistryDataChanceEventMapper) {
+ this.jdbcRegistryDataChanceEventMapper = jdbcRegistryDataChanceEventMapper;
+ }
+
+ public long getMaxJdbcRegistryDataChanceEventId() {
+ Long maxId = jdbcRegistryDataChanceEventMapper.getMaxId();
+ if (maxId == null) {
+ return -1;
+ } else {
+ return maxId;
+ }
+ }
+
+ public List selectJdbcRegistryDataChangeEventWhereIdAfter(long id) {
+ return jdbcRegistryDataChanceEventMapper.selectJdbcRegistryDataChangeEventWhereIdAfter(id)
+ .stream()
+ .map(JdbcRegistryDataChanceEventDTO::fromJdbcRegistryDataChanceEvent)
+ .collect(Collectors.toList());
+ }
+
+ public void insert(JdbcRegistryDataChanceEventDTO registryDataChanceEvent) {
+ JdbcRegistryDataChanceEvent jdbcRegistryDataChanceEvent =
+ JdbcRegistryDataChanceEventDTO.toJdbcRegistryDataChanceEvent(registryDataChanceEvent);
+ jdbcRegistryDataChanceEventMapper.insert(jdbcRegistryDataChanceEvent);
+ registryDataChanceEvent.setId(jdbcRegistryDataChanceEvent.getId());
+ }
+
+ public void deleteJdbcRegistryDataChangeEventBeforeCreateTime(Date createTime) {
+ jdbcRegistryDataChanceEventMapper.deleteJdbcRegistryDataChangeEventBeforeCreateTime(createTime);
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java
new file mode 100644
index 0000000000..1a0040db6a
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java
@@ -0,0 +1,73 @@
+/*
+ * 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.registry.jdbc.repository;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryDataMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+
+@Repository
+public class JdbcRegistryDataRepository {
+
+ @Autowired
+ private JdbcRegistryDataMapper jdbcRegistryDataMapper;
+
+ public List selectAll() {
+ return jdbcRegistryDataMapper
+ .selectAll()
+ .stream()
+ .map(JdbcRegistryDataDTO::fromJdbcRegistryData)
+ .collect(Collectors.toList());
+ }
+
+ public Optional selectByKey(String key) {
+ return Optional.ofNullable(jdbcRegistryDataMapper.selectByKey(key))
+ .map(JdbcRegistryDataDTO::fromJdbcRegistryData);
+ }
+
+ public void deleteEphemeralDateByClientIds(List clientIds) {
+ if (CollectionUtils.isEmpty(clientIds)) {
+ return;
+ }
+ jdbcRegistryDataMapper.deleteByClientIds(clientIds, DataType.EPHEMERAL.name());
+ }
+
+ public void deleteByKey(String key) {
+ jdbcRegistryDataMapper.deleteByKey(key);
+ }
+
+ public void insert(JdbcRegistryDataDTO jdbcRegistryData) {
+ JdbcRegistryData jdbcRegistryDataDO = JdbcRegistryDataDTO.toJdbcRegistryData(jdbcRegistryData);
+ jdbcRegistryDataMapper.insert(jdbcRegistryDataDO);
+ jdbcRegistryData.setId(jdbcRegistryDataDO.getId());
+ }
+
+ public void updateById(JdbcRegistryDataDTO jdbcRegistryDataDTO) {
+ jdbcRegistryDataMapper.updateById(JdbcRegistryDataDTO.toJdbcRegistryData(jdbcRegistryDataDTO));
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.java
new file mode 100644
index 0000000000..ef7e23fbbf
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.java
@@ -0,0 +1,56 @@
+/*
+ * 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.registry.jdbc.repository;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.mapper.JdbcRegistryLockMapper;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryLockDTO;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.List;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+
+@Repository
+public class JdbcRegistryLockRepository {
+
+ @Autowired
+ private JdbcRegistryLockMapper jdbcRegistryLockMapper;
+
+ public void deleteByClientIds(List clientIds) {
+ if (CollectionUtils.isEmpty(clientIds)) {
+ return;
+ }
+ jdbcRegistryLockMapper.deleteByClientIds(clientIds);
+ }
+
+ public void insert(JdbcRegistryLockDTO jdbcRegistryLock) {
+ checkNotNull(jdbcRegistryLock);
+ JdbcRegistryLock jdbcRegistryLockDO = JdbcRegistryLockDTO.toJdbcRegistryLock(jdbcRegistryLock);
+ jdbcRegistryLockMapper.insert(jdbcRegistryLockDO);
+ jdbcRegistryLock.setId(jdbcRegistryLockDO.getId());
+ }
+
+ public void deleteById(Long id) {
+ jdbcRegistryLockMapper.deleteById(id);
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryConstant.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/ConnectionStateListener.java
similarity index 79%
rename from dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryConstant.java
rename to dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/ConnectionStateListener.java
index 84496afb80..3d2397f4f5 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryConstant.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/ConnectionStateListener.java
@@ -15,13 +15,13 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.plugin.registry.jdbc;
+package org.apache.dolphinscheduler.plugin.registry.jdbc.server;
-import lombok.experimental.UtilityClass;
+public interface ConnectionStateListener {
-@UtilityClass
-final class JdbcRegistryConstant {
+ void onConnected();
- public static final long LOCK_ACQUIRE_INTERVAL = 1_000;
+ void onDisConnected();
+ void onReconnected();
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryDataManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryDataManager.java
new file mode 100644
index 0000000000..1c12f8cd8e
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryDataManager.java
@@ -0,0 +1,55 @@
+/*
+ * 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.registry.jdbc.server;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+
+import java.util.List;
+import java.util.Optional;
+
+public interface IJdbcRegistryDataManager {
+
+ boolean existKey(String key);
+
+ /**
+ * Get the {@link JdbcRegistryDataDTO} by key.
+ */
+ Optional getRegistryDataByKey(String key);
+
+ /**
+ * List all the {@link JdbcRegistryDataDTO} children by key.
+ *
+ * e.g. key = "/dolphinscheduler/master", and data exist in db is "/dolphinscheduler/master/master1", "/dolphinscheduler/master/master2"
+ *
+ * then the return value will be ["master1", "master2"]
+ */
+ List listJdbcRegistryDataChildren(String key);
+
+ /**
+ * Put the {@link JdbcRegistryDataDTO} to the jdbc registry server.
+ *
+ * If the key is already exist, then update the {@link JdbcRegistryDataDTO}. If the key is not exist, then insert a new {@link JdbcRegistryDataDTO}.
+ */
+ void putJdbcRegistryData(Long clientId, String key, String value, DataType dataType);
+
+ /**
+ * Delete the {@link JdbcRegistryDataDTO} by key.
+ */
+ void deleteJdbcRegistryDataByKey(String key);
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryLockManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryLockManager.java
new file mode 100644
index 0000000000..aebb5955bc
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryLockManager.java
@@ -0,0 +1,37 @@
+/*
+ * 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.registry.jdbc.server;
+
+public interface IJdbcRegistryLockManager {
+
+ /**
+ * Acquire the jdbc registry lock by key. this is a blocking method. if you want to stop the blocking, you can use interrupt the thread.
+ */
+ void acquireJdbcRegistryLock(Long clientId, String lockKey) throws InterruptedException;
+
+ /**
+ * Acquire the jdbc registry lock by key until timeout.
+ */
+ boolean acquireJdbcRegistryLock(Long clientId, String lockKey, long timeout);
+
+ /**
+ * Release the jdbc registry lock by key, if the lockKey is not exist will do nothing.
+ */
+ void releaseJdbcRegistryLock(Long clientId, String lockKey);
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryServer.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryServer.java
new file mode 100644
index 0000000000..0bd88f5683
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IJdbcRegistryServer.java
@@ -0,0 +1,110 @@
+/*
+ * 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.registry.jdbc.server;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.client.IJdbcRegistryClient;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The JdbcRegistryServer is represent the server side of the jdbc registry, it can be thought as db server.
+ */
+public interface IJdbcRegistryServer extends AutoCloseable {
+
+ void start();
+
+ /**
+ * Register a client to the server, once the client connect to the server then the server will refresh the client's term interval.
+ */
+ void registerClient(IJdbcRegistryClient jdbcRegistryClient);
+
+ /**
+ * Deregister a client to the server, once the client id deregister, then the server will deleted the data related to the client and stop refresh the client's term.
+ */
+ void deregisterClient(IJdbcRegistryClient jdbcRegistryClient);
+
+ /**
+ * Get the {@link JdbcRegistryServerState}
+ */
+ JdbcRegistryServerState getServerState();
+
+ /**
+ * Subscribe the jdbc registry connection state change
+ */
+ void subscribeConnectionStateChange(ConnectionStateListener connectionStateListener);
+
+ /**
+ * Subscribe the {@link org.apache.dolphinscheduler.plugin.registry.jdbc.model.DO.JdbcRegistryData} change.
+ */
+ void subscribeJdbcRegistryDataChange(JdbcRegistryDataChangeListener jdbcRegistryDataChangeListener);
+
+ /**
+ * Check the jdbc registry data key is exist or not.
+ */
+ boolean existJdbcRegistryDataKey(String key);
+
+ /**
+ * Get the {@link JdbcRegistryDataDTO} by key.
+ */
+ Optional getJdbcRegistryDataByKey(String key);
+
+ /**
+ * List all the {@link JdbcRegistryDataDTO} children by key.
+ *
+ * e.g. key = "/dolphinscheduler/master", and data exist in db is "/dolphinscheduler/master/master1", "/dolphinscheduler/master/master2"
+ *
+ * then the return value will be ["master1", "master2"]
+ */
+ List listJdbcRegistryDataChildren(String key);
+
+ /**
+ * Put the {@link JdbcRegistryDataDTO} to the jdbc registry server.
+ *
+ * If the key is already exist, then update the {@link JdbcRegistryDataDTO}. If the key is not exist, then insert a new {@link JdbcRegistryDataDTO}.
+ */
+ void putJdbcRegistryData(Long clientId, String key, String value, DataType dataType);
+
+ /**
+ * Delete the {@link JdbcRegistryDataDTO} by key.
+ */
+ void deleteJdbcRegistryDataByKey(String key);
+
+ /**
+ * Acquire the jdbc registry lock by key. this is a blocking method. if you want to stop the blocking, you can use interrupt the thread.
+ */
+ void acquireJdbcRegistryLock(Long clientId, String key);
+
+ /**
+ * Acquire the jdbc registry lock by key until timeout.
+ */
+ boolean acquireJdbcRegistryLock(Long clientId, String key, long timeout);
+
+ /**
+ * Release the jdbc registry lock by key, if the lockKey is not exist will do nothing.
+ */
+ void releaseJdbcRegistryLock(Long clientId, String key);
+
+ /**
+ * Close the server, once the server been closed, it cannot work anymore.
+ */
+ @Override
+ void close();
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IRegistryRowChangeNotifier.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IRegistryRowChangeNotifier.java
new file mode 100644
index 0000000000..895521d5b2
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/IRegistryRowChangeNotifier.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.plugin.registry.jdbc.server;
+
+public interface IRegistryRowChangeNotifier {
+
+ void start();
+
+ void subscribeRegistryRowChange(RegistryRowChangeListener registryRowChangeListener);
+
+ interface RegistryRowChangeListener {
+
+ void onRegistryRowUpdated(T data);
+
+ void onRegistryRowAdded(T data);
+
+ void onRegistryRowDeleted(T data);
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataChangeListener.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataChangeListener.java
new file mode 100644
index 0000000000..cea1df2149
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataChangeListener.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.registry.jdbc.server;
+
+public interface JdbcRegistryDataChangeListener {
+
+ void onJdbcRegistryDataChanged(String key, String value);
+
+ void onJdbcRegistryDataDeleted(String key);
+
+ void onJdbcRegistryDataAdded(String key, String value);
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataManager.java
new file mode 100644
index 0000000000..5d44949b3f
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryDataManager.java
@@ -0,0 +1,265 @@
+/*
+ * 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.registry.jdbc.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryThreadFactory;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataChanceEventDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.commons.lang3.time.DateUtils;
+
+import java.time.Duration;
+import java.util.Date;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import lombok.extern.slf4j.Slf4j;
+
+import com.google.common.collect.Lists;
+
+@Slf4j
+public class JdbcRegistryDataManager
+ implements
+ IRegistryRowChangeNotifier,
+ IJdbcRegistryDataManager {
+
+ private final Integer keepJdbcRegistryDataChanceEventHours = 2;
+
+ private final JdbcRegistryProperties registryProperties;
+
+ private final JdbcRegistryDataRepository jdbcRegistryDataRepository;
+
+ private final JdbcRegistryDataChanceEventRepository jdbcRegistryDataChanceEventRepository;
+
+ private final List> registryRowChangeListeners;
+
+ private long lastDetectedJdbcRegistryDataChangeEventId = -1;
+
+ public JdbcRegistryDataManager(JdbcRegistryProperties registryProperties,
+ JdbcRegistryDataRepository jdbcRegistryDataRepository,
+ JdbcRegistryDataChanceEventRepository jdbcRegistryDataChanceEventRepository) {
+ this.registryProperties = registryProperties;
+ this.jdbcRegistryDataChanceEventRepository = jdbcRegistryDataChanceEventRepository;
+ this.jdbcRegistryDataRepository = jdbcRegistryDataRepository;
+ this.registryRowChangeListeners = new CopyOnWriteArrayList<>();
+ this.lastDetectedJdbcRegistryDataChangeEventId =
+ jdbcRegistryDataChanceEventRepository.getMaxJdbcRegistryDataChanceEventId();
+ }
+
+ @Override
+ public void start() {
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor().scheduleWithFixedDelay(
+ this::detectJdbcRegistryDataChangeEvent,
+ registryProperties.getHeartbeatRefreshInterval().toMillis(),
+ registryProperties.getHeartbeatRefreshInterval().toMillis(),
+ TimeUnit.MILLISECONDS);
+
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor().scheduleWithFixedDelay(
+ this::purgeHistoryJdbcRegistryDataChangeEvent,
+ 0,
+ Duration.ofHours(keepJdbcRegistryDataChanceEventHours).toHours(),
+ TimeUnit.HOURS);
+ }
+
+ private void detectJdbcRegistryDataChangeEvent() {
+ final List jdbcRegistryDataChanceEvents = jdbcRegistryDataChanceEventRepository
+ .selectJdbcRegistryDataChangeEventWhereIdAfter(lastDetectedJdbcRegistryDataChangeEventId);
+ if (CollectionUtils.isEmpty(jdbcRegistryDataChanceEvents)) {
+ return;
+ }
+ for (JdbcRegistryDataChanceEventDTO jdbcRegistryDataChanceEvent : jdbcRegistryDataChanceEvents) {
+ log.debug("Detect JdbcRegistryDataChangeEvent: {}", jdbcRegistryDataChanceEvent);
+ switch (jdbcRegistryDataChanceEvent.getEventType()) {
+ case ADD:
+ doTriggerJdbcRegistryDataAddedListener(
+ Lists.newArrayList(jdbcRegistryDataChanceEvent.getJdbcRegistryData()));
+ break;
+ case UPDATE:
+ doTriggerJdbcRegistryDataUpdatedListener(
+ Lists.newArrayList(jdbcRegistryDataChanceEvent.getJdbcRegistryData()));
+ break;
+ case DELETE:
+ doTriggerJdbcRegistryDataRemovedListener(
+ Lists.newArrayList(jdbcRegistryDataChanceEvent.getJdbcRegistryData()));
+ break;
+ default:
+ log.error("Unknown event type: {}", jdbcRegistryDataChanceEvent.getEventType());
+ break;
+ }
+ if (jdbcRegistryDataChanceEvent.getId() > lastDetectedJdbcRegistryDataChangeEventId) {
+ lastDetectedJdbcRegistryDataChangeEventId = jdbcRegistryDataChanceEvent.getId();
+ }
+ }
+ }
+
+ private void purgeHistoryJdbcRegistryDataChangeEvent() {
+ log.info("Purge JdbcRegistryDataChanceEvent which createTime is before: {} hours",
+ keepJdbcRegistryDataChanceEventHours);
+ jdbcRegistryDataChanceEventRepository.deleteJdbcRegistryDataChangeEventBeforeCreateTime(
+ DateUtils.addHours(new Date(), -keepJdbcRegistryDataChanceEventHours));
+ }
+
+ @Override
+ public void subscribeRegistryRowChange(RegistryRowChangeListener registryRowChangeListener) {
+ registryRowChangeListeners.add(checkNotNull(registryRowChangeListener));
+ }
+
+ @Override
+ public boolean existKey(String key) {
+ checkNotNull(key);
+ return jdbcRegistryDataRepository.selectByKey(key).isPresent();
+ }
+
+ @Override
+ public Optional getRegistryDataByKey(String key) {
+ checkNotNull(key);
+ return jdbcRegistryDataRepository.selectByKey(key);
+ }
+
+ @Override
+ public List listJdbcRegistryDataChildren(String key) {
+ checkNotNull(key);
+ return jdbcRegistryDataRepository.selectAll()
+ .stream()
+ .filter(jdbcRegistryDataDTO -> jdbcRegistryDataDTO.getDataKey().startsWith(key)
+ && !jdbcRegistryDataDTO.getDataKey().equals(key))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public void putJdbcRegistryData(Long clientId, String key, String value, DataType dataType) {
+ checkNotNull(clientId);
+ checkNotNull(key);
+ checkNotNull(dataType);
+
+ Optional jdbcRegistryDataOptional = jdbcRegistryDataRepository.selectByKey(key);
+ if (jdbcRegistryDataOptional.isPresent()) {
+ JdbcRegistryDataDTO jdbcRegistryData = jdbcRegistryDataOptional.get();
+ if (!dataType.name().equals(jdbcRegistryData.getDataType())) {
+ throw new UnsupportedOperationException("The data type: " + jdbcRegistryData.getDataType()
+ + " of the key: " + key + " cannot be updated");
+ }
+
+ if (DataType.EPHEMERAL.name().equals(jdbcRegistryData.getDataType())) {
+ if (!jdbcRegistryData.getClientId().equals(clientId)) {
+ throw new UnsupportedOperationException(
+ "The EPHEMERAL data: " + key + " can only be updated by its owner: "
+ + jdbcRegistryData.getClientId() + " but not: " + clientId);
+ }
+ }
+
+ jdbcRegistryData.setDataValue(value);
+ jdbcRegistryData.setLastUpdateTime(new Date());
+ jdbcRegistryDataRepository.updateById(jdbcRegistryData);
+
+ JdbcRegistryDataChanceEventDTO jdbcRegistryDataChanceEvent = JdbcRegistryDataChanceEventDTO.builder()
+ .jdbcRegistryData(jdbcRegistryData)
+ .eventType(JdbcRegistryDataChanceEventDTO.EventType.UPDATE)
+ .createTime(new Date())
+ .build();
+ jdbcRegistryDataChanceEventRepository.insert(jdbcRegistryDataChanceEvent);
+ } else {
+ JdbcRegistryDataDTO jdbcRegistryDataDTO = JdbcRegistryDataDTO.builder()
+ .clientId(clientId)
+ .dataKey(key)
+ .dataValue(value)
+ .dataType(dataType.name())
+ .createTime(new Date())
+ .lastUpdateTime(new Date())
+ .build();
+ jdbcRegistryDataRepository.insert(jdbcRegistryDataDTO);
+ JdbcRegistryDataChanceEventDTO registryDataChanceEvent = JdbcRegistryDataChanceEventDTO.builder()
+ .jdbcRegistryData(jdbcRegistryDataDTO)
+ .eventType(JdbcRegistryDataChanceEventDTO.EventType.ADD)
+ .createTime(new Date())
+ .build();
+ jdbcRegistryDataChanceEventRepository.insert(registryDataChanceEvent);
+ }
+
+ }
+
+ @Override
+ public void deleteJdbcRegistryDataByKey(String key) {
+ checkNotNull(key);
+ // todo: this is not atomic, need to be improved
+ Optional jdbcRegistryDataOptional = jdbcRegistryDataRepository.selectByKey(key);
+ if (!jdbcRegistryDataOptional.isPresent()) {
+ return;
+ }
+ jdbcRegistryDataRepository.deleteByKey(key);
+ JdbcRegistryDataChanceEventDTO registryDataChanceEvent = JdbcRegistryDataChanceEventDTO.builder()
+ .jdbcRegistryData(jdbcRegistryDataOptional.get())
+ .eventType(JdbcRegistryDataChanceEventDTO.EventType.DELETE)
+ .createTime(new Date())
+ .build();
+ jdbcRegistryDataChanceEventRepository.insert(registryDataChanceEvent);
+ }
+
+ private void doTriggerJdbcRegistryDataAddedListener(List valuesToAdd) {
+ if (CollectionUtils.isEmpty(valuesToAdd)) {
+ return;
+ }
+ log.debug("Trigger:onJdbcRegistryDataAdded: {}", valuesToAdd);
+ valuesToAdd.forEach(jdbcRegistryData -> {
+ try {
+ registryRowChangeListeners.forEach(listener -> listener.onRegistryRowAdded(jdbcRegistryData));
+ } catch (Exception ex) {
+ log.error("Trigger:onRegistryRowAdded: {} failed", jdbcRegistryData, ex);
+ }
+ });
+ }
+
+ private void doTriggerJdbcRegistryDataRemovedListener(List valuesToRemoved) {
+ if (CollectionUtils.isEmpty(valuesToRemoved)) {
+ return;
+ }
+ log.debug("Trigger:onJdbcRegistryDataDeleted: {}", valuesToRemoved);
+ valuesToRemoved.forEach(jdbcRegistryData -> {
+ try {
+ registryRowChangeListeners.forEach(listener -> listener.onRegistryRowDeleted(jdbcRegistryData));
+ } catch (Exception ex) {
+ log.error("Trigger:onRegistryRowAdded: {} failed", jdbcRegistryData, ex);
+ }
+ });
+ }
+
+ private void doTriggerJdbcRegistryDataUpdatedListener(List valuesToUpdated) {
+ if (CollectionUtils.isEmpty(valuesToUpdated)) {
+ return;
+ }
+ log.debug("Trigger:onJdbcRegistryDataUpdated: {}", valuesToUpdated);
+ valuesToUpdated.forEach(jdbcRegistryData -> {
+ try {
+ registryRowChangeListeners.forEach(listener -> listener.onRegistryRowUpdated(jdbcRegistryData));
+ } catch (Exception ex) {
+ log.error("Trigger:onRegistryRowAdded: {} failed", jdbcRegistryData, ex);
+ }
+ });
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryLockManager.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryLockManager.java
new file mode 100644
index 0000000000..3b1acf1d4d
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryLockManager.java
@@ -0,0 +1,149 @@
+/*
+ * 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.registry.jdbc.server;
+
+import org.apache.dolphinscheduler.common.thread.ThreadUtils;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.LockUtils;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryLockDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+
+import org.springframework.dao.DuplicateKeyException;
+
+@Slf4j
+public class JdbcRegistryLockManager implements IJdbcRegistryLockManager {
+
+ private final JdbcRegistryProperties jdbcRegistryProperties;
+ private final JdbcRegistryLockRepository jdbcRegistryLockRepository;
+
+ // lockKey -> LockEntry
+ private final Map jdbcRegistryLockHolderMap = new HashMap<>();
+
+ public JdbcRegistryLockManager(JdbcRegistryProperties jdbcRegistryProperties,
+ JdbcRegistryLockRepository jdbcRegistryLockRepository) {
+ this.jdbcRegistryProperties = jdbcRegistryProperties;
+ this.jdbcRegistryLockRepository = jdbcRegistryLockRepository;
+ }
+
+ @Override
+ public void acquireJdbcRegistryLock(Long clientId, String lockKey) {
+ String lockOwner = LockUtils.getLockOwner();
+ while (true) {
+ LockEntry lockEntry = jdbcRegistryLockHolderMap.get(lockKey);
+ if (lockEntry != null && lockOwner.equals(lockEntry.getLockOwner())) {
+ return;
+ }
+ JdbcRegistryLockDTO jdbcRegistryLock = JdbcRegistryLockDTO.builder()
+ .lockKey(lockKey)
+ .clientId(clientId)
+ .lockOwner(lockOwner)
+ .createTime(new Date())
+ .build();
+ try {
+ jdbcRegistryLockRepository.insert(jdbcRegistryLock);
+ if (jdbcRegistryLock != null) {
+ jdbcRegistryLockHolderMap.put(lockKey, LockEntry.builder()
+ .lockKey(lockKey)
+ .lockOwner(lockOwner)
+ .jdbcRegistryLock(jdbcRegistryLock)
+ .build());
+ return;
+ }
+ log.debug("{} acquire the lock {} success", lockOwner, lockKey);
+ } catch (DuplicateKeyException duplicateKeyException) {
+ // The lock is already exist, wait it release.
+ continue;
+ }
+ log.debug("Acquire the lock {} failed try again", lockKey);
+ // acquire failed, wait and try again
+ ThreadUtils.sleep(jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis());
+ }
+ }
+
+ @Override
+ public boolean acquireJdbcRegistryLock(Long clientId, String lockKey, long timeout) {
+ String lockOwner = LockUtils.getLockOwner();
+ long start = System.currentTimeMillis();
+ while (System.currentTimeMillis() - start <= timeout) {
+ LockEntry lockEntry = jdbcRegistryLockHolderMap.get(lockKey);
+ if (lockEntry != null && lockOwner.equals(lockEntry.getLockOwner())) {
+ return true;
+ }
+ JdbcRegistryLockDTO jdbcRegistryLock = JdbcRegistryLockDTO.builder()
+ .lockKey(lockKey)
+ .clientId(clientId)
+ .lockOwner(lockOwner)
+ .createTime(new Date())
+ .build();
+ try {
+ jdbcRegistryLockRepository.insert(jdbcRegistryLock);
+ if (jdbcRegistryLock != null) {
+ jdbcRegistryLockHolderMap.put(lockKey, LockEntry.builder()
+ .lockKey(lockKey)
+ .lockOwner(lockOwner)
+ .jdbcRegistryLock(jdbcRegistryLock)
+ .build());
+ return true;
+ }
+ log.debug("{} acquire the lock {} success", lockOwner, lockKey);
+ } catch (DuplicateKeyException duplicateKeyException) {
+ // The lock is already exist, wait it release.
+ continue;
+ }
+ log.debug("Acquire the lock {} failed try again", lockKey);
+ // acquire failed, wait and try again
+ ThreadUtils.sleep(jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis());
+ }
+ return false;
+ }
+
+ @Override
+ public void releaseJdbcRegistryLock(Long clientId, String lockKey) {
+ LockEntry lockEntry = jdbcRegistryLockHolderMap.get(lockKey);
+ if (lockEntry == null) {
+ return;
+ }
+ if (!clientId.equals(lockEntry.getJdbcRegistryLock().getClientId())) {
+ throw new UnsupportedOperationException(
+ "The client " + clientId + " is not the lock owner of the lock: " + lockKey);
+ }
+ jdbcRegistryLockRepository.deleteById(lockEntry.getJdbcRegistryLock().getId());
+ jdbcRegistryLockHolderMap.remove(lockKey);
+ }
+
+ @Data
+ @Builder
+ @NoArgsConstructor
+ @AllArgsConstructor
+ public static class LockEntry {
+
+ private String lockKey;
+ private String lockOwner;
+ private JdbcRegistryLockDTO jdbcRegistryLock;
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServer.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServer.java
new file mode 100644
index 0000000000..0ac985dae3
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServer.java
@@ -0,0 +1,387 @@
+/*
+ * 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.registry.jdbc.server;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryThreadFactory;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.client.IJdbcRegistryClient;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.client.JdbcRegistryClientIdentify;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryClientHeartbeatDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryClientRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+import org.apache.dolphinscheduler.registry.api.RegistryException;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import java.util.Date;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+
+import com.google.common.collect.Lists;
+
+/**
+ * The JdbcRegistryServer will manage the client, once a client is disconnected, the server will remove the client from the registry, and remove it's related data and lock.
+ */
+@Slf4j
+public class JdbcRegistryServer implements IJdbcRegistryServer {
+
+ private final JdbcRegistryProperties jdbcRegistryProperties;
+
+ private final JdbcRegistryDataRepository jdbcRegistryDataRepository;
+
+ private final JdbcRegistryLockRepository jdbcRegistryLockRepository;
+
+ private final JdbcRegistryClientRepository jdbcRegistryClientRepository;
+
+ private final JdbcRegistryDataManager jdbcRegistryDataManager;
+
+ private final JdbcRegistryLockManager jdbcRegistryLockManager;
+
+ private JdbcRegistryServerState jdbcRegistryServerState;
+
+ private final List jdbcRegistryClients = new CopyOnWriteArrayList<>();
+
+ private final List connectionStateListeners = new CopyOnWriteArrayList<>();
+
+ private final Map jdbcRegistryClientDTOMap =
+ new ConcurrentHashMap<>();
+
+ private Long lastSuccessHeartbeat;
+
+ public JdbcRegistryServer(JdbcRegistryDataRepository jdbcRegistryDataRepository,
+ JdbcRegistryLockRepository jdbcRegistryLockRepository,
+ JdbcRegistryClientRepository jdbcRegistryClientRepository,
+ JdbcRegistryDataChanceEventRepository jdbcRegistryDataChanceEventRepository,
+ JdbcRegistryProperties jdbcRegistryProperties) {
+ this.jdbcRegistryDataRepository = checkNotNull(jdbcRegistryDataRepository);
+ this.jdbcRegistryLockRepository = checkNotNull(jdbcRegistryLockRepository);
+ this.jdbcRegistryClientRepository = checkNotNull(jdbcRegistryClientRepository);
+ this.jdbcRegistryProperties = checkNotNull(jdbcRegistryProperties);
+ this.jdbcRegistryDataManager = new JdbcRegistryDataManager(
+ jdbcRegistryProperties, jdbcRegistryDataRepository, jdbcRegistryDataChanceEventRepository);
+ this.jdbcRegistryLockManager = new JdbcRegistryLockManager(
+ jdbcRegistryProperties, jdbcRegistryLockRepository);
+ this.jdbcRegistryServerState = JdbcRegistryServerState.INIT;
+ lastSuccessHeartbeat = System.currentTimeMillis();
+ }
+
+ @Override
+ public void start() {
+ if (jdbcRegistryServerState != JdbcRegistryServerState.INIT) {
+ // The server is already started or stopped, will not start again.
+ return;
+ }
+ // Purge the previous client to avoid the client is still in the registry.
+ purgePreviousJdbcRegistryClient();
+ // Start the Purge thread
+ // The Purge thread will remove the client from the registry, and remove it's related data and lock.
+ // Connect to the database, load the data and lock.
+ purgeDeadJdbcRegistryClient();
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor()
+ .scheduleWithFixedDelay(this::purgeDeadJdbcRegistryClient,
+ jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis(),
+ jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis(),
+ TimeUnit.MILLISECONDS);
+ jdbcRegistryDataManager.start();
+ jdbcRegistryServerState = JdbcRegistryServerState.STARTED;
+ doTriggerOnConnectedListener();
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor()
+ .scheduleWithFixedDelay(this::refreshClientsHeartbeat,
+ 0,
+ jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis(),
+ TimeUnit.MILLISECONDS);
+ }
+
+ @SneakyThrows
+ @Override
+ public void registerClient(IJdbcRegistryClient jdbcRegistryClient) {
+ checkNotNull(jdbcRegistryClient);
+
+ JdbcRegistryClientIdentify jdbcRegistryClientIdentify = jdbcRegistryClient.getJdbcRegistryClientIdentify();
+ checkNotNull(jdbcRegistryClientIdentify);
+
+ JdbcRegistryClientHeartbeatDTO registryClientDTO = JdbcRegistryClientHeartbeatDTO.builder()
+ .id(jdbcRegistryClientIdentify.getClientId())
+ .clientName(jdbcRegistryClientIdentify.getClientName())
+ .clientConfig(
+ new JdbcRegistryClientHeartbeatDTO.ClientConfig(
+ jdbcRegistryProperties.getSessionTimeout().toMillis()))
+ .createTime(new Date())
+ .lastHeartbeatTime(System.currentTimeMillis())
+ .build();
+
+ while (jdbcRegistryClientDTOMap.containsKey(jdbcRegistryClientIdentify)) {
+ log.warn("The client {} is already exist the registry.", jdbcRegistryClientIdentify.getClientId());
+ Thread.sleep(jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis());
+ }
+ jdbcRegistryClientRepository.insert(registryClientDTO);
+ jdbcRegistryClients.add(jdbcRegistryClient);
+ jdbcRegistryClientDTOMap.put(jdbcRegistryClientIdentify, registryClientDTO);
+ }
+
+ @Override
+ public void deregisterClient(IJdbcRegistryClient jdbcRegistryClient) {
+ checkNotNull(jdbcRegistryClient);
+ jdbcRegistryClients.remove(jdbcRegistryClient);
+ jdbcRegistryClientDTOMap.remove(jdbcRegistryClient.getJdbcRegistryClientIdentify());
+
+ JdbcRegistryClientIdentify jdbcRegistryClientIdentify = jdbcRegistryClient.getJdbcRegistryClientIdentify();
+ checkNotNull(jdbcRegistryClientIdentify);
+
+ doPurgeJdbcRegistryClientInDB(Lists.newArrayList(jdbcRegistryClientIdentify.getClientId()));
+ }
+
+ @Override
+ public JdbcRegistryServerState getServerState() {
+ return jdbcRegistryServerState;
+ }
+
+ @Override
+ public void subscribeConnectionStateChange(ConnectionStateListener connectionStateListener) {
+ checkNotNull(connectionStateListener);
+ connectionStateListeners.add(connectionStateListener);
+ }
+
+ @Override
+ public void subscribeJdbcRegistryDataChange(JdbcRegistryDataChangeListener jdbcRegistryDataChangeListener) {
+ checkNotNull(jdbcRegistryDataChangeListener);
+ jdbcRegistryDataManager.subscribeRegistryRowChange(
+ new IRegistryRowChangeNotifier.RegistryRowChangeListener() {
+
+ @Override
+ public void onRegistryRowUpdated(JdbcRegistryDataDTO data) {
+ jdbcRegistryDataChangeListener.onJdbcRegistryDataChanged(data.getDataKey(),
+ data.getDataValue());
+ }
+
+ @Override
+ public void onRegistryRowAdded(JdbcRegistryDataDTO data) {
+ jdbcRegistryDataChangeListener.onJdbcRegistryDataAdded(data.getDataKey(), data.getDataValue());
+ }
+
+ @Override
+ public void onRegistryRowDeleted(JdbcRegistryDataDTO data) {
+ jdbcRegistryDataChangeListener.onJdbcRegistryDataDeleted(data.getDataKey());
+ }
+ });
+ }
+
+ @Override
+ public boolean existJdbcRegistryDataKey(String key) {
+ return jdbcRegistryDataManager.existKey(key);
+ }
+
+ @Override
+ public Optional getJdbcRegistryDataByKey(String key) {
+ return jdbcRegistryDataManager.getRegistryDataByKey(key);
+ }
+
+ @Override
+ public List listJdbcRegistryDataChildren(String key) {
+ return jdbcRegistryDataManager.listJdbcRegistryDataChildren(key);
+ }
+
+ @Override
+ public void putJdbcRegistryData(Long clientId, String key, String value, DataType dataType) {
+ jdbcRegistryDataManager.putJdbcRegistryData(clientId, key, value, dataType);
+ }
+
+ @Override
+ public void deleteJdbcRegistryDataByKey(String key) {
+ jdbcRegistryDataManager.deleteJdbcRegistryDataByKey(key);
+ }
+
+ @Override
+ public void acquireJdbcRegistryLock(Long clientId, String lockKey) {
+ try {
+ jdbcRegistryLockManager.acquireJdbcRegistryLock(clientId, lockKey);
+ } catch (Exception ex) {
+ throw new RegistryException("Acquire the lock: " + lockKey + " error", ex);
+ }
+ }
+
+ @Override
+ public boolean acquireJdbcRegistryLock(Long clientId, String lockKey, long timeout) {
+ try {
+ return jdbcRegistryLockManager.acquireJdbcRegistryLock(clientId, lockKey, timeout);
+ } catch (Exception ex) {
+ throw new RegistryException("Acquire the lock: " + lockKey + " error", ex);
+ }
+ }
+
+ @Override
+ public void releaseJdbcRegistryLock(Long clientId, String lockKey) {
+ try {
+ jdbcRegistryLockManager.releaseJdbcRegistryLock(clientId, lockKey);
+ } catch (Exception ex) {
+ throw new RegistryException("Release the lock: " + lockKey + " error", ex);
+ }
+ }
+
+ @Override
+ public void close() {
+ jdbcRegistryServerState = JdbcRegistryServerState.STOPPED;
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor().shutdown();
+ List clientIds = jdbcRegistryClients.stream()
+ .map(IJdbcRegistryClient::getJdbcRegistryClientIdentify)
+ .map(JdbcRegistryClientIdentify::getClientId)
+ .collect(Collectors.toList());
+ doPurgeJdbcRegistryClientInDB(clientIds);
+ jdbcRegistryClients.clear();
+ jdbcRegistryClientDTOMap.clear();
+ }
+
+ private void purgePreviousJdbcRegistryClient() {
+ if (jdbcRegistryServerState == JdbcRegistryServerState.STOPPED) {
+ return;
+ }
+ List previousJdbcRegistryClientIds = jdbcRegistryClientRepository.queryAll()
+ .stream()
+ .filter(jdbcRegistryClientHeartbeat -> jdbcRegistryClientHeartbeat.getClientName()
+ .equals(jdbcRegistryProperties.getJdbcRegistryClientName()))
+ .map(JdbcRegistryClientHeartbeatDTO::getId)
+ .collect(Collectors.toList());
+ doPurgeJdbcRegistryClientInDB(previousJdbcRegistryClientIds);
+
+ }
+
+ private void purgeDeadJdbcRegistryClient() {
+ if (jdbcRegistryServerState == JdbcRegistryServerState.STOPPED) {
+ return;
+ }
+ List deadJdbcRegistryClientIds = jdbcRegistryClientRepository.queryAll()
+ .stream()
+ .filter(JdbcRegistryClientHeartbeatDTO::isDead)
+ .map(JdbcRegistryClientHeartbeatDTO::getId)
+ .collect(Collectors.toList());
+ doPurgeJdbcRegistryClientInDB(deadJdbcRegistryClientIds);
+
+ }
+
+ private void doPurgeJdbcRegistryClientInDB(List jdbcRegistryClientIds) {
+ if (CollectionUtils.isEmpty(jdbcRegistryClientIds)) {
+ return;
+ }
+ log.info("Begin to delete dead jdbcRegistryClient: {}", jdbcRegistryClientIds);
+ jdbcRegistryDataRepository.deleteEphemeralDateByClientIds(jdbcRegistryClientIds);
+ jdbcRegistryLockRepository.deleteByClientIds(jdbcRegistryClientIds);
+ jdbcRegistryClientRepository.deleteByIds(jdbcRegistryClientIds);
+ log.info("Success delete dead jdbcRegistryClient: {}", jdbcRegistryClientIds);
+ }
+
+ private void refreshClientsHeartbeat() {
+ if (CollectionUtils.isEmpty(jdbcRegistryClients)) {
+ return;
+ }
+ if (jdbcRegistryServerState == JdbcRegistryServerState.STOPPED) {
+ log.warn("The JdbcRegistryServer is STOPPED, will not refresh clients: {} heartbeat.",
+ CollectionUtils.collect(jdbcRegistryClients, IJdbcRegistryClient::getJdbcRegistryClientIdentify));
+ return;
+ }
+ // Refresh the client's term
+ try {
+ long now = System.currentTimeMillis();
+ for (IJdbcRegistryClient jdbcRegistryClient : jdbcRegistryClients) {
+ JdbcRegistryClientHeartbeatDTO jdbcRegistryClientHeartbeatDTO =
+ jdbcRegistryClientDTOMap.get(jdbcRegistryClient.getJdbcRegistryClientIdentify());
+ if (jdbcRegistryClientHeartbeatDTO == null) {
+ // This may occur when the data in db has been deleted, but the client is still alive.
+ log.error(
+ "The client {} is not found in the registry, will not refresh it's term. (This may happen when the client is removed from the db)",
+ jdbcRegistryClient.getJdbcRegistryClientIdentify().getClientId());
+ continue;
+ }
+ JdbcRegistryClientHeartbeatDTO clone = jdbcRegistryClientHeartbeatDTO.clone();
+ clone.setLastHeartbeatTime(now);
+ jdbcRegistryClientRepository.updateById(jdbcRegistryClientHeartbeatDTO);
+ jdbcRegistryClientHeartbeatDTO.setLastHeartbeatTime(clone.getLastHeartbeatTime());
+ }
+ if (jdbcRegistryServerState == JdbcRegistryServerState.SUSPENDED) {
+ jdbcRegistryServerState = JdbcRegistryServerState.STARTED;
+ doTriggerReconnectedListener();
+ }
+ lastSuccessHeartbeat = now;
+ log.debug("Success refresh clients: {} heartbeat.",
+ CollectionUtils.collect(jdbcRegistryClients, IJdbcRegistryClient::getJdbcRegistryClientIdentify));
+ } catch (Exception ex) {
+ log.error("Failed to refresh the client's term", ex);
+ switch (jdbcRegistryServerState) {
+ case STARTED:
+ jdbcRegistryServerState = JdbcRegistryServerState.SUSPENDED;
+ break;
+ case SUSPENDED:
+ if (System.currentTimeMillis() - lastSuccessHeartbeat > jdbcRegistryProperties.getSessionTimeout()
+ .toMillis()) {
+ jdbcRegistryServerState = JdbcRegistryServerState.DISCONNECTED;
+ doTriggerOnDisConnectedListener();
+ }
+ break;
+ default:
+ break;
+ }
+ }
+ }
+
+ private void doTriggerReconnectedListener() {
+ log.info("Trigger:onReconnected listener.");
+ connectionStateListeners.forEach(listener -> {
+ try {
+ listener.onReconnected();
+ } catch (Exception ex) {
+ log.error("Trigger:onReconnected failed", ex);
+ }
+ });
+ }
+
+ private void doTriggerOnConnectedListener() {
+ log.info("Trigger:onConnected listener.");
+ connectionStateListeners.forEach(listener -> {
+ try {
+ listener.onConnected();
+ } catch (Exception ex) {
+ log.error("Trigger:onConnected failed", ex);
+ }
+ });
+ }
+
+ private void doTriggerOnDisConnectedListener() {
+ log.info("Trigger:onDisConnected listener.");
+ connectionStateListeners.forEach(listener -> {
+ try {
+ listener.onDisConnected();
+ } catch (Exception ex) {
+ log.error("Trigger:onDisConnected failed", ex);
+ }
+ });
+ }
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServerState.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServerState.java
new file mode 100644
index 0000000000..d62c03d4e6
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/java/org/apache/dolphinscheduler/plugin/registry/jdbc/server/JdbcRegistryServerState.java
@@ -0,0 +1,46 @@
+/*
+ * 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.registry.jdbc.server;
+
+import org.apache.dolphinscheduler.plugin.registry.jdbc.JdbcRegistryProperties;
+
+public enum JdbcRegistryServerState {
+ /**
+ * Once the {@link JdbcRegistryServer} created, it will be in the INIT state.
+ */
+ INIT,
+ /**
+ * After the {@link JdbcRegistryServer} started, it will be in the STARTED state.
+ */
+ STARTED,
+ /**
+ * Once the {@link JdbcRegistryServer} cannot connect to DB, it will be in the SUSPENDED state, and if it can reconnect to DB in {@link JdbcRegistryProperties#getSessionTimeout()} ()},
+ * it will be changed to the STARTED state again.
+ */
+ SUSPENDED,
+ /**
+ * If the {@link JdbcRegistryServer} cannot connected to DB in {@link JdbcRegistryProperties#getSessionTimeout()}, it will be in the DISCONNECTED state.
+ */
+ DISCONNECTED,
+ /**
+ * If the {@link JdbcRegistryServer} closed, it will be in the STOPPED state.
+ */
+ STOPPED,
+ ;
+
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/mysql_registry_init.sql b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/mysql_registry_init.sql
index 408a2810a9..8e8762e5b4 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/mysql_registry_init.sql
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/mysql_registry_init.sql
@@ -19,15 +19,15 @@
DROP TABLE IF EXISTS `t_ds_jdbc_registry_data`;
CREATE TABLE `t_ds_jdbc_registry_data`
(
- `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
`data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',
`data_value` text NOT NULL COMMENT 'data, like zookeeper node value',
- `data_type` tinyint(4) NOT NULL COMMENT '1: ephemeral node, 2: persistent node',
- `last_term` bigint NOT NULL COMMENT 'last term time',
- `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'last update time',
+ `data_type` varchar(64) NOT NULL COMMENT 'EPHEMERAL, PERSISTENT',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
`create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'last update time',
PRIMARY KEY (`id`),
- unique (`data_key`)
+ unique Key `uk_t_ds_jdbc_registry_dataKey` (`data_key`)
) ENGINE = InnoDB
DEFAULT CHARSET = utf8;
@@ -35,13 +35,35 @@ CREATE TABLE `t_ds_jdbc_registry_data`
DROP TABLE IF EXISTS `t_ds_jdbc_registry_lock`;
CREATE TABLE `t_ds_jdbc_registry_lock`
(
- `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
- `lock_key` varchar(256) NOT NULL COMMENT 'lock path',
- `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',
- `last_term` bigint NOT NULL COMMENT 'last term time',
- `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'last update time',
- `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `lock_key` varchar(256) NOT NULL COMMENT 'lock path',
+ `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',
+ `client_id` bigint(11) NOT NULL COMMENT 'client id',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
PRIMARY KEY (`id`),
- unique (`lock_key`)
+ unique Key `uk_t_ds_jdbc_registry_lockKey` (`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_ds_jdbc_registry_client_heartbeat`
+(
+ `id` bigint(11) NOT NULL COMMENT 'primary key',
+ `client_name` varchar(256) NOT NULL COMMENT 'client name, ip_processId',
+ `last_heartbeat_time` bigint(11) NOT NULL COMMENT 'last heartbeat timestamp',
+ `connection_config` text NOT NULL COMMENT 'connection config',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_ds_jdbc_registry_data_change_event`;
+CREATE TABLE `t_ds_jdbc_registry_data_change_event`
+(
+ `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',
+ `event_type` varchar(64) NOT NULL COMMENT 'ADD, UPDATE, DELETE',
+ `jdbc_registry_data` text NOT NULL COMMENT 'jdbc registry data',
+ `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',
+ PRIMARY KEY (`id`)
) ENGINE = InnoDB
DEFAULT CHARSET = utf8;
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/postgresql_registry_init.sql b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/postgresql_registry_init.sql
index 795bc9b3eb..cb7a63909c 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/postgresql_registry_init.sql
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/main/resources/postgresql_registry_init.sql
@@ -18,28 +18,48 @@
DROP TABLE IF EXISTS t_ds_jdbc_registry_data;
create table t_ds_jdbc_registry_data
(
- id serial
- constraint t_ds_jdbc_registry_data_pk primary key,
- data_key varchar not null,
- data_value text not null,
- data_type int4 not null,
- last_term bigint not null,
- last_update_time timestamp default current_timestamp not null,
- create_time timestamp default current_timestamp not null
+ id bigserial not null,
+ data_key varchar not null,
+ data_value text not null,
+ data_type varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ last_update_time timestamp not null default current_timestamp,
+ primary key (id)
);
-
-create unique index t_ds_jdbc_registry_data_key_uindex on t_ds_jdbc_registry_data (data_key);
+create unique index uk_t_ds_jdbc_registry_dataKey on t_ds_jdbc_registry_data (data_key);
DROP TABLE IF EXISTS t_ds_jdbc_registry_lock;
create table t_ds_jdbc_registry_lock
(
- id serial
- constraint t_ds_jdbc_registry_lock_pk primary key,
- lock_key varchar not null,
- lock_owner varchar not null,
- last_term bigint not null,
- last_update_time timestamp default current_timestamp not null,
- create_time timestamp default current_timestamp not null
+ id bigserial not null,
+ lock_key varchar not null,
+ lock_owner varchar not null,
+ client_id bigint not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+create unique index uk_t_ds_jdbc_registry_lockKey on t_ds_jdbc_registry_lock (lock_key);
+
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_client_heartbeat;
+create table t_ds_jdbc_registry_client_heartbeat
+(
+ id bigint not null,
+ client_name varchar not null,
+ last_heartbeat_time bigint not null,
+ connection_config text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
+);
+
+DROP TABLE IF EXISTS t_ds_jdbc_registry_data_change_event;
+create table t_ds_jdbc_registry_data_change_event
+(
+ id bigserial not null,
+ event_type varchar not null,
+ jdbc_registry_data text not null,
+ create_time timestamp not null default current_timestamp,
+ primary key (id)
);
-create unique index t_ds_jdbc_registry_lock_key_uindex on t_ds_jdbc_registry_lock (lock_key);
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryTestCase.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryTestCase.java
index c5c83b62e4..5c0862f258 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryTestCase.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryTestCase.java
@@ -17,10 +17,11 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
-import static com.google.common.truth.Truth.assertThat;
-
import org.apache.dolphinscheduler.plugin.registry.RegistryTestCase;
-import org.apache.dolphinscheduler.plugin.registry.jdbc.model.JdbcRegistryLock;
+import org.apache.dolphinscheduler.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.dolphinscheduler.registry.api.ConnectionState;
+
+import java.util.concurrent.atomic.AtomicReference;
import lombok.SneakyThrows;
@@ -28,32 +29,36 @@ import org.junit.jupiter.api.Test;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.boot.autoconfigure.SpringBootApplication;
import org.springframework.boot.test.context.SpringBootTest;
+import org.springframework.test.annotation.DirtiesContext;
+
+import com.google.common.truth.Truth;
@SpringBootTest(classes = {JdbcRegistryProperties.class})
@SpringBootApplication(scanBasePackageClasses = JdbcRegistryProperties.class)
+@DirtiesContext(classMode = DirtiesContext.ClassMode.BEFORE_EACH_TEST_METHOD)
public abstract class JdbcRegistryTestCase extends RegistryTestCase {
@Autowired
private JdbcRegistryProperties jdbcRegistryProperties;
@Autowired
- private JdbcOperator jdbcOperator;
+ private IJdbcRegistryServer jdbcRegistryServer;
- @Test
@SneakyThrows
- public void testTryToAcquireLock_lockIsAlreadyBeenAcquired() {
- final String lockKey = "testTryToAcquireLock_lockIsAlreadyBeenAcquired";
- // acquire success
- JdbcRegistryLock jdbcRegistryLock = jdbcOperator.tryToAcquireLock(lockKey);
- // acquire failed
- assertThat(jdbcOperator.tryToAcquireLock(lockKey)).isNull();
- // release
- jdbcOperator.releaseLock(jdbcRegistryLock.getId());
+ @Test
+ public void testAddConnectionStateListener() {
+
+ AtomicReference connectionState = new AtomicReference<>();
+ registry.addConnectionStateListener(connectionState::set);
+
+ // todo: since the jdbc registry is started at the auto configuration, the stateListener is added after the
+ // registry is started.
+ Truth.assertThat(connectionState.get()).isNull();
}
@Override
public JdbcRegistry createRegistry() {
- return new JdbcRegistry(jdbcRegistryProperties, jdbcOperator);
+ return new JdbcRegistry(jdbcRegistryProperties, jdbcRegistryServer);
}
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactoryTest.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactoryTest.java
new file mode 100644
index 0000000000..cd5e085949
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/JdbcRegistryThreadFactoryTest.java
@@ -0,0 +1,43 @@
+/*
+ * 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.registry.jdbc;
+
+import static com.google.common.truth.Truth.assertThat;
+import static org.awaitility.Awaitility.await;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.junit.jupiter.api.Test;
+
+class JdbcRegistryThreadFactoryTest {
+
+ @Test
+ void getDefaultSchedulerThreadExecutor() {
+ ScheduledExecutorService schedulerThreadExecutor =
+ JdbcRegistryThreadFactory.getDefaultSchedulerThreadExecutor();
+ AtomicInteger atomicInteger = new AtomicInteger(0);
+ for (int i = 0; i < 100; i++) {
+ schedulerThreadExecutor.scheduleWithFixedDelay(atomicInteger::incrementAndGet, 0, 1, TimeUnit.SECONDS);
+ }
+ await()
+ .atMost(10, TimeUnit.SECONDS)
+ .untilAsserted(() -> assertThat(atomicInteger.get()).isEqualTo(100));
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/LockUtilsTest.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/LockUtilsTest.java
new file mode 100644
index 0000000000..84d6b0727a
--- /dev/null
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/LockUtilsTest.java
@@ -0,0 +1,33 @@
+/*
+ * 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.registry.jdbc;
+
+import static com.google.common.truth.Truth.assertThat;
+import static com.google.common.truth.Truth.assertWithMessage;
+
+import org.junit.jupiter.api.Test;
+
+class LockUtilsTest {
+
+ @Test
+ void getLockOwner() {
+ assertThat(LockUtils.getLockOwner()).isNotNull();
+ assertWithMessage("Lock owner should not change")
+ .that(LockUtils.getLockOwner())
+ .isEqualTo(LockUtils.getLockOwner());
+ }
+}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java
index 6dca1aeb2a..244206eafd 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java
@@ -17,9 +17,8 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.Statement;
+import org.apache.dolphinscheduler.common.sql.SqlScriptRunner;
+
import java.time.Duration;
import java.util.stream.Stream;
@@ -35,6 +34,9 @@ import org.testcontainers.containers.wait.strategy.Wait;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.utility.DockerImageName;
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+
@ActiveProfiles("mysql")
class MysqlJdbcRegistryTestCase extends JdbcRegistryTestCase {
@@ -58,42 +60,14 @@ class MysqlJdbcRegistryTestCase extends JdbcRegistryTestCase {
System.clearProperty("spring.datasource.url");
System.setProperty("spring.datasource.url", jdbcUrl);
- try (
- Connection connection = DriverManager.getConnection(jdbcUrl, "root", "root");
- Statement statement = connection.createStatement();) {
- statement.execute(
- "CREATE TABLE `t_ds_jdbc_registry_data`\n" +
- "(\n" +
- " `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',\n" +
- " `data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',\n" +
- " `data_value` text NOT NULL COMMENT 'data, like zookeeper node value',\n"
- +
- " `data_type` tinyint(4) NOT NULL COMMENT '1: ephemeral node, 2: persistent node',\n"
- +
- " `last_term` bigint NOT NULL COMMENT 'last term time',\n" +
- " `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'last update time',\n"
- +
- " `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',\n"
- +
- " PRIMARY KEY (`id`),\n" +
- " unique (`data_key`)\n" +
- ") ENGINE = InnoDB\n" +
- " DEFAULT CHARSET = utf8;");
- statement.execute(
- "CREATE TABLE `t_ds_jdbc_registry_lock`\n" +
- "(\n" +
- " `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',\n" +
- " `lock_key` varchar(256) NOT NULL COMMENT 'lock path',\n" +
- " `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',\n" +
- " `last_term` bigint NOT NULL COMMENT 'last term time',\n" +
- " `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'last update time',\n"
- +
- " `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',\n"
- +
- " PRIMARY KEY (`id`),\n" +
- " unique (`lock_key`)\n" +
- ") ENGINE = InnoDB\n" +
- " DEFAULT CHARSET = utf8;");
+ HikariConfig config = new HikariConfig();
+ config.setJdbcUrl(jdbcUrl);
+ config.setUsername("root");
+ config.setPassword("root");
+
+ try (HikariDataSource dataSource = new HikariDataSource(config)) {
+ SqlScriptRunner sqlScriptRunner = new SqlScriptRunner(dataSource, "mysql_registry_init.sql");
+ sqlScriptRunner.execute();
}
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java
index f34015e5a2..da5bf6c3bc 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/java/org/apache/dolphinscheduler/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java
@@ -17,9 +17,8 @@
package org.apache.dolphinscheduler.plugin.registry.jdbc;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.Statement;
+import org.apache.dolphinscheduler.common.sql.SqlScriptRunner;
+
import java.util.stream.Stream;
import lombok.SneakyThrows;
@@ -35,6 +34,9 @@ import org.testcontainers.containers.PostgreSQLContainer;
import org.testcontainers.lifecycle.Startables;
import org.testcontainers.utility.DockerImageName;
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+
@ActiveProfiles("postgresql")
@SpringBootTest(classes = {JdbcRegistryProperties.class})
@SpringBootApplication(scanBasePackageClasses = JdbcRegistryProperties.class)
@@ -57,36 +59,15 @@ public class PostgresqlJdbcRegistryTestCase extends JdbcRegistryTestCase {
String jdbcUrl = "jdbc:postgresql://localhost:" + postgresqlContainer.getMappedPort(5432) + "/dolphinscheduler";
System.clearProperty("spring.datasource.url");
System.setProperty("spring.datasource.url", jdbcUrl);
- try (
- Connection connection = DriverManager.getConnection(jdbcUrl, "root", "root");
- Statement statement = connection.createStatement();) {
- statement.execute(
- "create table t_ds_jdbc_registry_data\n" +
- "(\n" +
- " id serial\n" +
- " constraint t_ds_jdbc_registry_data_pk primary key,\n" +
- " data_key varchar not null,\n" +
- " data_value text not null,\n" +
- " data_type int4 not null,\n" +
- " last_term bigint not null,\n" +
- " last_update_time timestamp default current_timestamp not null,\n" +
- " create_time timestamp default current_timestamp not null\n" +
- ");");
- statement.execute(
- "create unique index t_ds_jdbc_registry_data_key_uindex on t_ds_jdbc_registry_data (data_key);");
- statement.execute(
- "create table t_ds_jdbc_registry_lock\n" +
- "(\n" +
- " id serial\n" +
- " constraint t_ds_jdbc_registry_lock_pk primary key,\n" +
- " lock_key varchar not null,\n" +
- " lock_owner varchar not null,\n" +
- " last_term bigint not null,\n" +
- " last_update_time timestamp default current_timestamp not null,\n" +
- " create_time timestamp default current_timestamp not null\n" +
- ");");
- statement.execute(
- "create unique index t_ds_jdbc_registry_lock_key_uindex on t_ds_jdbc_registry_lock (lock_key);");
+
+ HikariConfig config = new HikariConfig();
+ config.setJdbcUrl(jdbcUrl);
+ config.setUsername("root");
+ config.setPassword("root");
+
+ try (HikariDataSource dataSource = new HikariDataSource(config)) {
+ SqlScriptRunner sqlScriptRunner = new SqlScriptRunner(dataSource, "postgresql_registry_init.sql");
+ sqlScriptRunner.execute();
}
}
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-mysql.yaml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-mysql.yaml
index a65dd8c5d2..9fb8ea8d0b 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-mysql.yaml
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-mysql.yaml
@@ -26,6 +26,5 @@ spring:
registry:
type: jdbc
- term-refresh-interval: 1s
- term-expire-times: 1
-
+ heartbeat-refresh-interval: 1s
+ session-timeout: 3s
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-postgresql.yaml b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-postgresql.yaml
index 4d5625635b..c54e26e763 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-postgresql.yaml
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-jdbc/src/test/resources/application-postgresql.yaml
@@ -23,6 +23,6 @@ spring:
registry:
type: jdbc
- term-refresh-interval: 1s
- term-expire-times: 1
+ heartbeat-refresh-interval: 1s
+ session-timeout: 3s
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java
index 76313df02a..d02b5f3c0c 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistry.java
@@ -147,11 +147,6 @@ final class ZookeeperRegistry implements Registry {
}
}
- @Override
- public void unsubscribe(String path) {
- CloseableUtils.closeQuietly(treeCacheMap.get(path));
- }
-
@Override
public String get(String key) {
try {
diff --git a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java
index 208fb8fc83..c5b27d8b88 100644
--- a/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java
+++ b/dolphinscheduler-registry/dolphinscheduler-registry-plugins/dolphinscheduler-registry-zookeeper/src/main/java/org/apache/dolphinscheduler/plugin/registry/zookeeper/ZookeeperRegistryProperties.java
@@ -26,6 +26,7 @@ import lombok.Data;
import lombok.NoArgsConstructor;
import lombok.extern.slf4j.Slf4j;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
import org.springframework.boot.context.properties.ConfigurationProperties;
import org.springframework.context.annotation.Configuration;
import org.springframework.validation.Errors;
@@ -36,6 +37,7 @@ import org.springframework.validation.Validator;
@NoArgsConstructor
@AllArgsConstructor
@Configuration
+@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "zookeeper")
@ConfigurationProperties(prefix = "registry")
class ZookeeperRegistryProperties implements Validator {
diff --git a/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java b/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java
index a39d3c9a22..4184aecfff 100644
--- a/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java
+++ b/dolphinscheduler-standalone-server/src/main/java/org/apache/dolphinscheduler/StandaloneServer.java
@@ -17,8 +17,6 @@
package org.apache.dolphinscheduler;
-import org.apache.curator.test.TestingServer;
-
import lombok.extern.slf4j.Slf4j;
import org.springframework.boot.SpringApplication;
@@ -30,10 +28,6 @@ public class StandaloneServer {
public static void main(String[] args) throws Exception {
try {
- // We cannot use try-with-resources to close "TestingServer", since SpringApplication.run() will not block
- // the main thread.
- TestingServer zookeeperServer = new TestingServer(true);
- System.setProperty("registry.zookeeper.connect-string", zookeeperServer.getConnectString());
SpringApplication.run(StandaloneServer.class, args);
} catch (Exception ex) {
log.error("StandaloneServer start failed", ex);
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index 2a056305a7..973515772f 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -80,18 +80,7 @@ mybatis-plus:
banner: false
registry:
- type: zookeeper
- zookeeper:
- namespace: dolphinscheduler
- connect-string: localhost:2181
- retry-policy:
- base-sleep-time: 1s
- max-sleep: 3s
- max-retries: 5
- session-timeout: 60s
- connection-timeout: 15s
- block-until-connected: 15s
- digest: ~
+ type: jdbc
security:
authentication:
diff --git a/dolphinscheduler-tools/src/main/bin/initialize-jdbc-registry.sh b/dolphinscheduler-tools/src/main/bin/initialize-jdbc-registry.sh
deleted file mode 100644
index 895d62a0bb..0000000000
--- a/dolphinscheduler-tools/src/main/bin/initialize-jdbc-registry.sh
+++ /dev/null
@@ -1,31 +0,0 @@
-#!/bin/bash
-#
-# 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.
-#
-
-BIN_DIR=$(dirname $0)
-DOLPHINSCHEDULER_HOME=${DOLPHINSCHEDULER_HOME:-$(cd $BIN_DIR/../..; pwd)}
-
-if [ "$DOCKER" != "true" ]; then
- source "$DOLPHINSCHEDULER_HOME/bin/env/dolphinscheduler_env.sh"
-fi
-
-JAVA_OPTS=${JAVA_OPTS:-"-server -Duser.timezone=${SPRING_JACKSON_TIME_ZONE} -Xms1g -Xmx1g -Xmn512m -XX:+PrintGCDetails -Xloggc:gc.log -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=dump.hprof"}
-
-$JAVA_HOME/bin/java $JAVA_OPTS \
- -cp "$DOLPHINSCHEDULER_HOME/tools/conf":"$DOLPHINSCHEDULER_HOME/tools/libs/*":"$DOLPHINSCHEDULER_HOME/tools/sql" \
- -Dspring.profiles.active=${DATABASE} \
- org.apache.dolphinscheduler.tools.command.CommandApplication
diff --git a/dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/command/CommandApplication.java b/dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/command/CommandApplication.java
deleted file mode 100644
index 1e419f4d19..0000000000
--- a/dolphinscheduler-tools/src/main/java/org/apache/dolphinscheduler/tools/command/CommandApplication.java
+++ /dev/null
@@ -1,152 +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.tools.command;
-
-import org.apache.dolphinscheduler.dao.DaoConfiguration;
-import org.apache.dolphinscheduler.dao.plugin.api.dialect.DatabaseDialect;
-
-import java.sql.Connection;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import javax.sql.DataSource;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.springframework.beans.factory.annotation.Autowired;
-import org.springframework.boot.CommandLineRunner;
-import org.springframework.boot.SpringApplication;
-import org.springframework.boot.autoconfigure.ImportAutoConfiguration;
-import org.springframework.boot.autoconfigure.SpringBootApplication;
-import org.springframework.stereotype.Component;
-
-import com.baomidou.mybatisplus.annotation.DbType;
-
-// todo: use spring-shell to manage the command
-@SpringBootApplication
-@ImportAutoConfiguration(DaoConfiguration.class)
-public class CommandApplication {
-
- public static void main(String[] args) {
- SpringApplication.run(CommandApplication.class, args);
- }
-
- @Component
- @Slf4j
- static class JdbcRegistrySchemaInitializeCommand implements CommandLineRunner {
-
- @Autowired
- private DatabaseDialect databaseDialect;
-
- @Autowired
- private DbType dbType;
-
- @Autowired
- private DataSource dataSource;
-
- JdbcRegistrySchemaInitializeCommand() {
- }
-
- @Override
- public void run(String... args) throws Exception {
- if (databaseDialect.tableExists("t_ds_jdbc_registry_data")
- || databaseDialect.tableExists("t_ds_jdbc_registry_lock")) {
- log.warn("t_ds_jdbc_registry_data/t_ds_jdbc_registry_lock already exists");
- return;
- }
- if (dbType == DbType.MYSQL) {
- jdbcRegistrySchemaInitializeInMysql();
- } else if (dbType == DbType.POSTGRE_SQL) {
- jdbcRegistrySchemaInitializeInPG();
- } else {
- log.error("Unsupported database type: {}", dbType);
- }
- }
-
- private void jdbcRegistrySchemaInitializeInMysql() throws SQLException {
- try (
- Connection connection = dataSource.getConnection();
- Statement statement = connection.createStatement()) {
- statement.execute("CREATE TABLE `t_ds_jdbc_registry_data`\n" +
- "(\n" +
- " `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',\n" +
- " `data_key` varchar(256) NOT NULL COMMENT 'key, like zookeeper node path',\n" +
- " `data_value` text NOT NULL COMMENT 'data, like zookeeper node value',\n" +
- " `data_type` tinyint(4) NOT NULL COMMENT '1: ephemeral node, 2: persistent node',\n"
- +
- " `last_term` bigint NOT NULL COMMENT 'last term time',\n" +
- " `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'last update time',\n"
- +
- " `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',\n"
- +
- " PRIMARY KEY (`id`),\n" +
- " unique (`data_key`)\n" +
- ") ENGINE = InnoDB\n" +
- " DEFAULT CHARSET = utf8;");
-
- statement.execute("CREATE TABLE `t_ds_jdbc_registry_lock`\n" +
- "(\n" +
- " `id` bigint(11) NOT NULL AUTO_INCREMENT COMMENT 'primary key',\n" +
- " `lock_key` varchar(256) NOT NULL COMMENT 'lock path',\n" +
- " `lock_owner` varchar(256) NOT NULL COMMENT 'the lock owner, ip_processId',\n" +
- " `last_term` bigint NOT NULL COMMENT 'last term time',\n" +
- " `last_update_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP ON UPDATE CURRENT_TIMESTAMP COMMENT 'last update time',\n"
- +
- " `create_time` timestamp NOT NULL DEFAULT CURRENT_TIMESTAMP COMMENT 'create time',\n"
- +
- " PRIMARY KEY (`id`),\n" +
- " unique (`lock_key`)\n" +
- ") ENGINE = InnoDB\n" +
- " DEFAULT CHARSET = utf8;");
- }
- }
-
- private void jdbcRegistrySchemaInitializeInPG() throws SQLException {
- try (
- Connection connection = dataSource.getConnection();
- Statement statement = connection.createStatement()) {
- statement.execute("create table t_ds_jdbc_registry_data\n" +
- "(\n" +
- " id serial\n" +
- " constraint t_ds_jdbc_registry_data_pk primary key,\n" +
- " data_key varchar not null,\n" +
- " data_value text not null,\n" +
- " data_type int4 not null,\n" +
- " last_term bigint not null,\n" +
- " last_update_time timestamp default current_timestamp not null,\n" +
- " create_time timestamp default current_timestamp not null\n" +
- ");");
- statement.execute(
- "create unique index t_ds_jdbc_registry_data_key_uindex on t_ds_jdbc_registry_data (data_key);");
- statement.execute("create table t_ds_jdbc_registry_lock\n" +
- "(\n" +
- " id serial\n" +
- " constraint t_ds_jdbc_registry_lock_pk primary key,\n" +
- " lock_key varchar not null,\n" +
- " lock_owner varchar not null,\n" +
- " last_term bigint not null,\n" +
- " last_update_time timestamp default current_timestamp not null,\n" +
- " create_time timestamp default current_timestamp not null\n" +
- ");");
- statement.execute(
- "create unique index t_ds_jdbc_registry_lock_key_uindex on t_ds_jdbc_registry_lock (lock_key);");
- }
- }
-
- }
-}