getBeansWithAnnotation(Class extends Annotation> annotationType) {
return applicationContext.getBeansWithAnnotation(annotationType);
}
+
+ /**
+ * Close this application context, destroying all beans in its bean factory.
+ */
+ @Override
+ public void close() {
+ ((AbstractApplicationContext) applicationContext).close();
+ }
}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/config/ConsoleConfig.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/config/ConsoleConfig.java
new file mode 100644
index 0000000000..1b0bf03adb
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/config/ConsoleConfig.java
@@ -0,0 +1,102 @@
+/*
+ * 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.streampark.console.core.config;
+
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.registry.api.ConnectStrategyProperties;
+import org.apache.streampark.registry.api.enums.RegistryNodeType;
+
+import org.apache.commons.lang3.StringUtils;
+
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Value;
+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 java.time.Duration;
+
+@Data
+@Validated
+@Configuration
+@ConfigurationProperties(prefix = "server")
+@Slf4j
+public class ConsoleConfig implements Validator {
+
+ /**
+ * The console server listen port.
+ */
+ @Value("${server.port:10000}")
+ private int listenPort = 10000;
+
+ /**
+ * Console heart beat task execute interval.
+ */
+ @Value("${streampark.max-heartbeat-interval:10}")
+ private Duration maxHeartbeatInterval = Duration.ofSeconds(10);
+
+ private ConnectStrategyProperties registryDisconnectStrategy = new ConnectStrategyProperties();
+
+ /**
+ * The IP address and listening port of the console server in the format 'ip:listenPort'.
+ */
+ private String consoleAddress;
+
+ /**
+ * The registry path for the console server in the format '/nodes/console/ip:listenPort'.
+ */
+ private String consoleRegistryPath;
+
+ @Override
+ public boolean supports(Class> clazz) {
+ return ConsoleConfig.class.isAssignableFrom(clazz);
+ }
+
+ @Override
+ public void validate(Object target, Errors errors) {
+ ConsoleConfig consoleConfig = (ConsoleConfig) target;
+ if (consoleConfig.getListenPort() <= 0) {
+ errors.rejectValue("listen-port", null, "is invalidated");
+ }
+ if (consoleConfig.getMaxHeartbeatInterval().toMillis() < 0) {
+ errors.rejectValue("max-heartbeat-interval", null, "should be a valid duration");
+ }
+ if (StringUtils.isEmpty(consoleConfig.getConsoleAddress())) {
+ consoleConfig.setConsoleAddress(NetworkUtils.getAddr(consoleConfig.getListenPort()));
+ }
+
+ consoleConfig.setConsoleRegistryPath(
+ RegistryNodeType.CONSOLE_SERVER.getRegistryPath() + "/" + consoleConfig.getConsoleAddress());
+
+ printConfig();
+ }
+
+ private void printConfig() {
+ String config =
+ "\n****************************Console Configuration**************************************" +
+ "\n listen-port -> " + listenPort +
+ "\n registry-disconnect-strategy -> " + registryDisconnectStrategy +
+ "\n console-address -> " + consoleAddress +
+ "\n console-registry-path: " + consoleRegistryPath +
+ "\n****************************Master Configuration**************************************";
+ log.info(config);
+ }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Application.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Application.java
index f277d734ec..5685f933e5 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Application.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Application.java
@@ -17,9 +17,9 @@
package org.apache.streampark.console.core.entity;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.ConfigKeys;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.FlinkDevelopmentMode;
import org.apache.streampark.common.enums.FlinkExecutionMode;
@@ -98,7 +98,7 @@ public class Application implements Serializable {
private String flinkImage;
/** k8s namespace */
- private String k8sNamespace = Constant.DEFAULT;
+ private String k8sNamespace = Constants.DEFAULT;
/** The exposed type of the rest service of K8s(kubernetes.rest-service.exposed.type) */
private Integer k8sRestExposedType;
@@ -262,7 +262,7 @@ public class Application implements Serializable {
private transient AppControl appControl;
public void setK8sNamespace(String k8sNamespace) {
- this.k8sNamespace = StringUtils.isBlank(k8sNamespace) ? Constant.DEFAULT : k8sNamespace;
+ this.k8sNamespace = StringUtils.isBlank(k8sNamespace) ? Constants.DEFAULT : k8sNamespace;
}
public K8sPodTemplates getK8sPodTemplates() {
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java
index 8dad101fbc..26b73d957e 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/SparkApplication.java
@@ -17,9 +17,9 @@
package org.apache.streampark.console.core.entity;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.ConfigKeys;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.FlinkDevelopmentMode;
import org.apache.streampark.common.enums.SparkExecutionMode;
@@ -103,7 +103,7 @@ public class SparkApplication extends BaseEntity {
private String k8sName;
/** k8s namespace */
- private String k8sNamespace = Constant.DEFAULT;
+ private String k8sNamespace = Constants.DEFAULT;
/** The exposed type of the rest service of K8s(kubernetes.rest-service.exposed.type) */
private Integer k8sRestExposedType;
@@ -256,7 +256,7 @@ public void setDefaultModeIngress(String defaultModeIngress) {
}
public void setK8sNamespace(String k8sNamespace) {
- this.k8sNamespace = StringUtils.isBlank(k8sNamespace) ? Constant.DEFAULT : k8sNamespace;
+ this.k8sNamespace = StringUtils.isBlank(k8sNamespace) ? Constants.DEFAULT : k8sNamespace;
}
public K8sPodTemplates getK8sPodTemplates() {
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Variable.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Variable.java
index 86e0d186fe..3821e10e5a 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Variable.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/entity/Variable.java
@@ -17,7 +17,7 @@
package org.apache.streampark.console.core.entity;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.console.base.mybatis.entity.BaseEntity;
import com.baomidou.mybatisplus.annotation.IdType;
@@ -62,7 +62,7 @@ public class Variable extends BaseEntity {
public void dataMasking() {
if (desensitization) {
- this.setVariableValue(Constant.DEFAULT_DATAMASK_STRING);
+ this.setVariableValue(Constants.DEFAULT_DATAMASK_STRING);
}
}
}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectStrategy.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectStrategy.java
new file mode 100644
index 0000000000..97bb3aa6b2
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectStrategy.java
@@ -0,0 +1,24 @@
+/*
+ * 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.streampark.console.core.registry;
+
+import org.apache.streampark.registry.api.ConnectStrategy;
+
+public interface ConsoleConnectStrategy extends ConnectStrategy {
+
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectionStateListener.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectionStateListener.java
new file mode 100644
index 0000000000..7ad712d25d
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleConnectionStateListener.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.console.core.registry;
+
+import org.apache.streampark.common.lifecycle.ServerLifeCycleManager;
+import org.apache.streampark.registry.api.ConnectionListener;
+import org.apache.streampark.registry.api.ConnectionState;
+
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class ConsoleConnectionStateListener implements ConnectionListener {
+
+ private final ConsoleConnectStrategy consoleConnectStrategy;
+
+ public ConsoleConnectionStateListener(@NonNull ConsoleConnectStrategy consoleConnectStrategy) {
+ this.consoleConnectStrategy = consoleConnectStrategy;
+ }
+
+ @Override
+ public void onUpdate(ConnectionState state) {
+ log.info("Master received a {} event from registry, the current server state is {}", state,
+ ServerLifeCycleManager.getServerStatus());
+ switch (state) {
+ case CONNECTED:
+ break;
+ case SUSPENDED:
+ break;
+ case RECONNECTED:
+ consoleConnectStrategy.reconnect();
+ break;
+ case DISCONNECTED:
+ consoleConnectStrategy.disconnect();
+ break;
+ default:
+ }
+ }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryClient.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryClient.java
new file mode 100644
index 0000000000..74de5cbbe0
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryClient.java
@@ -0,0 +1,161 @@
+/*
+ * 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.streampark.console.core.registry;
+
+import org.apache.streampark.common.IStoppable;
+import org.apache.streampark.common.enums.ServerStatus;
+import org.apache.streampark.common.model.ConsoleHeartBeat;
+import org.apache.streampark.common.thread.ThreadUtils;
+import org.apache.streampark.common.utils.JSONUtils;
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.console.core.config.ConsoleConfig;
+import org.apache.streampark.console.core.task.ConsoleHeartBeatTask;
+import org.apache.streampark.registry.api.RegistryClient;
+import org.apache.streampark.registry.api.RegistryException;
+import org.apache.streampark.registry.api.enums.RegistryNodeType;
+
+import org.apache.commons.lang3.StringUtils;
+
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Component;
+
+import static org.apache.streampark.common.constants.Constants.SLEEP_TIME_MILLIS;
+
+/**
+ * streampark console register client, used to connect to registry and hand the registry events.
+ *
When the console node startup, it will register in registry center. And start a {@link ConsoleHeartBeatTask} to update its metadata in registry.
+ */
+@Component
+@Slf4j
+public class ConsoleRegistryClient implements AutoCloseable {
+
+ @Autowired
+ private RegistryClient registryClient;
+
+ @Autowired
+ private ConsoleConfig consoleConfig;
+
+ @Autowired
+ private ConsoleConnectStrategy consoleConnectStrategy;
+
+ private ConsoleHeartBeatTask consoleHeartBeatTask;
+
+ public void start() {
+ try {
+ log.info("consoleConfig: {}", consoleConfig);
+ this.consoleHeartBeatTask = new ConsoleHeartBeatTask(consoleConfig, registryClient);
+ // console registry
+ registry();
+ registryClient.addConnectionStateListener(new ConsoleConnectionStateListener(consoleConnectStrategy));
+ registryClient.subscribe(RegistryNodeType.ALL_SERVERS.getRegistryPath(), new ConsoleRegistryDataListener());
+ } catch (Exception e) {
+ throw new RegistryException("Console registry client start up error", e);
+ }
+ }
+
+ public void setRegistryStoppable(IStoppable stoppable) {
+ registryClient.setStoppable(stoppable);
+ }
+
+ @Override
+ public void close() {
+ // TODO unsubscribe ConsoleRegistryDataListener
+ deregister();
+ }
+
+ /**
+ * remove console node path
+ *
+ * @param path node path
+ * @param nodeType node type
+ * @param failover is failover
+ */
+ public void removeConsoleNodePath(String path, RegistryNodeType nodeType, boolean failover) {
+ log.info("{} node deleted : {}", nodeType, path);
+
+ if (StringUtils.isEmpty(path)) {
+ log.error("server down error: empty path: {}, nodeType:{}", path, nodeType);
+ return;
+ }
+
+ String serverHost = registryClient.getHostByEventDataPath(path);
+ if (StringUtils.isEmpty(serverHost)) {
+ log.error("server down error: unknown path: {}, nodeType:{}", path, nodeType);
+ return;
+ }
+
+ try {
+ if (!registryClient.exists(path)) {
+ log.info("path: {} not exists", path);
+ }
+ // todo: add failover logic here
+ } catch (Exception e) {
+ log.error("{} server failover failed, host:{}", nodeType, serverHost, e);
+ }
+ }
+
+ /**
+ * Registry the current console server itself to registry.
+ */
+ void registry() {
+ log.info("Console node : {} registering to registry center", consoleConfig.getConsoleAddress());
+ String consoleRegistryPath = consoleConfig.getConsoleRegistryPath();
+
+ ConsoleHeartBeat heartBeat = consoleHeartBeatTask.getHeartBeat();
+ while (ServerStatus.BUSY.equals(heartBeat.getServerStatus())) {
+ log.warn("Console node is BUSY: {}", heartBeat);
+ heartBeat = consoleHeartBeatTask.getHeartBeat();
+ ThreadUtils.sleep(SLEEP_TIME_MILLIS);
+ }
+
+ // remove before persist
+ registryClient.remove(consoleRegistryPath);
+ registryClient.persistEphemeral(consoleRegistryPath,
+ JSONUtils.toJsonString(consoleHeartBeatTask.getHeartBeat()));
+
+ while (!registryClient.checkNodeExists(NetworkUtils.getHost(), RegistryNodeType.CONSOLE_SERVER)) {
+ log.warn("The current console server node:{} cannot find in registry", NetworkUtils.getHost());
+ ThreadUtils.sleep(SLEEP_TIME_MILLIS);
+ }
+
+ // sleep 1s, waiting console failover remove
+ ThreadUtils.sleep(SLEEP_TIME_MILLIS);
+
+ consoleHeartBeatTask.start();
+ log.info("Console node : {} registered to registry center successfully", consoleConfig.getConsoleAddress());
+
+ }
+
+ public void deregister() {
+ try {
+ registryClient.remove(consoleConfig.getConsoleRegistryPath());
+ log.info("Console node : {} unRegistry to register center.", consoleConfig.getConsoleAddress());
+ if (consoleHeartBeatTask != null) {
+ consoleHeartBeatTask.shutdown();
+ }
+ registryClient.close();
+ } catch (Exception e) {
+ log.error("ConsoleServer remove registry path exception ", e);
+ }
+ }
+
+ public boolean isAvailable() {
+ return registryClient.isConnected();
+ }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryDataListener.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryDataListener.java
new file mode 100644
index 0000000000..35e143e76c
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleRegistryDataListener.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.streampark.console.core.registry;
+
+import org.apache.streampark.common.constants.Constants;
+import org.apache.streampark.console.base.util.SpringContextUtils;
+import org.apache.streampark.registry.api.Event;
+import org.apache.streampark.registry.api.SubscribeListener;
+import org.apache.streampark.registry.api.enums.RegistryNodeType;
+
+import com.google.common.base.Strings;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class ConsoleRegistryDataListener implements SubscribeListener {
+
+ private final ConsoleRegistryClient consoleRegistryClient;
+
+ public ConsoleRegistryDataListener() {
+ consoleRegistryClient = SpringContextUtils.getBean(ConsoleRegistryClient.class);
+ }
+
+ @Override
+ public void notify(Event event) {
+ final String path = event.path();
+ if (Strings.isNullOrEmpty(path)) {
+ return;
+ }
+ // monitor console
+ if (path.startsWith(RegistryNodeType.CONSOLE_SERVER.getRegistryPath() + Constants.SINGLE_SLASH)) {
+ handleConsoleEvent(event);
+ }
+ }
+
+ private void handleConsoleEvent(Event event) {
+ final String path = event.path();
+ switch (event.type()) {
+ case ADD:
+ log.info("console node added : {}", path);
+ break;
+ case REMOVE:
+ log.info("console node deleted : {}", path);
+ consoleRegistryClient.removeConsoleNodePath(path, RegistryNodeType.CONSOLE_SERVER, true);
+ break;
+ default:
+ break;
+ }
+ }
+
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleStopStrategy.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleStopStrategy.java
new file mode 100644
index 0000000000..60034dd436
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/registry/ConsoleStopStrategy.java
@@ -0,0 +1,52 @@
+/*
+ * 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.streampark.console.core.registry;
+
+import org.apache.streampark.registry.api.RegistryClient;
+import org.apache.streampark.registry.api.StrategyType;
+
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Service;
+
+/**
+ * This strategy will stop the console server, when disconnected from {@link org.apache.streampark.registry.api.Registry}.
+ */
+@Service
+@Slf4j
+public class ConsoleStopStrategy implements ConsoleConnectStrategy {
+
+ @Autowired
+ private RegistryClient registryClient;
+
+ @Override
+ public void disconnect() {
+ registryClient.getStoppable()
+ .stop("Console disconnected from registry, will stop myself due to the stop strategy");
+ }
+
+ @Override
+ public void reconnect() {
+ log.warn("The current connect strategy is stop, so the console will not reconnect to registry");
+ }
+
+ @Override
+ public StrategyType getStrategyType() {
+ return StrategyType.STOP;
+ }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationActionServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationActionServiceImpl.java
index e19976dd4d..110dae8b9d 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationActionServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationActionServiceImpl.java
@@ -17,9 +17,9 @@
package org.apache.streampark.console.core.service.application.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.ConfigKeys;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.ClusterState;
import org.apache.streampark.common.enums.FlinkDevelopmentMode;
@@ -653,7 +653,7 @@ private Tuple2 getUserJarAndAppConf(FlinkEnv flinkEnv, Applicati
resource.getFilePath(), "pyflink file can't be null, start application failed.");
ApiAlertException.throwIfFalse(
- resource.getFilePath().endsWith(Constant.PYTHON_SUFFIX),
+ resource.getFilePath().endsWith(Constants.PYTHON_SUFFIX),
"pyflink format error, must be a \".py\" suffix, start application failed.");
flinkUserJar = resource.getFilePath();
@@ -693,7 +693,7 @@ private Tuple2 getUserJarAndAppConf(FlinkEnv flinkEnv, Applicati
flinkUserJar = String.format(
"%s/%s",
application.getAppLib(),
- application.getModule().concat(Constant.JAR_SUFFIX));
+ application.getModule().concat(Constants.JAR_SUFFIX));
break;
case APACHE_FLINK:
flinkUserJar = String.format("%s/%s", application.getAppHome(), application.getJar());
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationInfoServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationInfoServiceImpl.java
index 4b87659f53..9318b666c1 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationInfoServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/ApplicationInfoServiceImpl.java
@@ -17,8 +17,8 @@
package org.apache.streampark.console.core.service.application.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.FlinkExecutionMode;
import org.apache.streampark.common.fs.LfsOperator;
@@ -340,7 +340,7 @@ public List listHistoryUploadJars() {
.filter(File::isFile)
.sorted(Comparator.comparingLong(File::lastModified).reversed())
.map(File::getName)
- .filter(fn -> fn.endsWith(Constant.JAR_SUFFIX))
+ .filter(fn -> fn.endsWith(Constants.JAR_SUFFIX))
.limit(DEFAULT_HISTORY_RECORD_LIMIT)
.collect(Collectors.toList());
}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java
index 8d828dc9c6..c5329beef5 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationActionServiceImpl.java
@@ -17,9 +17,9 @@
package org.apache.streampark.console.core.service.application.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.ConfigKeys;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.SparkDevelopmentMode;
import org.apache.streampark.common.enums.SparkExecutionMode;
@@ -456,7 +456,7 @@ private Tuple2 getUserJarAndAppConf(
resource.getFilePath(), "pyflink file can't be null, start application failed.");
ApiAlertException.throwIfFalse(
- resource.getFilePath().endsWith(Constant.PYTHON_SUFFIX),
+ resource.getFilePath().endsWith(Constants.PYTHON_SUFFIX),
"pyflink format error, must be a \".py\" suffix, start application failed.");
flinkUserJar = resource.getFilePath();
@@ -496,7 +496,7 @@ private Tuple2 getUserJarAndAppConf(
flinkUserJar = String.format(
"%s/%s",
application.getAppLib(),
- application.getModule().concat(Constant.JAR_SUFFIX));
+ application.getModule().concat(Constants.JAR_SUFFIX));
break;
case APACHE_SPARK:
flinkUserJar = String.format("%s/%s", application.getAppHome(), application.getJar());
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java
index 01300ad6f2..c8fc174c60 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/application/impl/SparkApplicationInfoServiceImpl.java
@@ -17,8 +17,8 @@
package org.apache.streampark.console.core.service.application.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.SparkExecutionMode;
import org.apache.streampark.common.fs.LfsOperator;
@@ -184,7 +184,7 @@ public List listHistoryUploadJars() {
.filter(File::isFile)
.sorted(Comparator.comparingLong(File::lastModified).reversed())
.map(File::getName)
- .filter(fn -> fn.endsWith(Constant.JAR_SUFFIX))
+ .filter(fn -> fn.endsWith(Constants.JAR_SUFFIX))
.limit(DEFAULT_HISTORY_RECORD_LIMIT)
.collect(Collectors.toList());
}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/AppBuildPipeServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/AppBuildPipeServiceImpl.java
index e616ce4833..d78d2f751b 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/AppBuildPipeServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/AppBuildPipeServiceImpl.java
@@ -17,8 +17,8 @@
package org.apache.streampark.console.core.service.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.FlinkDevelopmentMode;
import org.apache.streampark.common.enums.FlinkExecutionMode;
@@ -456,7 +456,7 @@ private BuildPipeline createPipelineInstance(@Nonnull Application app) {
}
FlinkExecutionMode executionModeEnum = app.getFlinkExecutionMode();
- String mainClass = Constant.STREAMPARK_FLINKSQL_CLIENT_CLASS;
+ String mainClass = Constants.STREAMPARK_FLINKSQL_CLIENT_CLASS;
switch (executionModeEnum) {
case YARN_APPLICATION:
String yarnProvidedPath = app.getAppLib();
@@ -580,7 +580,7 @@ private String retrieveFlinkUserJar(FlinkEnv flinkEnv, Application app) {
switch (app.getApplicationType()) {
case STREAMPARK_FLINK:
return String.format(
- "%s/%s", app.getAppLib(), app.getModule().concat(Constant.JAR_SUFFIX));
+ "%s/%s", app.getAppLib(), app.getModule().concat(Constants.JAR_SUFFIX));
case APACHE_FLINK:
return String.format("%s/%s", app.getAppHome(), app.getJar());
default:
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ProjectServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ProjectServiceImpl.java
index 3723266905..9c2516e618 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ProjectServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ProjectServiceImpl.java
@@ -17,10 +17,10 @@
package org.apache.streampark.console.core.service.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.CommonConfig;
import org.apache.streampark.common.conf.InternalConfigHolder;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.util.AssertUtils;
import org.apache.streampark.common.util.CompletableFutureUtils;
import org.apache.streampark.common.util.FileUtils;
@@ -291,7 +291,7 @@ public List listJars(Project project) {
File projectModuleDir = new File(project.getDistHome(), project.getModule());
return Arrays.stream(Objects.requireNonNull(projectModuleDir.listFiles()))
.map(File::getName)
- .filter(name -> name.endsWith(Constant.JAR_SUFFIX))
+ .filter(name -> name.endsWith(Constants.JAR_SUFFIX))
.collect(Collectors.toList());
}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ResourceServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ResourceServiceImpl.java
index 6ca08054e2..b59a2e6aec 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ResourceServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/ResourceServiceImpl.java
@@ -17,8 +17,8 @@
package org.apache.streampark.console.core.service.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.fs.FsOperator;
import org.apache.streampark.common.util.ExceptionUtils;
import org.apache.streampark.common.util.Utils;
@@ -349,7 +349,7 @@ private RestResponse checkFlinkApp(Resource resourceParam) {
jarFile == null || !jarFile.exists(), "flink app jar must exist.");
Map resp = new HashMap<>(0);
resp.put(STATE, 0);
- if (jarFile.getName().endsWith(Constant.PYTHON_SUFFIX)) {
+ if (jarFile.getName().endsWith(Constants.PYTHON_SUFFIX)) {
return RestResponse.success().data(resp);
}
String mainClass = Utils.getJarManClass(jarFile);
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java
index b9dd40e1ca..75a960caa9 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/service/impl/SparkAppBuildPipeServiceImpl.java
@@ -17,8 +17,8 @@
package org.apache.streampark.console.core.service.impl;
-import org.apache.streampark.common.Constant;
import org.apache.streampark.common.conf.Workspace;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.enums.ApplicationType;
import org.apache.streampark.common.enums.FlinkDevelopmentMode;
import org.apache.streampark.common.enums.SparkExecutionMode;
@@ -372,7 +372,7 @@ private BuildPipeline createPipelineInstance(@Nonnull SparkApplication app) {
}
SparkExecutionMode executionModeEnum = app.getSparkExecutionMode();
- String mainClass = Constant.STREAMPARK_SPARKSQL_CLIENT_CLASS;
+ String mainClass = Constants.STREAMPARK_SPARKSQL_CLIENT_CLASS;
switch (executionModeEnum) {
case YARN_CLUSTER:
case YARN_CLIENT:
@@ -404,7 +404,7 @@ private String retrieveSparkUserJar(SparkEnv sparkEnv, SparkApplication app) {
switch (app.getApplicationType()) {
case STREAMPARK_SPARK:
return String.format(
- "%s/%s", app.getAppLib(), app.getModule().concat(Constant.JAR_SUFFIX));
+ "%s/%s", app.getAppLib(), app.getModule().concat(Constants.JAR_SUFFIX));
case APACHE_SPARK:
return String.format("%s/%s", app.getAppHome(), app.getJar());
default:
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ConsoleHeartBeatTask.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ConsoleHeartBeatTask.java
new file mode 100644
index 0000000000..6be3c5ace1
--- /dev/null
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ConsoleHeartBeatTask.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.console.core.task;
+
+import org.apache.streampark.common.enums.ServerStatus;
+import org.apache.streampark.common.lifecycle.ServerLifeCycleManager;
+import org.apache.streampark.common.model.BaseHeartBeatTask;
+import org.apache.streampark.common.model.ConsoleHeartBeat;
+import org.apache.streampark.common.utils.JSONUtils;
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.common.utils.OSUtils;
+import org.apache.streampark.console.core.config.ConsoleConfig;
+import org.apache.streampark.registry.api.RegistryClient;
+
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class ConsoleHeartBeatTask extends BaseHeartBeatTask {
+
+ private final ConsoleConfig consoleConfig;
+
+ private final RegistryClient registryClient;
+
+ private final String heartBeatPath;
+
+ private final int processId;
+
+ public ConsoleHeartBeatTask(@NonNull ConsoleConfig consoleConfig,
+ @NonNull RegistryClient registryClient) {
+ super("ConsoleHeartBeatTask", consoleConfig.getMaxHeartbeatInterval().toMillis());
+ this.consoleConfig = consoleConfig;
+ this.registryClient = registryClient;
+ this.heartBeatPath = consoleConfig.getConsoleRegistryPath();
+ this.processId = OSUtils.getProcessID();
+ }
+
+ @Override
+ public ConsoleHeartBeat getHeartBeat() {
+ return ConsoleHeartBeat.builder()
+ .startupTime(ServerLifeCycleManager.getServerStartupTime())
+ .reportTime(System.currentTimeMillis())
+ .processId(processId)
+ .serverStatus(ServerStatus.NORMAL)
+ .host(NetworkUtils.getHost())
+ .port(consoleConfig.getListenPort())
+ .build();
+ }
+
+ @Override
+ public void writeHeartBeat(ConsoleHeartBeat consoleHeartBeat) {
+ String masterHeartBeatJson = JSONUtils.toJsonString(consoleHeartBeat);
+ registryClient.persistEphemeral(heartBeatPath, masterHeartBeatJson);
+ log.debug("Success write master heartBeatInfo into registry, masterRegistryPath: {}, heartBeatInfo: {}",
+ heartBeatPath, masterHeartBeatJson);
+ }
+}
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ProjectBuildTask.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ProjectBuildTask.java
index 4406874e44..da511149e9 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ProjectBuildTask.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/core/task/ProjectBuildTask.java
@@ -17,7 +17,7 @@
package org.apache.streampark.console.core.task;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.common.util.CommandUtils;
import org.apache.streampark.common.util.Utils;
import org.apache.streampark.console.base.util.GitUtils;
@@ -177,7 +177,7 @@ private void deploy(Project project) throws Exception {
} else {
// 2) .jar file(normal or official standard flink project)
Utils.requireCheckJarFile(app.toURI().toURL());
- String moduleName = app.getName().replace(Constant.JAR_SUFFIX, "");
+ String moduleName = app.getName().replace(Constants.JAR_SUFFIX, "");
File distHome = project.getDistHome();
File targetDir = new File(distHome, moduleName);
if (!targetDir.exists()) {
@@ -207,7 +207,7 @@ private void findTarOrJar(List list, File path) {
// 2) try look for jar files, there may be multiple jars found.
if (!targetFile.getName().startsWith("original-")
&& !targetFile.getName().endsWith("-sources.jar")
- && targetFile.getName().endsWith(Constant.JAR_SUFFIX)) {
+ && targetFile.getName().endsWith(Constants.JAR_SUFFIX)) {
if (jar == null) {
jar = targetFile;
} else {
diff --git a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/system/entity/User.java b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/system/entity/User.java
index c348439356..1495426480 100644
--- a/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/system/entity/User.java
+++ b/streampark-console/streampark-console-service/src/main/java/org/apache/streampark/console/system/entity/User.java
@@ -17,7 +17,7 @@
package org.apache.streampark.console.system.entity;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.console.base.mybatis.entity.BaseEntity;
import org.apache.streampark.console.core.enums.LoginTypeEnum;
import org.apache.streampark.console.core.enums.UserTypeEnum;
@@ -94,7 +94,7 @@ public class User extends BaseEntity {
private Long lastTeamId;
public void dataMasking() {
- String dataMask = Constant.DEFAULT_DATAMASK_STRING;
+ String dataMask = Constants.DEFAULT_DATAMASK_STRING;
this.setPassword(dataMask);
this.setSalt(dataMask);
}
diff --git a/streampark-console/streampark-console-service/src/main/resources/config.yaml b/streampark-console/streampark-console-service/src/main/resources/config.yaml
index 49a9ef0e9e..e0c9b638ca 100644
--- a/streampark-console/streampark-console-service/src/main/resources/config.yaml
+++ b/streampark-console/streampark-console-service/src/main/resources/config.yaml
@@ -64,6 +64,8 @@ streampark:
project:
# Number of projects allowed to be running at the same time , If there is no limit, -1 can be configured
max-build: 16
+ # console heartbeat interval
+ max-heartbeat-interval: 10s
# flink on yarn or spark on yarn, when the hadoop cluster enable kerberos authentication, it is necessary to set Kerberos authentication parameters.
security:
@@ -97,3 +99,31 @@ sso:
principalNameAttribute:
# Optional, change by authentication client
# Please replace and fill in your client config below when enabled SSO
+
+registry:
+ type: jdbc
+ heartbeat-refresh-interval: 1s
+ session-timeout: 3s
+
+# zookeeper registry example
+#registry:
+# type: zookeeper
+# zookeeper:
+# namespace: streampark
+# 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: ~
+
+network:
+ # network interface preferred like eth0, default: empty
+ preferred-interface: ""
+ # network interface restricted like docker0, default: empty
+ restrict-interface: docker0
+ # network IP gets priority, default inner outer
+ priority-strategy: default
diff --git a/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql b/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
index e3fca631f5..2394c04c8b 100644
--- a/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
+++ b/streampark-console/streampark-console-service/src/main/resources/db/schema-h2.sql
@@ -575,3 +575,63 @@ create table if not exists `t_spark_app` (
`hadoop_user` varchar(500) default null,
primary key(`id`)
);
+
+-- ----------------------------
+-- Table structure for jdbc registry
+-- ----------------------------
+
+DROP TABLE IF EXISTS `t_jdbc_registry_data`;
+CREATE TABLE `t_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_jdbc_registry_dataKey`(`data_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+
+DROP TABLE IF EXISTS `t_jdbc_registry_lock`;
+CREATE TABLE `t_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_jdbc_registry_lockKey`(`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_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_jdbc_registry_data_change_event`;
+CREATE TABLE `t_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/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/EncryptUtilsTest.java b/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/EncryptUtilsTest.java
index 4b9d248479..6563b94e36 100644
--- a/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/EncryptUtilsTest.java
+++ b/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/EncryptUtilsTest.java
@@ -17,7 +17,7 @@
package org.apache.streampark.console.base.util;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@@ -27,8 +27,8 @@ class EncryptUtilsTest {
@Test
void testEncrypt() throws Exception {
String value = "apache streampark";
- String encrypt = EncryptUtils.encrypt(value, Constant.STREAM_PARK);
- String decrypt = EncryptUtils.decrypt(encrypt, Constant.STREAM_PARK);
+ String encrypt = EncryptUtils.encrypt(value, Constants.STREAM_PARK);
+ String decrypt = EncryptUtils.decrypt(encrypt, Constants.STREAM_PARK);
Assertions.assertEquals(value, decrypt);
}
}
diff --git a/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/ShaHashUtilsTest.java b/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/ShaHashUtilsTest.java
index 960b84985e..f85601359f 100644
--- a/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/ShaHashUtilsTest.java
+++ b/streampark-console/streampark-console-service/src/test/java/org/apache/streampark/console/base/util/ShaHashUtilsTest.java
@@ -17,7 +17,7 @@
package org.apache.streampark.console.base.util;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
@@ -28,7 +28,7 @@ class ShaHashUtilsTest {
@Test
void testEncrypt() {
String randomSalt = "rh8b1ojwog777yrg0daesf04gk";
- String encryptPassword = ShaHashUtils.encrypt(randomSalt, Constant.STREAM_PARK);
+ String encryptPassword = ShaHashUtils.encrypt(randomSalt, Constants.STREAM_PARK);
Assertions.assertEquals(
"2513f3748847298ea324dffbf67fe68681dd92315bda830065facd8efe08f54f", encryptPassword);
}
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/CancelRequest.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/CancelRequest.scala
index 8ee9577b6e..a22e7d3f9a 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/CancelRequest.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/CancelRequest.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.client.bean
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.FlinkVersion
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums.FlinkExecutionMode
import org.apache.streampark.common.util.Implicits.JavaMap
@@ -35,5 +35,5 @@ case class CancelRequest(
withDrain: Boolean,
savepointPath: String,
nativeFormat: Boolean,
- override val kubernetesNamespace: String = Constant.DEFAULT)
+ override val kubernetesNamespace: String = Constants.DEFAULT)
extends SavepointRequestTrait
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SavepointRequestTrait.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SavepointRequestTrait.scala
index 22c60b62f8..74fe3f08c3 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SavepointRequestTrait.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SavepointRequestTrait.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.client.bean
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.FlinkVersion
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums.FlinkExecutionMode
import org.apache.streampark.common.util.Implicits._
@@ -40,7 +40,7 @@ trait SavepointRequestTrait {
val nativeFormat: Boolean
- val kubernetesNamespace: String = Constant.DEFAULT
+ val kubernetesNamespace: String = Constants.DEFAULT
@Nullable val properties: JavaMap[String, Any]
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SubmitRequest.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SubmitRequest.scala
index 051171968b..97a00b3266 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SubmitRequest.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/SubmitRequest.scala
@@ -17,9 +17,9 @@
package org.apache.streampark.flink.client.bean
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.{FlinkVersion, Workspace}
import org.apache.streampark.common.conf.ConfigKeys._
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums._
import org.apache.streampark.common.util._
import org.apache.streampark.common.util.Implicits._
@@ -66,8 +66,8 @@ case class SubmitRequest(
lazy val appMain: String = this.developmentMode match {
case FlinkDevelopmentMode.FLINK_SQL =>
- Constant.STREAMPARK_FLINKSQL_CLIENT_CLASS
- case FlinkDevelopmentMode.PYFLINK => Constant.PYTHON_FLINK_DRIVER_CLASS_NAME
+ Constants.STREAMPARK_FLINKSQL_CLIENT_CLASS
+ case FlinkDevelopmentMode.PYFLINK => Constants.PYTHON_FLINK_DRIVER_CLASS_NAME
case _ => appProperties(KEY_FLINK_APPLICATION_MAIN_CLASS)
}
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/TriggerSavepointRequest.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/TriggerSavepointRequest.scala
index cb8024c2ed..a1f90475ca 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/TriggerSavepointRequest.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-api/src/main/scala/org/apache/streampark/flink/client/bean/TriggerSavepointRequest.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.client.bean
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.FlinkVersion
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums.FlinkExecutionMode
import org.apache.streampark.common.util.Implicits.JavaMap
@@ -34,5 +34,5 @@ case class TriggerSavepointRequest(
jobId: String,
savepointPath: String,
nativeFormat: Boolean,
- override val kubernetesNamespace: String = Constant.DEFAULT)
+ override val kubernetesNamespace: String = Constants.DEFAULT)
extends SavepointRequestTrait
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/impl/YarnApplicationClient.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/impl/YarnApplicationClient.scala
index 12f15e3011..d37808d798 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/impl/YarnApplicationClient.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/impl/YarnApplicationClient.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.client.impl
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.Workspace
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums.FlinkDevelopmentMode
import org.apache.streampark.common.fs.FsOperator
import org.apache.streampark.common.util.{AssertUtils, FileUtils, HdfsUtils}
@@ -117,16 +117,16 @@ object YarnApplicationClient extends YarnClientTrait {
// python.archives
.safeSet(PythonOptions.PYTHON_ARCHIVES, pyVenv)
// python.client.executable
- .safeSet(PythonOptions.PYTHON_CLIENT_EXECUTABLE, Constant.PYTHON_EXECUTABLE)
+ .safeSet(PythonOptions.PYTHON_CLIENT_EXECUTABLE, Constants.PYTHON_EXECUTABLE)
// python.executable
- .safeSet(PythonOptions.PYTHON_EXECUTABLE, Constant.PYTHON_EXECUTABLE)
+ .safeSet(PythonOptions.PYTHON_EXECUTABLE, Constants.PYTHON_EXECUTABLE)
val args: util.List[String] =
flinkConfig.get(ApplicationConfiguration.APPLICATION_ARGS)
// Caused by: java.lang.UnsupportedOperationException
val argsList: util.ArrayList[String] = new util.ArrayList[String](args)
argsList.add("-pym")
- argsList.add(submitRequest.userJarFile.getName.dropRight(Constant.PYTHON_SUFFIX.length))
+ argsList.add(submitRequest.userJarFile.getName.dropRight(Constants.PYTHON_SUFFIX.length))
flinkConfig.safeSet(ApplicationConfiguration.APPLICATION_ARGS, argsList)
}
diff --git a/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/trait/FlinkClientTrait.scala b/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/trait/FlinkClientTrait.scala
index 7a88919623..06117a5185 100644
--- a/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/trait/FlinkClientTrait.scala
+++ b/streampark-flink/streampark-flink-client/streampark-flink-client-core/src/main/scala/org/apache/streampark/flink/client/trait/FlinkClientTrait.scala
@@ -17,9 +17,9 @@
package org.apache.streampark.flink.client.`trait`
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.ConfigKeys._
import org.apache.streampark.common.conf.Workspace
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.enums._
import org.apache.streampark.common.fs.FsOperator
import org.apache.streampark.common.util._
@@ -108,9 +108,9 @@ trait FlinkClientTrait extends Logger {
// python.archives
.safeSet(PythonOptions.PYTHON_ARCHIVES, pythonVenv)
// python.client.executable
- .safeSet(PythonOptions.PYTHON_CLIENT_EXECUTABLE, Constant.PYTHON_EXECUTABLE)
+ .safeSet(PythonOptions.PYTHON_CLIENT_EXECUTABLE, Constants.PYTHON_EXECUTABLE)
// python.executable
- .safeSet(PythonOptions.PYTHON_EXECUTABLE, Constant.PYTHON_EXECUTABLE)
+ .safeSet(PythonOptions.PYTHON_EXECUTABLE, Constants.PYTHON_EXECUTABLE)
val flinkOptPath: String =
System.getenv(ConfigConstants.ENV_FLINK_OPT_DIR)
diff --git a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/PackerResourceGC.scala b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/PackerResourceGC.scala
index aee57fe51f..e37aac5074 100644
--- a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/PackerResourceGC.scala
+++ b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/PackerResourceGC.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.packer
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.Workspace
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.util.Logger
import org.apache.commons.io.FileUtils
@@ -59,7 +59,7 @@ object PackerResourceGC extends Logger {
private def findLastModifiedOfSubFile(file: File): Array[(File, Long)] = {
val isApplicationMode =
- file.listFiles.map(_.getName).exists(_.contains(Constant.JAR_SUFFIX))
+ file.listFiles.map(_.getName).exists(_.contains(Constants.JAR_SUFFIX))
if (isApplicationMode) {
Array(file -> file.listFiles.map(_.lastModified).max)
} else {
diff --git a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/docker/FlinkDockerfileTemplateTrait.scala b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/docker/FlinkDockerfileTemplateTrait.scala
index 1c412a62fa..95208388bd 100644
--- a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/docker/FlinkDockerfileTemplateTrait.scala
+++ b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/docker/FlinkDockerfileTemplateTrait.scala
@@ -17,7 +17,7 @@
package org.apache.streampark.flink.packer.docker
-import org.apache.streampark.common.Constant
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.fs.LfsOperator
import org.apache.commons.io.FileUtils
@@ -81,12 +81,12 @@ trait FlinkDockerfileTemplateTrait {
flinkExtraLibPaths
.map(new File(_))
.filter(_.exists())
- .filter(_.getName.endsWith(Constant.JAR_SUFFIX))
+ .filter(_.getName.endsWith(Constants.JAR_SUFFIX))
.flatMap {
case f if f.isDirectory =>
f.listFiles
.filter(_.isFile)
- .filter(_.getName.endsWith(Constant.JAR_SUFFIX))
+ .filter(_.getName.endsWith(Constants.JAR_SUFFIX))
.map(_.getAbsolutePath)
case f if f.isFile => Array(f.getAbsolutePath)
}
diff --git a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/maven/MavenTool.scala b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/maven/MavenTool.scala
index b0f1996486..9855eae8df 100644
--- a/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/maven/MavenTool.scala
+++ b/streampark-flink/streampark-flink-packer/src/main/scala/org/apache/streampark/flink/packer/maven/MavenTool.scala
@@ -17,9 +17,9 @@
package org.apache.streampark.flink.packer.maven
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.{InternalConfigHolder, Workspace}
import org.apache.streampark.common.conf.CommonConfig.{MAVEN_AUTH_PASSWORD, MAVEN_AUTH_USER, MAVEN_REMOTE_URL}
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.util.{AssertUtils, Logger, Utils}
import org.apache.streampark.common.util.Implicits._
@@ -63,7 +63,7 @@ object MavenTool extends Logger {
val builder =
new RemoteRepository.Builder(
"central",
- Constant.DEFAULT,
+ Constants.DEFAULT,
InternalConfigHolder.get(MAVEN_REMOTE_URL))
val remoteRepository = {
val buildState =
@@ -103,7 +103,7 @@ object MavenTool extends Logger {
// check userJarPath
val uberJar = new File(outFatJarPath)
require(
- outFatJarPath.endsWith(Constant.JAR_SUFFIX) && !uberJar.isDirectory,
+ outFatJarPath.endsWith(Constants.JAR_SUFFIX) && !uberJar.isDirectory,
s"[StreamPark] streampark-packer: outFatJarPath($outFatJarPath) should be a JAR file.")
uberJar.delete()
// resolve all jarLibs
diff --git a/streampark-flink/streampark-flink-proxy/src/main/scala/org/apache/streampark/flink/proxy/FlinkShimsProxy.scala b/streampark-flink/streampark-flink-proxy/src/main/scala/org/apache/streampark/flink/proxy/FlinkShimsProxy.scala
index 34ccb9f59e..bbd96110b5 100644
--- a/streampark-flink/streampark-flink-proxy/src/main/scala/org/apache/streampark/flink/proxy/FlinkShimsProxy.scala
+++ b/streampark-flink/streampark-flink-proxy/src/main/scala/org/apache/streampark/flink/proxy/FlinkShimsProxy.scala
@@ -17,8 +17,8 @@
package org.apache.streampark.flink.proxy
-import org.apache.streampark.common.Constant
import org.apache.streampark.common.conf.{ConfigKeys, FlinkVersion}
+import org.apache.streampark.common.constants.Constants
import org.apache.streampark.common.util.{ChildFirstClassLoader, ClassLoaderObjectInputStream, ClassLoaderUtils, Logger}
import org.apache.streampark.common.util.Implicits._
@@ -138,7 +138,7 @@ object FlinkShimsProxy extends Logger {
.listFiles()
.foreach((jar: File) => {
val jarName = jar.getName
- if (jarName.endsWith(Constant.JAR_SUFFIX)) {
+ if (jarName.endsWith(Constants.JAR_SUFFIX)) {
if (jarName.startsWith(FLINK_SHIMS_PREFIX)) {
val prefixVer =
s"$FLINK_SHIMS_PREFIX-${majorVersion}_$scalaVersion"
diff --git a/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/FactoryUtil.java b/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/FactoryUtil.java
index 2686a29f45..3ed6b55b81 100644
--- a/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/FactoryUtil.java
+++ b/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/FactoryUtil.java
@@ -17,7 +17,7 @@
package org.apache.streampark.gateway.factories;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.gateway.ConfigOption;
import org.apache.streampark.gateway.exception.ValidationException;
@@ -31,7 +31,7 @@
/** Factory utils for {@link Factory}. */
public class FactoryUtil {
- private static final String DEFAULT_IDENTIFIER = Constant.DEFAULT;
+ private static final String DEFAULT_IDENTIFIER = Constants.DEFAULT;
private static final Logger LOG = LoggerFactory.getLogger(FactoryUtil.class);
public static final ConfigOption SQL_GATEWAY_SERVICE_TYPE = ConfigOption
.key("streampark.sql-gateway.service")
diff --git a/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/SqlGatewayServiceFactoryUtils.java b/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/SqlGatewayServiceFactoryUtils.java
index 19c8942f6d..c37c1da794 100644
--- a/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/SqlGatewayServiceFactoryUtils.java
+++ b/streampark-flink/streampark-flink-sql-gateway/streampark-flink-sql-gateway-base/src/main/java/org/apache/streampark/gateway/factories/SqlGatewayServiceFactoryUtils.java
@@ -17,7 +17,7 @@
package org.apache.streampark.gateway.factories;
-import org.apache.streampark.common.Constant;
+import org.apache.streampark.common.constants.Constants;
import org.apache.streampark.gateway.ConfigOption;
import org.apache.streampark.gateway.exception.ValidationException;
import org.apache.streampark.gateway.service.SqlGatewayService;
@@ -55,7 +55,7 @@ public static List createSqlGatewayService(Map identifiers = Arrays.asList(identifiersStr.split(Constant.SEMICOLON));
+ List identifiers = Arrays.asList(identifiersStr.split(Constants.SEMICOLON));
if (identifiers.isEmpty()) {
throw new ValidationException(
diff --git a/streampark-registry/README.md b/streampark-registry/README.md
new file mode 100644
index 0000000000..1c9d40d0ad
--- /dev/null
+++ b/streampark-registry/README.md
@@ -0,0 +1,37 @@
+## Function
+
+DS use registry to do the below three things:
+
+1. Store the metadata of master/worker so that it can get notify when nodes up and down.
+2. Store the metadata of worker to do load balance.
+3. Acquire a global lock when do failover.
+
+So for DS, the registry need to notify the server when the server subscribe data have added/deleted/updated, support a way to create/release a global lock,
+delete the server's metadata when server down.
+
+## How to use
+
+At present, we have implements three registry: Zookeeper(Default),Etcd,Mysql. If you
+want to use them, you should config it at resource/application.yaml. The configuration details
+can be viewed in the README of plugin under Module streampark-registry-plugins
+
+## Module
+
+### streampark-registry-all
+
+This module is used for exporting the implementation of registry.
+If you want to add new registry,you should add the dependency in the pom.xml
+
+### streampark-registry-api
+
+This module contains the relevant interfaces involved in the use of the registry.
+The following are several important interfaces
+1. Registry Interface: If you want to implement your own registry, you just need to implement this interface
+2. ConnectionListener Interface: This interface is responsible for the connection status between the client and the registry,
+The connection state can be viewed in ConnectionState.java
+3. SubscribeListener Interface: This interface is responsible for monitoring the state changes of child nodes under the specified prefix.
+Event content can be viewed in event.java
+
+### streampark-registry-plugins
+
+This module contains all registry implementations in DS
diff --git a/streampark-registry/pom.xml b/streampark-registry/pom.xml
new file mode 100644
index 0000000000..cc5b647d6d
--- /dev/null
+++ b/streampark-registry/pom.xml
@@ -0,0 +1,91 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark
+ 2.2.0-SNAPSHOT
+
+ streampark-registry
+ pom
+ StreamPark : Registry
+
+
+ streampark-registry-api
+ streampark-registry-plugins
+ streampark-registry-all
+
+
+
+
+ org.springframework
+ spring-context
+ provided
+
+
+ org.springframework.boot
+ spring-boot-autoconfigure
+ provided
+
+
+ javax.annotation
+ javax.annotation-api
+
+
+
+ org.projectlombok
+ lombok
+
+
+
+ com.google.guava
+ guava
+
+
+
+ org.apache.commons
+ commons-lang3
+
+
+
+ org.apache.commons
+ commons-collections4
+
+
+
+
+ org.testcontainers
+ testcontainers
+ test
+
+
+
+ org.testcontainers
+ mysql
+ test
+
+
+
+ org.testcontainers
+ postgresql
+ test
+
+
+
diff --git a/streampark-registry/streampark-registry-all/pom.xml b/streampark-registry/streampark-registry-all/pom.xml
new file mode 100644
index 0000000000..6cdab77552
--- /dev/null
+++ b/streampark-registry/streampark-registry-all/pom.xml
@@ -0,0 +1,41 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry
+ 2.2.0-SNAPSHOT
+
+
+ streampark-registry-all
+
+
+
+ org.apache.streampark
+ streampark-registry-zookeeper
+ ${project.version}
+
+
+ org.apache.streampark
+ streampark-registry-jdbc
+ ${project.version}
+
+
+
diff --git a/streampark-registry/streampark-registry-api/pom.xml b/streampark-registry/streampark-registry-api/pom.xml
new file mode 100644
index 0000000000..c4e0171bea
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/pom.xml
@@ -0,0 +1,37 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry
+ 2.2.0-SNAPSHOT
+
+
+ streampark-registry-api
+
+
+
+ org.apache.streampark
+ streampark-common_${scala.binary.version}
+
+
+
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategy.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategy.java
new file mode 100644
index 0000000000..7aeaac4bc6
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategy.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.streampark.registry.api;
+
+/**
+ * This interface defined a method to be executed when the server disconnected from registry.
+ */
+public interface ConnectStrategy {
+
+ void disconnect();
+
+ void reconnect();
+
+ StrategyType getStrategyType();
+
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategyProperties.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategyProperties.java
new file mode 100644
index 0000000000..8f8a6aac9e
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectStrategyProperties.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.streampark.registry.api;
+
+import lombok.Data;
+
+import java.time.Duration;
+
+@Data
+public class ConnectStrategyProperties {
+
+ private StrategyType strategy = StrategyType.STOP;
+
+ private Duration maxWaitingTime = Duration.ofSeconds(0);
+
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionListener.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionListener.java
new file mode 100644
index 0000000000..ebdc56eb31
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionListener.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.streampark.registry.api;
+
+/**
+ * when the connect state between client and registry center changed,
+ * the {@code onUpdate} function is triggered
+ */
+@FunctionalInterface
+public interface ConnectionListener {
+
+ void onUpdate(ConnectionState newState);
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionState.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionState.java
new file mode 100644
index 0000000000..dfcdbc10f7
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ConnectionState.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.streampark.registry.api;
+
+/**
+ * Connection state between client and registry center(Etcd, MySql, Zookeeper)
+ */
+public enum ConnectionState {
+ CONNECTED,
+ RECONNECTED,
+ SUSPENDED,
+ DISCONNECTED
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Event.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Event.java
new file mode 100644
index 0000000000..deb9d5f7bf
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Event.java
@@ -0,0 +1,131 @@
+/*
+ * 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.streampark.registry.api;
+
+public class Event {
+
+ // The prefix which is watched
+ private String key;
+ // The full path where the event was generated
+ private String path;
+ // The value corresponding to the path
+ private String data;
+ // The event type {ADD, REMOVE, UPDATE}
+ private Type type;
+
+ public Event(String key, String path, String data, Type type) {
+ this.key = key;
+ this.path = path;
+ this.data = data;
+ this.type = type;
+ }
+
+ public Event() {
+ }
+
+ public static EventBuilder builder() {
+ return new EventBuilder();
+ }
+
+ public String key() {
+ return this.key;
+ }
+
+ public String path() {
+ return this.path;
+ }
+
+ public String data() {
+ return this.data;
+ }
+
+ public Type type() {
+ return this.type;
+ }
+
+ public Event key(String key) {
+ this.key = key;
+ return this;
+ }
+
+ public Event path(String path) {
+ this.path = path;
+ return this;
+ }
+
+ public Event data(String data) {
+ this.data = data;
+ return this;
+ }
+
+ public Event type(Type type) {
+ this.type = type;
+ return this;
+ }
+
+ public String toString() {
+ return "Event(key=" + this.key() + ", path=" + this.path() + ", data=" + this.data() + ", type=" + this.type()
+ + ")";
+ }
+
+ public enum Type {
+ ADD,
+ REMOVE,
+ UPDATE
+ }
+
+ public static class EventBuilder {
+
+ private String key;
+ private String path;
+ private String data;
+ private Type type;
+
+ EventBuilder() {
+ }
+
+ public EventBuilder key(String key) {
+ this.key = key;
+ return this;
+ }
+
+ public EventBuilder path(String path) {
+ this.path = path;
+ return this;
+ }
+
+ public EventBuilder data(String data) {
+ this.data = data;
+ return this;
+ }
+
+ public EventBuilder type(Type type) {
+ this.type = type;
+ return this;
+ }
+
+ public Event build() {
+ return new Event(key, path, data, type);
+ }
+
+ public String toString() {
+ return "Event.EventBuilder(key=" + this.key + ", path=" + this.path + ", data=" + this.data + ", type="
+ + this.type + ")";
+ }
+ }
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Registry.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Registry.java
new file mode 100644
index 0000000000..2f349be1d0
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/Registry.java
@@ -0,0 +1,112 @@
+/*
+ * 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.streampark.registry.api;
+
+import lombok.NonNull;
+
+import java.io.Closeable;
+import java.time.Duration;
+import java.util.Collection;
+
+/**
+ * The SPI interface for registry center, each registry plugin should implement this interface.
+ */
+public interface Registry extends Closeable {
+
+ /**
+ * Start the registry, once started, the registry will connect to the registry center.
+ */
+ void start();
+
+ /**
+ * Whether the registry is connected
+ *
+ * @return true if connected, false otherwise.
+ */
+ boolean isConnected();
+
+ /**
+ * Connect to the registry, will wait in the given timeout
+ *
+ * @param timeout max timeout, if timeout <= 0 will wait indefinitely.
+ * @throws RegistryException cannot connect in the given timeout
+ */
+ void connectUntilTimeout(@NonNull Duration timeout) throws RegistryException;
+
+ /**
+ * 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);
+
+ /**
+ * Add a connection listener to collection.
+ */
+ void addConnectionStateListener(ConnectionListener listener);
+
+ /**
+ * Get the value of the key, if key not exist will throw {@link RegistryException}
+ */
+ String get(String key) throws RegistryException;
+
+ /**
+ * Put the key-value pair into the registry
+ *
+ * @param key the key, cannot be null
+ * @param value the value, cannot be null
+ * @param deleteOnDisconnect if true, when the connection state is disconnected, the key will be deleted
+ */
+ void put(String key, String value, boolean deleteOnDisconnect);
+
+ /**
+ * Delete the key from the registry
+ */
+ void delete(String key);
+
+ /**
+ * Return the children of the key
+ */
+ Collection children(String key);
+
+ /**
+ * Check if the key exists
+ *
+ * @param key the key to check
+ * @return true if the key exists
+ */
+ boolean exists(String key);
+
+ /**
+ * Acquire the lock of the prefix {@param key}
+ */
+ boolean acquireLock(String key);
+
+ /**
+ * Acquire the lock of the prefix {@param key}, if acquire in the given timeout return true, else return false.
+ */
+ boolean acquireLock(String key, long timeout);
+
+ /**
+ * Release the lock of the prefix {@param key}
+ */
+ boolean releaseLock(String key);
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryClient.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryClient.java
new file mode 100644
index 0000000000..eee0735dc8
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryClient.java
@@ -0,0 +1,214 @@
+/*
+ * 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.streampark.registry.api;
+
+import org.apache.streampark.common.IStoppable;
+import org.apache.streampark.common.constants.Constants;
+import org.apache.streampark.common.model.ConsoleHeartBeat;
+import org.apache.streampark.common.model.Server;
+import org.apache.streampark.common.utils.JSONUtils;
+import org.apache.streampark.registry.api.enums.RegistryNodeType;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.google.common.base.Strings;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.stereotype.Component;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+@Component
+@Slf4j
+public class RegistryClient {
+
+ private IStoppable stoppable;
+
+ private static final String EMPTY = "";
+
+ private final Registry registry;
+
+ public RegistryClient(Registry registry) {
+ this.registry = registry;
+ if (!registry.exists(RegistryNodeType.CONSOLE_SERVER.getRegistryPath())) {
+ registry.put(RegistryNodeType.CONSOLE_SERVER.getRegistryPath(), EMPTY, false);
+ }
+ }
+
+ public boolean isConnected() {
+ return registry.isConnected();
+
+ }
+
+ public void connectUntilTimeout(@NonNull Duration duration) throws RegistryException {
+ registry.connectUntilTimeout(duration);
+ }
+
+ public List getServerList(RegistryNodeType registryNodeType) {
+ Map serverMaps = getServerMaps(registryNodeType);
+
+ List serverList = new ArrayList<>();
+ for (Map.Entry entry : serverMaps.entrySet()) {
+ String serverPath = entry.getKey();
+ String heartBeatJson = entry.getValue();
+ if (StringUtils.isEmpty(heartBeatJson)) {
+ log.error("The heartBeatJson is empty, serverPath: {}", serverPath);
+ continue;
+ }
+ Server server = new Server();
+ switch (registryNodeType) {
+ case CONSOLE_SERVER:
+ ConsoleHeartBeat consoleHeartBeat =
+ JSONUtils.parseObject(heartBeatJson, ConsoleHeartBeat.class);
+ server.setCreateTime(new Date(consoleHeartBeat.getStartupTime()));
+ server.setLastHeartbeatTime(new Date(consoleHeartBeat.getReportTime()));
+ server.setId(consoleHeartBeat.getProcessId());
+ server.setHost(consoleHeartBeat.getHost());
+ server.setPort(consoleHeartBeat.getPort());
+ break;
+ default:
+ log.warn("unknown registry node type: {}", registryNodeType);
+ }
+
+ server.setResInfo(heartBeatJson);
+ // todo: add host, port in heartBeat Info, so that we don't need to parse this again
+ server.setZkDirectory(registryNodeType.getRegistryPath() + "/" + serverPath);
+ serverList.add(server);
+ }
+ return serverList;
+ }
+
+ /**
+ * Return server host:port -> value
+ */
+ public Map getServerMaps(RegistryNodeType nodeType) {
+ Map serverMap = new HashMap<>();
+ try {
+ Collection serverList = getServerNodes(nodeType);
+ for (String server : serverList) {
+ serverMap.putIfAbsent(server, get(nodeType.getRegistryPath() + Constants.SINGLE_SLASH + server));
+ }
+ } catch (Exception e) {
+ log.error("get server list failed", e);
+ }
+
+ return serverMap;
+ }
+
+ public boolean checkNodeExists(String host, RegistryNodeType nodeType) {
+ return getServerMaps(nodeType).keySet()
+ .stream()
+ .anyMatch(it -> it.contains(host));
+ }
+
+ public Collection getConsoleNodesDirectly() {
+ return getChildrenKeys(RegistryNodeType.CONSOLE_SERVER.getRegistryPath());
+ }
+
+ /**
+ * get host ip:port, path format: parentPath/ip:port
+ *
+ * @param path path
+ * @return host ip:port, string format: parentPath/ip:port
+ */
+ public String getHostByEventDataPath(String path) {
+ checkArgument(!Strings.isNullOrEmpty(path), "path cannot be null or empty");
+
+ final String[] pathArray = path.split(Constants.SINGLE_SLASH);
+
+ checkArgument(pathArray.length >= 1, "cannot parse path: %s", path);
+
+ return pathArray[pathArray.length - 1];
+ }
+
+ public void close() throws IOException {
+ registry.close();
+ }
+
+ public void persistEphemeral(String key, String value) {
+ registry.put(key, value, true);
+ }
+
+ public void remove(String key) {
+ registry.delete(key);
+ }
+
+ public String get(String key) {
+ return registry.get(key);
+ }
+
+ public void subscribe(String path, SubscribeListener listener) {
+ registry.subscribe(path, listener);
+ }
+
+ public void addConnectionStateListener(ConnectionListener listener) {
+ registry.addConnectionStateListener(listener);
+ }
+
+ public boolean exists(String key) {
+ return registry.exists(key);
+ }
+
+ public boolean getLock(String key) {
+ return registry.acquireLock(key);
+ }
+
+ public boolean releaseLock(String key) {
+ return registry.releaseLock(key);
+ }
+
+ public void setStoppable(IStoppable stoppable) {
+ this.stoppable = stoppable;
+ }
+
+ public IStoppable getStoppable() {
+ return stoppable;
+ }
+
+ public boolean isConsolePath(String path) {
+ return path != null
+ && path.startsWith(RegistryNodeType.CONSOLE_SERVER.getRegistryPath() + Constants.SINGLE_SLASH);
+ }
+
+ public Collection getChildrenKeys(final String key) {
+ return registry.children(key);
+ }
+
+ public Set getServerNodeSet(RegistryNodeType nodeType) {
+ try {
+ return new HashSet<>(getServerNodes(nodeType));
+ } catch (Exception e) {
+ throw new RegistryException("Failed to get server node: " + nodeType, e);
+ }
+ }
+
+ private Collection getServerNodes(RegistryNodeType nodeType) {
+ return getChildrenKeys(nodeType.getRegistryPath());
+ }
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryConfiguration.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryConfiguration.java
new file mode 100644
index 0000000000..629b148cc8
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryConfiguration.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.streampark.registry.api;
+
+import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.Configuration;
+
+@Configuration
+public class RegistryConfiguration {
+
+ @Bean
+ @ConditionalOnMissingBean
+ public RegistryClient registryClient(Registry registry) {
+ return new RegistryClient(registry);
+ }
+
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryException.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryException.java
new file mode 100644
index 0000000000..8611f102cb
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/RegistryException.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.registry.api;
+
+public final class RegistryException extends RuntimeException {
+
+ public RegistryException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public RegistryException(String message) {
+ super(message);
+ }
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/StrategyType.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/StrategyType.java
new file mode 100644
index 0000000000..450a4a2721
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/StrategyType.java
@@ -0,0 +1,25 @@
+/*
+ * 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.streampark.registry.api;
+
+public enum StrategyType {
+
+ STOP,
+ WAITING,
+ ;
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/SubscribeListener.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/SubscribeListener.java
new file mode 100644
index 0000000000..9c3ab224ec
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/SubscribeListener.java
@@ -0,0 +1,23 @@
+/*
+ * 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.streampark.registry.api;
+
+public interface SubscribeListener {
+
+ void notify(Event event);
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/enums/RegistryNodeType.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/enums/RegistryNodeType.java
new file mode 100644
index 0000000000..8d7086334d
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/enums/RegistryNodeType.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.streampark.registry.api.enums;
+
+import lombok.AllArgsConstructor;
+import lombok.Getter;
+
+@Getter
+@AllArgsConstructor
+public enum RegistryNodeType {
+
+ ALL_SERVERS("nodes", "/nodes"),
+ CONSOLE_SERVER("ConsoleServer", "/nodes/console-server"),
+ CONSOLE_NODE_LOCK("ConsoleNodeLock", "/lock/console"),
+ ;
+
+ private final String name;
+
+ private final String registryPath;
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractHAServer.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractHAServer.java
new file mode 100644
index 0000000000..130c5914a4
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractHAServer.java
@@ -0,0 +1,104 @@
+/*
+ * 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.streampark.registry.api.ha;
+
+import org.apache.streampark.common.thread.ThreadUtils;
+import org.apache.streampark.registry.api.Event;
+import org.apache.streampark.registry.api.Registry;
+
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+@Slf4j
+public abstract class AbstractHAServer implements HAServer {
+
+ private final Registry registry;
+
+ private final String serverPath;
+
+ private ServerStatus serverStatus;
+
+ private final List serverStatusChangeListeners;
+
+ public AbstractHAServer(Registry registry, String serverPath) {
+ this.registry = registry;
+ this.serverPath = serverPath;
+ this.serverStatus = ServerStatus.STAND_BY;
+ this.serverStatusChangeListeners = Lists.newArrayList(new DefaultServerStatusChangeListener());
+ }
+
+ @Override
+ public void start() {
+ registry.subscribe(serverPath, event -> {
+ if (Event.Type.REMOVE.equals(event.type())) {
+ if (isActive() && !participateElection()) {
+ statusChange(ServerStatus.STAND_BY);
+ }
+ }
+ });
+ ScheduledExecutorService electionSelectionThread =
+ ThreadUtils.newSingleDaemonScheduledExecutorService("election-selection-thread");
+ electionSelectionThread.schedule(() -> {
+ if (isActive()) {
+ return;
+ }
+ if (participateElection()) {
+ statusChange(ServerStatus.ACTIVE);
+ }
+ }, 10, TimeUnit.SECONDS);
+ }
+
+ @Override
+ public boolean isActive() {
+ return ServerStatus.ACTIVE.equals(getServerStatus());
+ }
+
+ @Override
+ public boolean participateElection() {
+ return registry.acquireLock(serverPath, 3_000);
+ }
+
+ @Override
+ public void addServerStatusChangeListener(ServerStatusChangeListener listener) {
+ serverStatusChangeListeners.add(listener);
+ }
+
+ @Override
+ public ServerStatus getServerStatus() {
+ return serverStatus;
+ }
+
+ @Override
+ public void shutdown() {
+ if (isActive()) {
+ registry.releaseLock(serverPath);
+ }
+ }
+
+ private void statusChange(ServerStatus targetStatus) {
+ synchronized (this) {
+ ServerStatus originStatus = serverStatus;
+ serverStatus = targetStatus;
+ serverStatusChangeListeners.forEach(listener -> listener.change(originStatus, serverStatus));
+ }
+ }
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractServerStatusChangeListener.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractServerStatusChangeListener.java
new file mode 100644
index 0000000000..4fcd474518
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/AbstractServerStatusChangeListener.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.registry.api.ha;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public abstract class AbstractServerStatusChangeListener implements ServerStatusChangeListener {
+
+ @Override
+ public void change(HAServer.ServerStatus originStatus, HAServer.ServerStatus currentStatus) {
+ log.info("The status change from {} to {}.", originStatus, currentStatus);
+ if (originStatus == HAServer.ServerStatus.ACTIVE) {
+ if (currentStatus == HAServer.ServerStatus.STAND_BY) {
+ changeToStandBy();
+ }
+ } else if (originStatus == HAServer.ServerStatus.STAND_BY) {
+ if (currentStatus == HAServer.ServerStatus.ACTIVE) {
+ changeToActive();
+ }
+ }
+ }
+
+ public abstract void changeToActive();
+
+ public abstract void changeToStandBy();
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/DefaultServerStatusChangeListener.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/DefaultServerStatusChangeListener.java
new file mode 100644
index 0000000000..f169fd3b7d
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/DefaultServerStatusChangeListener.java
@@ -0,0 +1,34 @@
+/*
+ * 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.streampark.registry.api.ha;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+public class DefaultServerStatusChangeListener extends AbstractServerStatusChangeListener {
+
+ @Override
+ public void changeToActive() {
+ log.info("The status is active now.");
+ }
+
+ @Override
+ public void changeToStandBy() {
+ log.info("The status is standby now.");
+ }
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/HAServer.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/HAServer.java
new file mode 100644
index 0000000000..0b588664f9
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/HAServer.java
@@ -0,0 +1,68 @@
+/*
+ * 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.streampark.registry.api.ha;
+
+/**
+ * Interface for HA server, used to select a active server from multiple servers.
+ * In HA mode, there are multiple servers, only one server is active, others are standby.
+ */
+public interface HAServer {
+
+ /**
+ * Start the server.
+ */
+ void start();
+
+ /**
+ * Judge whether the server is active.
+ *
+ * @return true if the current server is active.
+ */
+ boolean isActive();
+
+ /**
+ * Participate in the election of active server, this method will block until the server is active.
+ */
+ boolean participateElection();
+
+ /**
+ * Add a listener to listen to the status change of the server.
+ *
+ * @param listener listener to add.
+ */
+ void addServerStatusChangeListener(ServerStatusChangeListener listener);
+
+ /**
+ * Get the status of the server.
+ *
+ * @return the status of the server.
+ */
+ ServerStatus getServerStatus();
+
+ /**
+ * Shutdown the server, release resources.
+ */
+ void shutdown();
+
+ enum ServerStatus {
+ ACTIVE,
+ STAND_BY,
+ ;
+ }
+
+}
diff --git a/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/ServerStatusChangeListener.java b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/ServerStatusChangeListener.java
new file mode 100644
index 0000000000..47df81570a
--- /dev/null
+++ b/streampark-registry/streampark-registry-api/src/main/java/org/apache/streampark/registry/api/ha/ServerStatusChangeListener.java
@@ -0,0 +1,24 @@
+/*
+ * 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.streampark.registry.api.ha;
+
+public interface ServerStatusChangeListener {
+
+ void change(HAServer.ServerStatus originStatus, HAServer.ServerStatus currentStatus);
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/pom.xml b/streampark-registry/streampark-registry-plugins/pom.xml
new file mode 100644
index 0000000000..8929ccffa6
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/pom.xml
@@ -0,0 +1,36 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry
+ 2.2.0-SNAPSHOT
+
+ streampark-registry-plugins
+ pom
+
+
+ streampark-registry-zookeeper
+ streampark-registry-jdbc
+ streampark-registry-it
+
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-it/pom.xml b/streampark-registry/streampark-registry-plugins/streampark-registry-it/pom.xml
new file mode 100644
index 0000000000..2342e17772
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-it/pom.xml
@@ -0,0 +1,61 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry-plugins
+ 2.2.0-SNAPSHOT
+
+
+ streampark-registry-it
+
+
+
+ org.apache.streampark
+ streampark-registry-api
+ ${project.version}
+
+
+ org.springframework.boot
+ spring-boot-starter-test
+ test
+
+
+
+
+
+
+ org.apache.maven.plugins
+ maven-jar-plugin
+
+ false
+
+
+
+
+ test-jar
+
+
+
+
+
+
+
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-it/src/test/java/org/apache/streampark/plugin/registry/RegistryTestCase.java b/streampark-registry/streampark-registry-plugins/streampark-registry-it/src/test/java/org/apache/streampark/plugin/registry/RegistryTestCase.java
new file mode 100644
index 0000000000..f2d878f921
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-it/src/test/java/org/apache/streampark/plugin/registry/RegistryTestCase.java
@@ -0,0 +1,262 @@
+/*
+ * 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.streampark.plugin.registry;
+
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.registry.api.ConnectionState;
+import org.apache.streampark.registry.api.Event;
+import org.apache.streampark.registry.api.Registry;
+import org.apache.streampark.registry.api.RegistryException;
+import org.apache.streampark.registry.api.SubscribeListener;
+
+import lombok.SneakyThrows;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.springframework.test.util.ReflectionTestUtils;
+
+import java.time.Duration;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.testcontainers.shaded.org.awaitility.Awaitility.await;
+
+public abstract class RegistryTestCase {
+
+ protected R registry;
+
+ @BeforeAll
+ public static void init() {
+ ReflectionTestUtils.setField(NetworkUtils.class, "priorityStrategy", "default");
+ }
+
+ @BeforeEach
+ public void setupRegistry() {
+
+ registry = createRegistry();
+ }
+
+ @SneakyThrows
+ @AfterEach
+ public void tearDownRegistry() {
+ try (R registry = this.registry) {
+ }
+ }
+
+ @Test
+ public void testIsConnected() {
+ registry.start();
+ assertThat(registry.isConnected()).isTrue();
+ }
+
+ @Test
+ public void testConnectUntilTimeout() {
+ registry.start();
+ await().atMost(Duration.ofSeconds(10))
+ .untilAsserted(() -> registry.connectUntilTimeout(Duration.ofSeconds(3)));
+
+ }
+
+ @SneakyThrows
+ @Test
+ public void testSubscribe() {
+ registry.start();
+
+ final AtomicBoolean subscribeAdded = new AtomicBoolean(false);
+ final AtomicBoolean subscribeRemoved = new AtomicBoolean(false);
+ final AtomicBoolean subscribeUpdated = new AtomicBoolean(false);
+
+ SubscribeListener subscribeListener = event -> {
+ System.out.println("Receive event: " + 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/console-server" + System.nanoTime();
+ registry.subscribe(key, subscribeListener);
+ registry.put(key, String.valueOf(System.nanoTime()), true);
+ // Sleep 3 seconds here since in mysql jdbc registry
+ // If multiple event occurs in a refresh time, only the last event will be triggered
+ Thread.sleep(3000);
+ registry.put(key, String.valueOf(System.nanoTime()), true);
+ Thread.sleep(3000);
+ registry.delete(key);
+
+ await().atMost(Duration.ofSeconds(10))
+ .untilAsserted(() -> {
+ Assertions.assertTrue(subscribeAdded.get());
+ Assertions.assertTrue(subscribeUpdated.get());
+ Assertions.assertTrue(subscribeRemoved.get());
+ });
+ }
+
+ @SneakyThrows
+ @Test
+ public void testAddConnectionStateListener() {
+
+ AtomicReference connectionState = new AtomicReference<>();
+ registry.addConnectionStateListener(connectionState::set);
+
+ assertThat(connectionState.get()).isNull();
+ registry.start();
+
+ await().atMost(Duration.ofSeconds(2))
+ .until(() -> ConnectionState.CONNECTED == connectionState.get());
+
+ }
+
+ @Test
+ public void testGet() {
+ registry.start();
+ String key = "/nodes/console-server" + System.nanoTime();
+ String value = "127.0.0.1:8080";
+ assertThrows(RegistryException.class, () -> registry.get(key));
+ registry.put(key, value, true);
+ assertThat(registry.get(key)).isEqualTo(value);
+ }
+
+ @Test
+ public void testPut() {
+ registry.start();
+ String key = "/nodes/console-server" + System.nanoTime();
+ String value = "127.0.0.1:8080";
+ registry.put(key, value, true);
+ assertThat(registry.get(key)).isEqualTo(value);
+
+ // Update the value
+ registry.put(key, "123", true);
+ assertThat(registry.get(key)).isEqualTo("123");
+ }
+
+ @Test
+ public void testDelete() {
+ registry.start();
+ String key = "/nodes/console-server" + System.nanoTime();
+ String value = "127.0.0.1:8080";
+ // Delete a non-existent key
+ registry.delete(key);
+
+ registry.put(key, value, true);
+ assertThat(registry.get(key)).isEqualTo(value);
+ registry.delete(key);
+ assertThat(registry.exists(key)).isFalse();
+
+ }
+
+ @Test
+ public void testChildren() {
+ registry.start();
+ String master1 = "/nodes/children/childGroup1/127.0.0.1:8080";
+ String master2 = "/nodes/children/childGroup1/127.0.0.2:8080";
+ String value = "123";
+ registry.put(master1, value, true);
+ registry.put(master2, value, true);
+ assertThat(registry.children("/nodes/children")).contains("childGroup1");
+ assertThat(registry.children("/nodes/children/childGroup1")).contains("127.0.0.1:8080",
+ "127.0.0.2:8080");
+ }
+
+ @Test
+ public void testExists() {
+ registry.start();
+ String key = "/nodes/console-server" + System.nanoTime();
+ String value = "123";
+ assertThat(registry.exists(key)).isFalse();
+ registry.put(key, value, true);
+ assertThat(registry.exists(key)).isTrue();
+
+ }
+
+ @SneakyThrows
+ @Test
+ public void testAcquireLock() {
+ registry.start();
+ String lockKey = "/lock" + System.nanoTime();
+
+ // 1. Acquire the lock at the main thread
+ assertThat(registry.acquireLock(lockKey)).isTrue();
+ // Acquire the lock at the main thread again
+ // It should acquire success
+ assertThat(registry.acquireLock(lockKey)).isTrue();
+
+ // Acquire the lock at another thread
+ // It should acquire failed
+ CompletableFuture acquireResult = CompletableFuture.supplyAsync(() -> registry.acquireLock(lockKey));
+ assertThrows(TimeoutException.class, () -> acquireResult.get(3000, TimeUnit.MILLISECONDS));
+
+ }
+
+ @SneakyThrows
+ @Test
+ public void testAcquireLock_withTimeout() {
+ registry.start();
+ String lockKey = "/lock" + System.nanoTime();
+ // 1. Acquire the lock in the main thread
+ assertThat(registry.acquireLock(lockKey, 3000)).isTrue();
+
+ // Acquire the lock in the main thread
+ // It should acquire success
+ assertThat(registry.acquireLock(lockKey, 3000)).isTrue();
+
+ // Acquire the lock at another thread
+ // It should acquire failed
+ CompletableFuture acquireResult =
+ CompletableFuture.supplyAsync(() -> registry.acquireLock(lockKey, 3000));
+ assertThat(acquireResult.get()).isFalse();
+
+ }
+
+ @SneakyThrows
+ @Test
+ public void testReleaseLock() {
+ registry.start();
+ String lockKey = "/lock" + System.nanoTime();
+ // 1. Acquire the lock in the main thread
+ assertThat(registry.acquireLock(lockKey, 3000)).isTrue();
+
+ // Acquire the lock at another thread
+ // It should acquire failed
+ CompletableFuture acquireResult =
+ CompletableFuture.supplyAsync(() -> registry.acquireLock(lockKey, 3000));
+ assertThat(acquireResult.get()).isFalse();
+
+ // 2. Release the lock in the main thread
+ assertThat(registry.releaseLock(lockKey)).isTrue();
+
+ // Acquire the lock at another thread
+ // It should acquire success
+ acquireResult = CompletableFuture.supplyAsync(() -> registry.acquireLock(lockKey, 3000));
+ assertThat(acquireResult.get()).isTrue();
+ }
+
+ public abstract R createRegistry();
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/README.md b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/README.md
new file mode 100644
index 0000000000..a7c21761eb
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/README.md
@@ -0,0 +1,86 @@
+# Introduction
+
+This module is the jdbc registry plugin module, this plugin will use jdbc as the registry center. Will use the database
+configuration same as streampark in api'yaml default.
+
+# How to use
+
+1. Initialize the database table
+
+- If you use Mysql you can directly execute the sql script `src/main/resources/mysql_registry_init.sql`.
+
+- If you use Postgresql you can directly execute the sql script `src/main/resources/postgresql_registry_init.sql`.
+
+2. Change the config
+
+You need to set the registry properties in master/worker/api's application.yml
+
+```yaml
+registry:
+ type: jdbc
+```
+
+After do this two steps, you can start your streampark cluster, your cluster will use mysql as registry center to
+store server metadata.
+
+NOTE: You need to add `mysql-connector-java.jar` into DS classpath if you use mysql database, since this plugin will not
+bundle this driver in distribution.
+You can get the detail
+about Initialize the
+Database.
+
+## Optional configuration
+
+```yaml
+registry:
+ type: jdbc
+ # 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 streampark.
+ hikari-config:
+ jdbc-url: jdbc:mysql://127.0.0.1:3306/streampark
+ username: root
+ password: root
+ maximum-pool-size: 5
+ connection-timeout: 9000
+ idle-timeout: 600000
+```
+
+## Use different database configuration for jdbc registry center
+
+You need to set the registry properties in master/worker/api's application.yml
+
+### Use Mysql as registry center
+
+```yaml
+registry:
+ type: jdbc
+ heartbeat-refresh-interval: 3s
+ session-timeout: 60s
+ hikari-config:
+ jdbc-url: jdbc:mysql://127.0.0.1:3306/streampark
+ username: root
+ password: root
+ maximum-pool-size: 5
+ connection-timeout: 9000
+ idle-timeout: 600000
+```
+
+### Use Postgresql as registry center
+
+```yaml
+registry:
+ type: jdbc
+ heartbeat-refresh-interval: 3s
+ session-timeout: 60s
+ hikari-config:
+ jdbc-url: jdbc:postgresql://localhost:5432/streampark
+ username: root
+ password: root
+ maximum-pool-size: 5
+ connection-timeout: 9000
+ idle-timeout: 600000
+```
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/pom.xml b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/pom.xml
new file mode 100644
index 0000000000..436cebd5b4
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/pom.xml
@@ -0,0 +1,113 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry-plugins
+ 2.2.0-SNAPSHOT
+
+
+ streampark-registry-jdbc
+
+
+
+ org.apache.streampark
+ streampark-registry-api
+ ${project.version}
+
+
+ org.apache.streampark
+ streampark-common_${scala.binary.version}
+
+
+ com.zaxxer
+ HikariCP
+
+
+
+ mysql
+ mysql-connector-java
+
+
+
+ org.postgresql
+ postgresql
+
+
+
+ org.slf4j
+ slf4j-api
+
+
+
+ com.baomidou
+ mybatis-plus
+
+
+
+ com.baomidou
+ mybatis-plus-boot-starter
+
+
+ org.apache.logging.log4j
+ log4j-to-slf4j
+
+
+
+
+
+ org.hibernate.validator
+ hibernate-validator
+
+
+
+ org.apache.streampark
+ streampark-registry-it
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.testcontainers
+ mysql
+
+
+
+ org.testcontainers
+ postgresql
+
+
+
+ org.apache.httpcomponents.client5
+ httpclient5
+
+
+
+ org.springframework.boot
+ spring-boot-starter-test
+ test
+
+
+
+
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistry.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistry.java
new file mode 100644
index 0000000000..325e8ca6eb
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistry.java
@@ -0,0 +1,270 @@
+/*
+ * 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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.plugin.registry.jdbc.client.JdbcRegistryClient;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.streampark.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.streampark.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.streampark.plugin.registry.jdbc.server.JdbcRegistryDataChangeListener;
+import org.apache.streampark.registry.api.ConnectionListener;
+import org.apache.streampark.registry.api.ConnectionState;
+import org.apache.streampark.registry.api.Event;
+import org.apache.streampark.registry.api.Registry;
+import org.apache.streampark.registry.api.RegistryException;
+import org.apache.streampark.registry.api.SubscribeListener;
+
+import org.apache.commons.lang3.StringUtils;
+
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+import java.time.Duration;
+import java.util.Collection;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * This is one of the implementation of {@link Registry}, with this implementation, you need to rely on mysql database to
+ * store the streampark master/worker's metadata and do the server registry/unRegistry.
+ */
+@Slf4j
+public final class JdbcRegistry implements Registry {
+
+ private final JdbcRegistryProperties jdbcRegistryProperties;
+ private final JdbcRegistryClient jdbcRegistryClient;
+
+ private final IJdbcRegistryServer jdbcRegistryServer;
+
+ JdbcRegistry(JdbcRegistryProperties jdbcRegistryProperties, IJdbcRegistryServer jdbcRegistryServer) {
+ this.jdbcRegistryProperties = jdbcRegistryProperties;
+ this.jdbcRegistryServer = jdbcRegistryServer;
+ this.jdbcRegistryClient = new JdbcRegistryClient(jdbcRegistryProperties, jdbcRegistryServer);
+ log.info("Initialize Jdbc Registry...");
+ }
+
+ @Override
+ public void start() {
+ log.info("Starting Jdbc Registry...");
+ jdbcRegistryServer.start();
+ jdbcRegistryClient.start();
+ log.info("Started Jdbc Registry...");
+ }
+
+ @Override
+ public boolean isConnected() {
+ return jdbcRegistryClient.isConnectivity();
+ }
+
+ @Override
+ public void connectUntilTimeout(@NonNull Duration timeout) throws RegistryException {
+ long beginTimeMillis = System.currentTimeMillis();
+ long endTimeMills = timeout.getSeconds() <= 0 ? Long.MAX_VALUE : beginTimeMillis + timeout.toMillis();
+ while (true) {
+ if (System.currentTimeMillis() > endTimeMills) {
+ throw new RegistryException(
+ String.format("Cannot connect to jdbc registry in %s s", timeout.getSeconds()));
+ }
+ if (jdbcRegistryClient.isConnectivity()) {
+ return;
+ }
+ try {
+ Thread.sleep(jdbcRegistryProperties.getHeartbeatRefreshInterval().toMillis());
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ throw new RegistryException("Cannot connect to jdbc registry due to interrupted exception", e);
+ }
+ }
+ }
+
+ @Override
+ public void subscribe(String path, SubscribeListener listener) {
+ checkNotNull(path);
+ checkNotNull(listener);
+ jdbcRegistryClient.subscribeJdbcRegistryDataChange(new JdbcRegistryDataChangeListener() {
+
+ @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) {
+ 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
+ // Directly get from the db?
+ Optional jdbcRegistryDataOptional = jdbcRegistryClient.getJdbcRegistryDataByKey(key);
+ if (!jdbcRegistryDataOptional.isPresent()) {
+ throw new RegistryException("key: " + key + " not exist");
+ }
+ return jdbcRegistryDataOptional.get().getDataValue();
+ } catch (RegistryException registryException) {
+ throw registryException;
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Get key: %s error", key), e);
+ }
+ }
+
+ @Override
+ public void put(String key, String value, boolean deleteOnDisconnect) {
+ try {
+ 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);
+ }
+ }
+
+ @Override
+ public void delete(String key) {
+ try {
+ jdbcRegistryClient.deleteJdbcRegistryDataByKey(key);
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Delete key: %s error", key), e);
+ }
+ }
+
+ @Override
+ public Collection children(String key) {
+ try {
+ 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), "/"))
+ .distinct()
+ .collect(Collectors.toList());
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Get key: %s children error", key), e);
+ }
+ }
+
+ @Override
+ public boolean exists(String key) {
+ try {
+ return jdbcRegistryClient.existJdbcRegistryDataKey(key);
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Check key: %s exist error", key), e);
+ }
+ }
+
+ @Override
+ public boolean acquireLock(String key) {
+ try {
+ jdbcRegistryClient.acquireJdbcRegistryLock(key);
+ return true;
+ } catch (RegistryException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Acquire lock: %s error", key), e);
+ }
+ }
+
+ @Override
+ public boolean acquireLock(String key, long timeout) {
+ try {
+ return jdbcRegistryClient.acquireJdbcRegistryLock(key, timeout);
+ } catch (RegistryException e) {
+ throw e;
+ } catch (Exception e) {
+ throw new RegistryException(String.format("Acquire lock: %s error", key), e);
+ }
+ }
+
+ @Override
+ public boolean releaseLock(String key) {
+ jdbcRegistryClient.releaseJdbcRegistryLock(key);
+ return true;
+ }
+
+ @Override
+ public void close() {
+ log.info("Closing Jdbc Registry...");
+ // remove the current Ephemeral node, if can connect to jdbc
+ try (JdbcRegistryClient closed1 = jdbcRegistryClient) {
+ } catch (Exception e) {
+ log.error("Close Jdbc Registry error", e);
+ }
+ log.info("Closed Jdbc Registry...");
+ }
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java
new file mode 100644
index 0000000000..ecb52d47dc
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryAutoConfiguration.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryClientHeartbeatMapper;
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryDataChanceEventMapper;
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryDataMapper;
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryLockMapper;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryClientRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+import org.apache.streampark.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.streampark.plugin.registry.jdbc.server.JdbcRegistryServer;
+
+import org.apache.ibatis.session.SqlSessionFactory;
+
+import com.baomidou.mybatisplus.autoconfigure.MybatisPlusAutoConfiguration;
+import com.baomidou.mybatisplus.extension.spring.MybatisSqlSessionFactoryBean;
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.extern.slf4j.Slf4j;
+import org.mybatis.spring.SqlSessionTemplate;
+import org.mybatis.spring.annotation.MapperScan;
+import org.springframework.boot.autoconfigure.AutoConfigureAfter;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnMissingBean;
+import org.springframework.boot.autoconfigure.condition.ConditionalOnProperty;
+import org.springframework.context.annotation.Bean;
+import org.springframework.context.annotation.ComponentScan;
+import org.springframework.context.annotation.Configuration;
+
+@Slf4j
+@ComponentScan
+@Configuration(proxyBeanMethods = false)
+@MapperScan("org.apache.streampark.plugin.registry.jdbc.mapper")
+@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "jdbc")
+@AutoConfigureAfter(MybatisPlusAutoConfiguration.class)
+public class JdbcRegistryAutoConfiguration {
+
+ public JdbcRegistryAutoConfiguration() {
+ log.info("Load JdbcRegistryAutoConfiguration");
+ }
+
+ @Bean
+ 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,
+ IJdbcRegistryServer jdbcRegistryServer) {
+ JdbcRegistry jdbcRegistry = new JdbcRegistry(jdbcRegistryProperties, jdbcRegistryServer);
+ jdbcRegistry.start();
+ return jdbcRegistry;
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ public SqlSessionFactory sqlSessionFactory(JdbcRegistryProperties jdbcRegistryProperties) throws Exception {
+ log.info("Initialize jdbcRegistrySqlSessionFactory");
+ MybatisSqlSessionFactoryBean sqlSessionFactoryBean = new MybatisSqlSessionFactoryBean();
+ sqlSessionFactoryBean.setDataSource(new HikariDataSource(jdbcRegistryProperties.getHikariConfig()));
+ return sqlSessionFactoryBean.getObject();
+ }
+
+ @Bean
+ @ConditionalOnMissingBean
+ public SqlSessionTemplate sqlSessionTemplate(SqlSessionFactory jdbcRegistrySqlSessionFactory) {
+ log.info("Initialize jdbcRegistrySqlSessionTemplate");
+ return new SqlSessionTemplate(jdbcRegistrySqlSessionFactory);
+ }
+
+ @Bean
+ public JdbcRegistryDataMapper jdbcRegistryDataMapper(SqlSessionTemplate jdbcRegistrySqlSessionTemplate) {
+ jdbcRegistrySqlSessionTemplate.getConfiguration().addMapper(JdbcRegistryDataMapper.class);
+ return jdbcRegistrySqlSessionTemplate.getMapper(JdbcRegistryDataMapper.class);
+ }
+
+ @Bean
+ public JdbcRegistryLockMapper jdbcRegistryLockMapper(SqlSessionTemplate jdbcRegistrySqlSessionTemplate) {
+ jdbcRegistrySqlSessionTemplate.getConfiguration().addMapper(JdbcRegistryLockMapper.class);
+ 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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryProperties.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryProperties.java
new file mode 100644
index 0000000000..3e9bdda674
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryProperties.java
@@ -0,0 +1,95 @@
+/*
+ * 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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.common.utils.NetworkUtils;
+
+import org.apache.commons.lang3.StringUtils;
+
+import com.zaxxer.hikari.HikariConfig;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.beans.factory.annotation.Autowired;
+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 java.time.Duration;
+
+@Data
+@Slf4j
+@Validated
+@Configuration
+@ConditionalOnProperty(prefix = "registry", name = "type", havingValue = "jdbc")
+@ConfigurationProperties(prefix = "registry")
+public class JdbcRegistryProperties implements Validator {
+
+ @Autowired
+ private NetworkUtils networkUtils;
+
+ private static final Duration MIN_HEARTBEAT_REFRESH_INTERVAL = Duration.ofSeconds(1);
+
+ @Value("${server.port:10000}")
+ 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 = NetworkUtils.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryThreadFactory.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryThreadFactory.java
new file mode 100644
index 0000000000..63b201e288
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/LockUtils.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/LockUtils.java
new file mode 100644
index 0000000000..a486040dcc
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/LockUtils.java
@@ -0,0 +1,34 @@
+/*
+ * 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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.common.utils.OSUtils;
+
+import lombok.experimental.UtilityClass;
+
+@UtilityClass
+public class LockUtils {
+
+ private static final String LOCK_OWNER_PREFIX = NetworkUtils.getHost() + "_" + OSUtils.getProcessID() + "_";
+
+ public static String getLockOwner() {
+ return LOCK_OWNER_PREFIX + Thread.currentThread().getName();
+ }
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/IJdbcRegistryClient.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/IJdbcRegistryClient.java
new file mode 100644
index 0000000000..e02d0f388b
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.client;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.streampark.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.streampark.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 = "/streampark/master", and data exist in db is "/streampark/master/master1", "/streampark/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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/JdbcRegistryClient.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/JdbcRegistryClient.java
new file mode 100644
index 0000000000..925188a79d
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.client;
+
+import org.apache.streampark.common.utils.NetworkUtils;
+import org.apache.streampark.common.utils.OSUtils;
+import org.apache.streampark.common.utils.UUIDUtils;
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.streampark.plugin.registry.jdbc.server.ConnectionStateListener;
+import org.apache.streampark.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.streampark.plugin.registry.jdbc.server.JdbcRegistryDataChangeListener;
+import org.apache.streampark.plugin.registry.jdbc.server.JdbcRegistryServerState;
+
+import lombok.extern.slf4j.Slf4j;
+
+import java.util.List;
+import java.util.Optional;
+
+/**
+ * The client of jdbc registry, used to interact with the {@link org.apache.streampark.plugin.registry.jdbc.server.JdbcRegistryServer}.
+ */
+@Slf4j
+public class JdbcRegistryClient implements IJdbcRegistryClient {
+
+ private static final String DEFAULT_CLIENT_NAME = NetworkUtils.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(UUIDUtils.generateUUID(), 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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/JdbcRegistryClientIdentify.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/client/JdbcRegistryClientIdentify.java
new file mode 100644
index 0000000000..c54450103a
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryClientHeartbeatMapper.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryClientHeartbeatMapper.java
new file mode 100644
index 0000000000..371a1e6148
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.mapper;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+
+import org.apache.ibatis.annotations.Select;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+import java.util.List;
+
+public interface JdbcRegistryClientHeartbeatMapper extends BaseMapper {
+
+ @Select("select * from t_jdbc_registry_client_heartbeat")
+ List selectAll();
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryDataChanceEventMapper.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryDataChanceEventMapper.java
new file mode 100644
index 0000000000..dc336f0d1f
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.mapper;
+
+import org.apache.streampark.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 com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+import java.util.Date;
+import java.util.List;
+
+public interface JdbcRegistryDataChanceEventMapper extends BaseMapper {
+
+ @Select("select max(id) from t_jdbc_registry_data_change_event")
+ Long getMaxId();
+
+ @Select("select * from t_jdbc_registry_data_change_event where id > #{id} order by id asc limit 1000")
+ List selectJdbcRegistryDataChangeEventWhereIdAfter(@Param("id") long id);
+
+ @Delete("delete from t_jdbc_registry_data_change_event where create_time > #{createTime}")
+ void deleteJdbcRegistryDataChangeEventBeforeCreateTime(@Param("createTime") Date createTime);
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java
new file mode 100644
index 0000000000..c21e3304ca
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryDataMapper.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.plugin.registry.jdbc.mapper;
+
+import org.apache.streampark.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 com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+import java.util.List;
+
+public interface JdbcRegistryDataMapper extends BaseMapper {
+
+ @Select("select * from t_jdbc_registry_data")
+ List selectAll();
+
+ @Select("select * from t_jdbc_registry_data where data_key = #{key}")
+ JdbcRegistryData selectByKey(@Param("key") String key);
+
+ @Delete("delete from t_jdbc_registry_data where data_key = #{key}")
+ void deleteByKey(@Param("key") String key);
+
+ @Delete({""})
+ void deleteByClientIds(@Param("clientIds") List clientIds, @Param("dataType") String dataType);
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java
new file mode 100644
index 0000000000..7fd5e33b7b
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/mapper/JdbcRegistryLockMapper.java
@@ -0,0 +1,39 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.mapper;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
+
+import org.apache.ibatis.annotations.Delete;
+import org.apache.ibatis.annotations.Param;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+import java.util.List;
+
+public interface JdbcRegistryLockMapper extends BaseMapper {
+
+ @Delete({""})
+ void deleteByClientIds(@Param("clientIds") List clientIds);
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java
new file mode 100644
index 0000000000..15356bbe21
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryClientHeartbeat.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.plugin.registry.jdbc.model.DO;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@TableName(value = "t_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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryData.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryData.java
new file mode 100644
index 0000000000..d527e2d6c4
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryData.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.streampark.plugin.registry.jdbc.model.DO;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@TableName(value = "t_jdbc_registry_data")
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryData {
+
+ @TableId(value = "id", type = IdType.AUTO)
+ private Long id;
+ private String dataKey;
+ private String dataValue;
+ private String dataType;
+ private long clientId;
+ private Date createTime;
+ private Date lastUpdateTime;
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.java
new file mode 100644
index 0000000000..30d410e229
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryDataChanceEvent.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.streampark.plugin.registry.jdbc.model.DO;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@TableName(value = "t_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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryLock.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryLock.java
new file mode 100644
index 0000000000..835b6147f4
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DO/JdbcRegistryLock.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.streampark.plugin.registry.jdbc.model.DO;
+
+import com.baomidou.mybatisplus.annotation.IdType;
+import com.baomidou.mybatisplus.annotation.TableId;
+import com.baomidou.mybatisplus.annotation.TableName;
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@TableName(value = "t_jdbc_registry_lock")
+@Data
+@Builder
+@NoArgsConstructor
+@AllArgsConstructor
+public class JdbcRegistryLock {
+
+ @TableId(value = "id", type = IdType.AUTO)
+ private Long id;
+ private String lockKey;
+ private String lockOwner;
+ private Long clientId;
+ private Date createTime;
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/DataType.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/DataType.java
new file mode 100644
index 0000000000..4d06dc254e
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/DataType.java
@@ -0,0 +1,25 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.model.DTO;
+
+public enum DataType {
+
+ EPHEMERAL(),
+ PERSISTENT(),
+ ;
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryClientHeartbeatDTO.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryClientHeartbeatDTO.java
new file mode 100644
index 0000000000..6371f0ec08
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.model.DTO;
+
+import org.apache.streampark.common.utils.JSONUtils;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.SneakyThrows;
+
+import java.util.Date;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryDataChanceEventDTO.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryDataChanceEventDTO.java
new file mode 100644
index 0000000000..f96ff7c84c
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.model.DTO;
+
+import org.apache.streampark.common.utils.JSONUtils;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryDataChanceEvent;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryDataDTO.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryDataDTO.java
new file mode 100644
index 0000000000..f76150f487
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.model.DTO;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryLockDTO.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/model/DTO/JdbcRegistryLockDTO.java
new file mode 100644
index 0000000000..1c87e497fe
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.model.DTO;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+
+import java.util.Date;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java
new file mode 100644
index 0000000000..40570e611c
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryClientRepository.java
@@ -0,0 +1,66 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.repository;
+
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryClientHeartbeatMapper;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryClientHeartbeat;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryClientHeartbeatDTO;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+import org.springframework.util.CollectionUtils;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryDataChanceEventRepository.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryDataChanceEventRepository.java
new file mode 100644
index 0000000000..5a94a9651b
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.repository;
+
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryDataChanceEventMapper;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryDataChanceEvent;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataChanceEventDTO;
+
+import org.springframework.stereotype.Repository;
+
+import java.util.Date;
+import java.util.List;
+import java.util.stream.Collectors;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java
new file mode 100644
index 0000000000..4cf7b5ceb4
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryDataRepository.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.plugin.registry.jdbc.repository;
+
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryDataMapper;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryData;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+import org.springframework.util.CollectionUtils;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.java
new file mode 100644
index 0000000000..e8804bdd0f
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/repository/JdbcRegistryLockRepository.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.streampark.plugin.registry.jdbc.repository;
+
+import org.apache.streampark.plugin.registry.jdbc.mapper.JdbcRegistryLockMapper;
+import org.apache.streampark.plugin.registry.jdbc.model.DO.JdbcRegistryLock;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryLockDTO;
+
+import com.baomidou.mybatisplus.core.toolkit.CollectionUtils;
+import org.springframework.beans.factory.annotation.Autowired;
+import org.springframework.stereotype.Repository;
+
+import java.util.List;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/ConnectionStateListener.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/ConnectionStateListener.java
new file mode 100644
index 0000000000..b589387c06
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/ConnectionStateListener.java
@@ -0,0 +1,27 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.server;
+
+public interface ConnectionStateListener {
+
+ void onConnected();
+
+ void onDisConnected();
+
+ void onReconnected();
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryDataManager.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryDataManager.java
new file mode 100644
index 0000000000..c32f80c07e
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.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 = "/streampark/master", and data exist in db is "/streampark/master/master1", "/streampark/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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryLockManager.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryLockManager.java
new file mode 100644
index 0000000000..f4017d7214
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryServer.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IJdbcRegistryServer.java
new file mode 100644
index 0000000000..fdc9026dfa
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.plugin.registry.jdbc.client.IJdbcRegistryClient;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.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.streampark.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 = "/streampark/master", and data exist in db is "/streampark/master/master1", "/streampark/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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IRegistryRowChangeNotifier.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/IRegistryRowChangeNotifier.java
new file mode 100644
index 0000000000..828fe2343b
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryDataChangeListener.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryDataChangeListener.java
new file mode 100644
index 0000000000..ed26584db5
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryDataManager.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryDataManager.java
new file mode 100644
index 0000000000..7e066f3f23
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryDataManager.java
@@ -0,0 +1,264 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryThreadFactory;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataChanceEventDTO;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+
+import org.apache.commons.lang3.time.DateUtils;
+
+import com.google.common.collect.Lists;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.util.CollectionUtils;
+
+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 static com.google.common.base.Preconditions.checkNotNull;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryLockManager.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryLockManager.java
new file mode 100644
index 0000000000..a9fe4e3544
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryLockManager.java
@@ -0,0 +1,148 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.common.thread.ThreadUtils;
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.streampark.plugin.registry.jdbc.LockUtils;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryLockDTO;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+
+import lombok.AllArgsConstructor;
+import lombok.Builder;
+import lombok.Data;
+import lombok.NoArgsConstructor;
+import lombok.extern.slf4j.Slf4j;
+import org.springframework.dao.DuplicateKeyException;
+
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Map;
+
+@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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryServer.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryServer.java
new file mode 100644
index 0000000000..0515852041
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryServer.java
@@ -0,0 +1,386 @@
+/*
+ * 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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryProperties;
+import org.apache.streampark.plugin.registry.jdbc.JdbcRegistryThreadFactory;
+import org.apache.streampark.plugin.registry.jdbc.client.IJdbcRegistryClient;
+import org.apache.streampark.plugin.registry.jdbc.client.JdbcRegistryClientIdentify;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.DataType;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryClientHeartbeatDTO;
+import org.apache.streampark.plugin.registry.jdbc.model.DTO.JdbcRegistryDataDTO;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryClientRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataChanceEventRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryDataRepository;
+import org.apache.streampark.plugin.registry.jdbc.repository.JdbcRegistryLockRepository;
+import org.apache.streampark.registry.api.RegistryException;
+
+import org.apache.commons.collections4.CollectionUtils;
+
+import com.google.common.collect.Lists;
+import lombok.SneakyThrows;
+import lombok.extern.slf4j.Slf4j;
+
+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 static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * 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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryServerState.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/plugin/registry/jdbc/server/JdbcRegistryServerState.java
new file mode 100644
index 0000000000..f01ccb554e
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/java/org/apache/streampark/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.streampark.plugin.registry.jdbc.server;
+
+import org.apache.streampark.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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/META-INF/spring.factories b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/META-INF/spring.factories
new file mode 100644
index 0000000000..f29a793a45
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/META-INF/spring.factories
@@ -0,0 +1,19 @@
+#
+# 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.
+#
+
+org.springframework.boot.autoconfigure.EnableAutoConfiguration=\
+ org.apache.streampark.plugin.registry.jdbc.JdbcRegistryAutoConfiguration
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/mysql_registry_init.sql b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/mysql_registry_init.sql
new file mode 100644
index 0000000000..4ee6c47a52
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/mysql_registry_init.sql
@@ -0,0 +1,69 @@
+/*
+ * 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.
+*/
+
+
+DROP TABLE IF EXISTS `t_jdbc_registry_data`;
+CREATE TABLE `t_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_jdbc_registry_dataKey` (`data_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+
+DROP TABLE IF EXISTS `t_jdbc_registry_lock`;
+CREATE TABLE `t_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_jdbc_registry_lockKey` (`lock_key`)
+) ENGINE = InnoDB
+ DEFAULT CHARSET = utf8;
+
+DROP TABLE IF EXISTS `t_jdbc_registry_client_heartbeat`;
+CREATE TABLE `t_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_jdbc_registry_data_change_event`;
+CREATE TABLE `t_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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/postgresql_registry_init.sql b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/postgresql_registry_init.sql
new file mode 100644
index 0000000000..dcfd82074a
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/main/resources/postgresql_registry_init.sql
@@ -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.
+*/
+
+DROP TABLE IF EXISTS t_jdbc_registry_data;
+create table t_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_jdbc_registry_dataKey on t_jdbc_registry_data (data_key);
+
+
+DROP TABLE IF EXISTS t_jdbc_registry_lock;
+create table t_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_jdbc_registry_lockKey on t_jdbc_registry_lock (lock_key);
+
+
+DROP TABLE IF EXISTS t_jdbc_registry_client_heartbeat;
+create table t_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_jdbc_registry_data_change_event;
+create table t_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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryTestCase.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryTestCase.java
new file mode 100644
index 0000000000..e2cef1ade7
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryTestCase.java
@@ -0,0 +1,64 @@
+/*
+ * 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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.common.CommonConfiguration;
+import org.apache.streampark.plugin.registry.RegistryTestCase;
+import org.apache.streampark.plugin.registry.jdbc.server.IJdbcRegistryServer;
+import org.apache.streampark.registry.api.ConnectionState;
+
+import lombok.SneakyThrows;
+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 java.util.concurrent.atomic.AtomicReference;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+@SpringBootTest(classes = {CommonConfiguration.class, JdbcRegistryProperties.class})
+@SpringBootApplication(scanBasePackageClasses = {CommonConfiguration.class, JdbcRegistryProperties.class})
+@DirtiesContext(classMode = DirtiesContext.ClassMode.BEFORE_EACH_TEST_METHOD)
+public abstract class JdbcRegistryTestCase extends RegistryTestCase {
+
+ @Autowired
+ private JdbcRegistryProperties jdbcRegistryProperties;
+
+ @Autowired
+ private IJdbcRegistryServer jdbcRegistryServer;
+
+ @SneakyThrows
+ @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.
+ assertThat(connectionState.get()).isEqualTo(null);
+ }
+
+ @Override
+ public JdbcRegistry createRegistry() {
+ return new JdbcRegistry(jdbcRegistryProperties, jdbcRegistryServer);
+ }
+
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryThreadFactoryTest.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/JdbcRegistryThreadFactoryTest.java
new file mode 100644
index 0000000000..484dd1fd34
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/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.streampark.plugin.registry.jdbc;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+import static org.testcontainers.shaded.org.awaitility.Awaitility.await;
+
+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/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/LockUtilsTest.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/LockUtilsTest.java
new file mode 100644
index 0000000000..6e107c7ff9
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/LockUtilsTest.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.streampark.plugin.registry.jdbc;
+
+import org.junit.jupiter.api.Test;
+
+import static org.assertj.core.api.AssertionsForClassTypes.assertThat;
+
+class LockUtilsTest {
+
+ @Test
+ void getLockOwner() {
+ assertThat(LockUtils.getLockOwner()).isNotNull();
+ }
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java
new file mode 100644
index 0000000000..eebbf43410
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/MysqlJdbcRegistryTestCase.java
@@ -0,0 +1,77 @@
+/*
+ * 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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.common.sql.SqlScriptRunner;
+
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.SneakyThrows;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.springframework.test.context.ActiveProfiles;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.MySQLContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.wait.strategy.Wait;
+import org.testcontainers.lifecycle.Startables;
+import org.testcontainers.utility.DockerImageName;
+
+import java.time.Duration;
+import java.util.stream.Stream;
+
+@ActiveProfiles("mysql")
+class MysqlJdbcRegistryTestCase extends JdbcRegistryTestCase {
+
+ private static GenericContainer> mysqlContainer;
+
+ @SneakyThrows
+ @BeforeAll
+ public static void setUpTestingServer() {
+ mysqlContainer = new MySQLContainer(DockerImageName.parse("mysql:8.0"))
+ .withUsername("root")
+ .withPassword("root")
+ .withDatabaseName("streampark")
+ .withNetwork(Network.newNetwork())
+ .withExposedPorts(3306)
+ .waitingFor(Wait.forHealthcheck().withStartupTimeout(Duration.ofSeconds(300)));
+
+ Startables.deepStart(Stream.of(mysqlContainer)).join();
+
+ String jdbcUrl = "jdbc:mysql://localhost:" + mysqlContainer.getMappedPort(3306)
+ + "/streampark?useSSL=false&serverTimezone=UTC";
+ System.clearProperty("spring.datasource.url");
+ System.setProperty("spring.datasource.url", jdbcUrl);
+
+ 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();
+ }
+ }
+
+ @SneakyThrows
+ @AfterAll
+ public static void tearDownTestingServer() {
+ mysqlContainer.close();
+ }
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.java
new file mode 100644
index 0000000000..e3cbb00029
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/java/org/apache/streampark/plugin/registry/jdbc/PostgresqlJdbcRegistryTestCase.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.streampark.plugin.registry.jdbc;
+
+import org.apache.streampark.common.sql.SqlScriptRunner;
+
+import com.zaxxer.hikari.HikariConfig;
+import com.zaxxer.hikari.HikariDataSource;
+import lombok.SneakyThrows;
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.springframework.test.context.ActiveProfiles;
+import org.testcontainers.containers.GenericContainer;
+import org.testcontainers.containers.Network;
+import org.testcontainers.containers.PostgreSQLContainer;
+import org.testcontainers.lifecycle.Startables;
+import org.testcontainers.utility.DockerImageName;
+
+import java.util.stream.Stream;
+
+@ActiveProfiles("postgresql")
+public class PostgresqlJdbcRegistryTestCase extends JdbcRegistryTestCase {
+
+ private static GenericContainer> postgresqlContainer;
+
+ @SneakyThrows
+ @BeforeAll
+ public static void setUpTestingServer() {
+ postgresqlContainer = new PostgreSQLContainer(DockerImageName.parse("postgres:16.0"))
+ .withUsername("root")
+ .withPassword("root")
+ .withDatabaseName("streampark")
+ .withNetwork(Network.newNetwork())
+ .withExposedPorts(5432);
+
+ Startables.deepStart(Stream.of(postgresqlContainer)).join();
+
+ String jdbcUrl = "jdbc:postgresql://localhost:" + postgresqlContainer.getMappedPort(5432) + "/streampark";
+ System.clearProperty("spring.datasource.url");
+ System.setProperty("spring.datasource.url", jdbcUrl);
+
+ 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();
+ }
+ }
+
+ @SneakyThrows
+ @AfterAll
+ public static void tearDownTestingServer() {
+ postgresqlContainer.close();
+ }
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-mysql.yaml b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-mysql.yaml
new file mode 100644
index 0000000000..779b43a2cc
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-mysql.yaml
@@ -0,0 +1,38 @@
+#
+# 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.
+#
+spring:
+ sql:
+ init:
+ schema-locations: classpath:mysql_registry_init.sql
+ datasource:
+ driver-class-name: com.mysql.cj.jdbc.Driver
+ url: jdbc:mysql://localhost:3306/streampark?useUnicode=true&characterEncoding=UTF-8
+ username: root
+ password: root
+
+registry:
+ type: jdbc
+ heartbeat-refresh-interval: 1s
+ session-timeout: 3s
+
+network:
+ # network interface preferred like eth0, default: empty
+ preferred-interface: ""
+ # network interface restricted like docker0, default: empty
+ restrict-interface: docker0
+ # network IP gets priority, default inner outer
+ priority-strategy: default
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-postgresql.yaml b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-postgresql.yaml
new file mode 100644
index 0000000000..e5fbb5a56a
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/application-postgresql.yaml
@@ -0,0 +1,36 @@
+#
+# 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.
+#
+spring:
+ datasource:
+ driver-class-name: org.postgresql.Driver
+ url: jdbc:postgresql://localhost:5432/streampark
+ username: root
+ password: root
+
+registry:
+ type: jdbc
+ heartbeat-refresh-interval: 1s
+ session-timeout: 3s
+
+network:
+ # network interface preferred like eth0, default: empty
+ preferred-interface: ""
+ # network interface restricted like docker0, default: empty
+ restrict-interface: docker0
+ # network IP gets priority, default inner outer
+ priority-strategy: default
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/logback.xml b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/logback.xml
new file mode 100644
index 0000000000..6f211959c5
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-jdbc/src/test/resources/logback.xml
@@ -0,0 +1,21 @@
+
+
+
+
+
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/README.md b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/README.md
new file mode 100644
index 0000000000..b7c0131ca0
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/README.md
@@ -0,0 +1,27 @@
+# Introduction
+
+This module is the zookeeper registry plugin module, this plugin will use zookeeper as the registry center.
+
+# How to use
+
+If you want to set the registry center as zookeeper,you need to set the registry properties in master/worker/api's application.yml
+
+```yaml
+registry:
+ type: zookeeper
+ zookeeper:
+ namespace: streampark
+ connect-string: localhost:2181
+ retry-policy:
+ base-sleep-time: 60ms
+ max-sleep: 300ms
+ max-retries: 5
+ session-timeout: 30s
+ connection-timeout: 9s
+ block-until-connected: 600ms
+ # The following options are set according to personal needs
+ digest: ~
+```
+
+After do this config, you can start your streampark cluster, your cluster will use zookeeper as registry center to
+store server metadata.
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/pom.xml b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/pom.xml
new file mode 100644
index 0000000000..cc0934e9f6
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/pom.xml
@@ -0,0 +1,90 @@
+
+
+
+ 4.0.0
+
+ org.apache.streampark
+ streampark-registry-plugins
+ 2.2.0-SNAPSHOT
+
+
+ streampark-registry-zookeeper
+
+
+
+ org.apache.streampark
+ streampark-registry-api
+ ${project.version}
+
+
+
+ org.apache.streampark
+ streampark-registry-it
+ ${project.version}
+ test-jar
+ test
+
+
+
+ org.testcontainers
+ testcontainers
+
+
+
+ org.apache.zookeeper
+ zookeeper
+
+
+
+ org.apache.curator
+ curator-framework
+
+
+
+ org.apache.curator
+ curator-client
+
+
+
+ org.apache.curator
+ curator-recipes
+
+
+
+ io.dropwizard.metrics
+ metrics-core
+
+
+ org.xerial.snappy
+ snappy-java
+
+
+
+ org.apache.curator
+ curator-test
+ test
+
+
+
+ org.springframework.boot
+ spring-boot-starter-test
+ test
+
+
+
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperConnectionStateListener.java b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperConnectionStateListener.java
new file mode 100644
index 0000000000..ee18ecd046
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperConnectionStateListener.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.streampark.plugin.registry.zookeeper;
+
+import org.apache.streampark.registry.api.ConnectionListener;
+import org.apache.streampark.registry.api.ConnectionState;
+
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.state.ConnectionStateListener;
+
+import lombok.extern.slf4j.Slf4j;
+
+@Slf4j
+final class ZookeeperConnectionStateListener implements ConnectionStateListener {
+
+ private final ConnectionListener listener;
+
+ ZookeeperConnectionStateListener(ConnectionListener listener) {
+ this.listener = listener;
+ }
+
+ @Override
+ public void stateChanged(CuratorFramework client,
+ org.apache.curator.framework.state.ConnectionState newState) {
+ switch (newState) {
+ case CONNECTED:
+ log.info("Registry connected");
+ listener.onUpdate(ConnectionState.CONNECTED);
+ break;
+ case LOST:
+ log.warn("Registry disconnected");
+ listener.onUpdate(ConnectionState.DISCONNECTED);
+ break;
+ case RECONNECTED:
+ log.info("Registry reconnected");
+ listener.onUpdate(ConnectionState.RECONNECTED);
+ break;
+ case SUSPENDED:
+ log.warn("Registry suspended");
+ listener.onUpdate(ConnectionState.SUSPENDED);
+ break;
+ default:
+ break;
+ }
+ }
+}
diff --git a/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperRegistry.java b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperRegistry.java
new file mode 100644
index 0000000000..a711d1d14a
--- /dev/null
+++ b/streampark-registry/streampark-registry-plugins/streampark-registry-zookeeper/src/main/java/org/apache/streampark/plugin/registry/zookeeper/ZookeeperRegistry.java
@@ -0,0 +1,328 @@
+/*
+ * 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.streampark.plugin.registry.zookeeper;
+
+import org.apache.streampark.registry.api.ConnectionListener;
+import org.apache.streampark.registry.api.Event;
+import org.apache.streampark.registry.api.Registry;
+import org.apache.streampark.registry.api.RegistryException;
+import org.apache.streampark.registry.api.SubscribeListener;
+
+import org.apache.commons.lang3.time.DurationUtils;
+import org.apache.curator.framework.CuratorFramework;
+import org.apache.curator.framework.CuratorFrameworkFactory;
+import org.apache.curator.framework.api.ACLProvider;
+import org.apache.curator.framework.recipes.cache.ChildData;
+import org.apache.curator.framework.recipes.cache.TreeCache;
+import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
+import org.apache.curator.framework.recipes.locks.InterProcessMutex;
+import org.apache.curator.retry.ExponentialBackoffRetry;
+import org.apache.curator.utils.CloseableUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+
+import com.google.common.base.Strings;
+import lombok.NonNull;
+import lombok.extern.slf4j.Slf4j;
+
+import java.nio.charset.StandardCharsets;
+import java.time.Duration;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+
+@Slf4j
+final class ZookeeperRegistry implements Registry {
+
+ private final ZookeeperRegistryProperties.ZookeeperProperties properties;
+ private final CuratorFramework client;
+
+ private final Map treeCacheMap = new ConcurrentHashMap<>();
+
+ private static final ThreadLocal