diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
index 9932219e9e..f8ba88773e 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
@@ -18,6 +18,7 @@ package org.apache.dolphinscheduler.common;
import org.apache.dolphinscheduler.common.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.utils.OSUtils;
+import org.apache.dolphinscheduler.common.utils.StringUtils;
import java.util.regex.Pattern;
@@ -1003,4 +1004,10 @@ public final class Constants {
*/
public static final String PSTREE = "pstree";
+ /**
+ * docker & kubernetes
+ */
+ public static final boolean DOCKER_MODE = StringUtils.isNotEmpty(System.getenv("DOCKER"));
+ public static final boolean KUBERNETES_MODE = StringUtils.isNotEmpty(System.getenv("KUBERNETES_SERVICE_HOST")) && StringUtils.isNotEmpty(System.getenv("KUBERNETES_SERVICE_PORT"));
+
}
diff --git a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java
index c5fd7cd02d..7b255c2e63 100644
--- a/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java
+++ b/dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java
@@ -26,8 +26,6 @@ import oshi.hardware.CentralProcessor;
import oshi.hardware.GlobalMemory;
import oshi.hardware.HardwareAbstractionLayer;
-import org.apache.commons.configuration.Configuration;
-
import java.lang.management.OperatingSystemMXBean;
import java.io.BufferedReader;
import java.io.FileInputStream;
@@ -406,19 +404,45 @@ public class OSUtils {
return Integer.parseInt(runtimeMXBean.getName().split("@")[0]);
}
+ /**
+ * get local addr
+ * @return addr like host:port
+ */
+ public static String getAddr(int port) {
+ return getAddr(getHost(), port);
+ }
+
+ /**
+ * get addr
+ * @return addr like host:port
+ */
+ public static String getAddr(String host, int port) {
+ return String.format("%s:%d", host, port);
+ }
+
/**
* get local host
* @return host
*/
public static String getHost(){
try {
- return InetAddress.getLocalHost().getHostAddress();
+ return getHost(InetAddress.getLocalHost());
} catch (UnknownHostException e) {
logger.error(e.getMessage(),e);
}
return null;
}
+ /**
+ * get local host
+ * @return host
+ */
+ public static String getHost(InetAddress inetAddress){
+ if (inetAddress != null) {
+ return Constants.KUBERNETES_MODE ? inetAddress.getHostName() : inetAddress.getHostAddress();
+ }
+ return null;
+ }
/**
* whether is macOS
diff --git a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
index ac81722b74..cdda544b96 100644
--- a/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
+++ b/dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
@@ -88,6 +88,12 @@ public class OSUtilsTest {
Assert.assertNotEquals(0, processId);
}
@Test
+ public void getAddr(){
+ Assert.assertEquals(OSUtils.getHost() + ":5678", OSUtils.getAddr(5678));
+ Assert.assertEquals("127.0.0.1:5678", OSUtils.getAddr("127.0.0.1", 5678));
+ Assert.assertEquals("localhost:1234", OSUtils.getAddr("localhost", 1234));
+ }
+ @Test
public void getHost(){
String host = OSUtils.getHost();
Assert.assertNotNull(host);
diff --git a/dolphinscheduler-remote/pom.xml b/dolphinscheduler-remote/pom.xml
index 855f2d2c5a..0531ccf993 100644
--- a/dolphinscheduler-remote/pom.xml
+++ b/dolphinscheduler-remote/pom.xml
@@ -34,6 +34,10 @@
+
+ org.apache.dolphinscheduler
+ dolphinscheduler-common
+
io.netty
netty-all
@@ -51,6 +55,5 @@
junit
test
-
diff --git a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java
index 138a8f0bdf..bfa30825fb 100644
--- a/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java
+++ b/dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java
@@ -16,6 +16,8 @@
*/
package org.apache.dolphinscheduler.remote.utils;
+import org.apache.dolphinscheduler.common.utils.OSUtils;
+
import io.netty.channel.Channel;
import java.net.InetSocketAddress;
@@ -32,7 +34,7 @@ public class ChannelUtils {
* @return local address
*/
public static String getLocalAddress(Channel channel){
- return ((InetSocketAddress)channel.localAddress()).getAddress().getHostAddress();
+ return OSUtils.getHost(((InetSocketAddress)channel.localAddress()).getAddress());
}
/**
@@ -41,7 +43,7 @@ public class ChannelUtils {
* @return remote address
*/
public static String getRemoteAddress(Channel channel){
- return ((InetSocketAddress)channel.remoteAddress()).getAddress().getHostAddress();
+ return OSUtils.getHost(((InetSocketAddress)channel.remoteAddress()).getAddress());
}
/**
@@ -51,7 +53,7 @@ public class ChannelUtils {
*/
public static Host toAddress(Channel channel){
InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress());
- return new Host(socketAddress.getAddress().getHostAddress(), socketAddress.getPort());
+ return new Host(OSUtils.getHost(socketAddress.getAddress()), socketAddress.getPort());
}
}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java
index 4ce8166c6c..0f41ba5246 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java
@@ -26,7 +26,6 @@ import javax.annotation.PostConstruct;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
@@ -40,8 +39,6 @@ import org.springframework.stereotype.Service;
import com.google.common.collect.Sets;
-import static org.apache.dolphinscheduler.remote.utils.Constants.COMMA;
-
/**
* master registry
*/
@@ -135,7 +132,7 @@ public class MasterRegistry {
* @return
*/
private String getLocalAddress(){
- return OSUtils.getHost() + Constants.COLON + masterConfig.getListenPort();
+ return OSUtils.getAddr(masterConfig.getListenPort());
}
}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/ConditionsTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/ConditionsTaskExecThread.java
index 1b65fbf06f..8521dc9379 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/ConditionsTaskExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/ConditionsTaskExecThread.java
@@ -122,7 +122,7 @@ public class ConditionsTaskExecThread extends MasterBaseTaskExecThread {
private void initTaskParameters() {
this.taskInstance.setLogPath(getTaskLogPath(taskInstance));
- this.taskInstance.setHost(OSUtils.getHost() + Constants.COLON + masterConfig.getListenPort());
+ this.taskInstance.setHost(OSUtils.getAddr(masterConfig.getListenPort()));
taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
taskInstance.setStartTime(new Date());
this.processService.saveTaskInstance(taskInstance);
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/DependentTaskExecThread.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/DependentTaskExecThread.java
index d4d87b9587..db4f239b36 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/DependentTaskExecThread.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/DependentTaskExecThread.java
@@ -174,7 +174,7 @@ public class DependentTaskExecThread extends MasterBaseTaskExecThread {
private void initTaskParameters() {
taskInstance.setLogPath(getTaskLogPath(taskInstance));
- taskInstance.setHost(OSUtils.getHost() + Constants.COLON + masterConfig.getListenPort());
+ taskInstance.setHost(OSUtils.getAddr(masterConfig.getListenPort()));
taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
taskInstance.setStartTime(new Date());
processService.updateTaskInstance(taskInstance);
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
index 87e16596b4..33a7f4ff1a 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
@@ -158,6 +158,6 @@ public class MasterSchedulerService extends Thread {
}
private String getLocalAddress(){
- return OSUtils.getHost() + ":" + masterConfig.getListenPort();
+ return OSUtils.getAddr(masterConfig.getListenPort());
}
}
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
index f8b95678ec..d3d9045687 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
@@ -126,7 +126,7 @@ public class TaskExecuteProcessor implements NettyRequestProcessor {
taskExecutionContext.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId()));
- taskExecutionContext.setHost(OSUtils.getHost() + ":" + workerConfig.getListenPort());
+ taskExecutionContext.setHost(OSUtils.getAddr(workerConfig.getListenPort()));
taskExecutionContext.setStartTime(new Date());
taskExecutionContext.setLogPath(getTaskLogPath(taskExecutionContext));
diff --git a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
index e2ed55c41e..921d0defbb 100644
--- a/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
+++ b/dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
@@ -30,7 +30,6 @@ import javax.annotation.PostConstruct;
import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener;
-import org.apache.dolphinscheduler.common.Constants;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
@@ -44,7 +43,6 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service;
import com.google.common.collect.Sets;
-import com.google.common.collect.Sets;
/**
@@ -164,7 +162,7 @@ public class WorkerRegistry {
* @return local address
*/
private String getLocalAddress(){
- return OSUtils.getHost() + Constants.COLON + workerConfig.getListenPort();
+ return OSUtils.getAddr(workerConfig.getListenPort());
}
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java
index 5955f46056..de5d052120 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java
@@ -78,7 +78,7 @@ public class NettyExecutorManagerTest {
.buildProcessDefinitionRelatedInfo(processDefinition)
.create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER);
- executionContext.setHost(Host.of(OSUtils.getHost() + ":" + serverConfig.getListenPort()));
+ executionContext.setHost(Host.of(OSUtils.getAddr(OSUtils.getHost(), serverConfig.getListenPort())));
Boolean execute = nettyExecutorManager.execute(executionContext);
Assert.assertTrue(execute);
nettyRemotingServer.close();
@@ -97,7 +97,7 @@ public class NettyExecutorManagerTest {
.buildProcessDefinitionRelatedInfo(processDefinition)
.create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER);
- executionContext.setHost(Host.of(OSUtils.getHost() + ":4444"));
+ executionContext.setHost(Host.of(OSUtils.getAddr(OSUtils.getHost(), 4444)));
nettyExecutorManager.execute(executionContext);
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java
index e223a762dd..98f81fe6a1 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java
@@ -73,6 +73,6 @@ public class RoundRobinHostManagerTest {
ExecutionContext context = ExecutionContextTestUtils.getExecutionContext(10000);
Host host = roundRobinHostManager.select(context);
Assert.assertTrue(StringUtils.isNotEmpty(host.getAddress()));
- Assert.assertTrue(host.getAddress().equalsIgnoreCase(OSUtils.getHost() + ":" + workerConfig.getListenPort()));
+ Assert.assertTrue(host.getAddress().equalsIgnoreCase(OSUtils.getAddr(workerConfig.getListenPort())));
}
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java
index 9d90f20706..1a90c8be16 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java
@@ -17,6 +17,7 @@
package org.apache.dolphinscheduler.server.master.registry;
+import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.remote.utils.Constants;
import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
@@ -55,7 +56,7 @@ public class MasterRegistryTest {
masterRegistry.registry();
String masterPath = zookeeperRegistryCenter.getMasterPath();
TimeUnit.SECONDS.sleep(masterConfig.getMasterHeartbeatInterval() + 2); //wait heartbeat info write into zk node
- String masterNodePath = masterPath + "/" + (Constants.LOCAL_ADDRESS + ":" + masterConfig.getListenPort());
+ String masterNodePath = masterPath + "/" + (OSUtils.getAddr(Constants.LOCAL_ADDRESS, masterConfig.getListenPort()));
String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(masterNodePath);
Assert.assertEquals(HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH, heartbeat.split(",").length);
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java
index c99dfc1c9f..ae1f08f0e7 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java
@@ -74,7 +74,7 @@ public class ZookeeperNodeManagerTest {
Set masterNodes = zookeeperNodeManager.getMasterNodes();
Assert.assertTrue(CollectionUtils.isNotEmpty(masterNodes));
Assert.assertEquals(1, masterNodes.size());
- Assert.assertEquals(OSUtils.getHost() + ":" + masterConfig.getListenPort(), masterNodes.iterator().next());
+ Assert.assertEquals(OSUtils.getAddr(masterConfig.getListenPort()), masterNodes.iterator().next());
}
@Test
@@ -102,6 +102,6 @@ public class ZookeeperNodeManagerTest {
Set workerNodes = zookeeperNodeManager.getWorkerGroupNodes("default");
Assert.assertTrue(CollectionUtils.isNotEmpty(workerNodes));
Assert.assertEquals(1, workerNodes.size());
- Assert.assertEquals(OSUtils.getHost() + ":" + workerConfig.getListenPort(), workerNodes.iterator().next());
+ Assert.assertEquals(OSUtils.getAddr(workerConfig.getListenPort()), workerNodes.iterator().next());
}
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java
index 26d904f798..8facc1aa87 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java
@@ -47,7 +47,7 @@ public class ExecutionContextTestUtils {
.buildProcessDefinitionRelatedInfo(processDefinition)
.create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER);
- executionContext.setHost(Host.of(OSUtils.getHost() + ":" + port));
+ executionContext.setHost(Host.of(OSUtils.getAddr(OSUtils.getHost(), port)));
return executionContext;
}
diff --git a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java
index 6e92d6abc8..98a78e93c8 100644
--- a/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java
+++ b/dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java
@@ -113,7 +113,7 @@ public class WorkerRegistryTest {
int i = 0;
for (String workerGroup : workerConfig.getWorkerGroups()) {
- String workerZkPath = workerPath + "/" + workerGroup.trim() + "/" + (OSUtils.getHost() + ":" + workerConfig.getListenPort());
+ String workerZkPath = workerPath + "/" + workerGroup.trim() + "/" + (OSUtils.getAddr(workerConfig.getListenPort()));
String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(workerZkPath);
if (0 == i) {
Assert.assertTrue(workerZkPath.startsWith("/dolphinscheduler/nodes/worker/test/"));
diff --git a/sql/dolphinscheduler-postgre.sql b/sql/dolphinscheduler-postgre.sql
index 64c50bdae3..e628e27d80 100644
--- a/sql/dolphinscheduler-postgre.sql
+++ b/sql/dolphinscheduler-postgre.sql
@@ -329,7 +329,7 @@ CREATE TABLE t_ds_process_instance (
start_time timestamp DEFAULT NULL ,
end_time timestamp DEFAULT NULL ,
run_times int DEFAULT NULL ,
- host varchar(45) DEFAULT NULL ,
+ host varchar(150) DEFAULT NULL ,
command_type int DEFAULT NULL ,
command_param text ,
task_depend_type int DEFAULT NULL ,
@@ -556,7 +556,7 @@ CREATE TABLE t_ds_task_instance (
submit_time timestamp DEFAULT NULL ,
start_time timestamp DEFAULT NULL ,
end_time timestamp DEFAULT NULL ,
- host varchar(45) DEFAULT NULL ,
+ host varchar(135) DEFAULT NULL ,
execute_path varchar(200) DEFAULT NULL ,
log_path varchar(200) DEFAULT NULL ,
alert_flag int DEFAULT NULL ,
diff --git a/sql/dolphinscheduler_mysql.sql b/sql/dolphinscheduler_mysql.sql
index f932c58dcb..96db47f9ed 100644
--- a/sql/dolphinscheduler_mysql.sql
+++ b/sql/dolphinscheduler_mysql.sql
@@ -436,7 +436,7 @@ CREATE TABLE `t_ds_process_instance` (
`start_time` datetime DEFAULT NULL COMMENT 'process instance start time',
`end_time` datetime DEFAULT NULL COMMENT 'process instance end time',
`run_times` int(11) DEFAULT NULL COMMENT 'process instance run times',
- `host` varchar(45) DEFAULT NULL COMMENT 'process instance host',
+ `host` varchar(135) DEFAULT NULL COMMENT 'process instance host',
`command_type` tinyint(4) DEFAULT NULL COMMENT 'command type',
`command_param` text COMMENT 'json command parameters',
`task_depend_type` tinyint(4) DEFAULT NULL COMMENT 'task depend type. 0: only current node,1:before the node,2:later nodes',
@@ -695,7 +695,7 @@ CREATE TABLE `t_ds_task_instance` (
`submit_time` datetime DEFAULT NULL COMMENT 'task submit time',
`start_time` datetime DEFAULT NULL COMMENT 'task start time',
`end_time` datetime DEFAULT NULL COMMENT 'task end time',
- `host` varchar(45) DEFAULT NULL COMMENT 'host of task running on',
+ `host` varchar(135) DEFAULT NULL COMMENT 'host of task running on',
`execute_path` varchar(200) DEFAULT NULL COMMENT 'task execute path in the host',
`log_path` varchar(200) DEFAULT NULL COMMENT 'task log path',
`alert_flag` tinyint(4) DEFAULT NULL COMMENT 'whether alert',
diff --git a/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_ddl.sql b/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_ddl.sql
new file mode 100644
index 0000000000..5635073390
--- /dev/null
+++ b/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_ddl.sql
@@ -0,0 +1,58 @@
+/*
+ * 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.
+*/
+
+SET sql_mode=(SELECT REPLACE(@@sql_mode,'ONLY_FULL_GROUP_BY',''));
+
+-- uc_dolphin_T_t_ds_process_instance_R_host
+drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_process_instance_R_host;
+delimiter d//
+CREATE PROCEDURE uc_dolphin_T_t_ds_process_instance_R_host()
+BEGIN
+ IF EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_process_instance'
+ AND TABLE_SCHEMA=(SELECT DATABASE())
+ AND COLUMN_NAME ='host')
+ THEN
+ ALTER TABLE t_ds_process_instance MODIFY COLUMN `host` varchar(135);
+ END IF;
+END;
+
+d//
+
+delimiter ;
+CALL uc_dolphin_T_t_ds_process_instance_R_host;
+DROP PROCEDURE uc_dolphin_T_t_ds_process_instance_R_host;
+
+-- uc_dolphin_T_t_ds_task_instance_R_host
+drop PROCEDURE if EXISTS uc_dolphin_T_t_ds_task_instance_R_host;
+delimiter d//
+CREATE PROCEDURE uc_dolphin_T_t_ds_task_instance_R_host()
+BEGIN
+ IF EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_task_instance'
+ AND TABLE_SCHEMA=(SELECT DATABASE())
+ AND COLUMN_NAME ='host')
+ THEN
+ ALTER TABLE t_ds_task_instance MODIFY COLUMN `host` varchar(135);
+ END IF;
+END;
+
+d//
+
+delimiter ;
+CALL uc_dolphin_T_t_ds_task_instance_R_host;
+DROP PROCEDURE uc_dolphin_T_t_ds_task_instance_R_host;
diff --git a/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_dml.sql b/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_dml.sql
new file mode 100644
index 0000000000..38964cc551
--- /dev/null
+++ b/sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_dml.sql
@@ -0,0 +1,16 @@
+/*
+ * 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.
+*/
\ No newline at end of file
diff --git a/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_ddl.sql b/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_ddl.sql
new file mode 100644
index 0000000000..6dd15f7b07
--- /dev/null
+++ b/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_ddl.sql
@@ -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.
+*/
+
+-- uc_dolphin_T_t_ds_process_instance_A_host
+delimiter d//
+CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_process_instance_A_host() RETURNS void AS $$
+BEGIN
+ IF EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_process_instance'
+ AND COLUMN_NAME ='host')
+ THEN
+ ALTER TABLE t_ds_process_instance ALTER COLUMN host type varchar(135);
+ END IF;
+END;
+$$ LANGUAGE plpgsql;
+d//
+
+delimiter ;
+SELECT uc_dolphin_T_t_ds_process_instance_A_host();
+DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_process_instance_A_host();
+
+-- uc_dolphin_T_t_ds_task_instance_A_host
+delimiter d//
+CREATE OR REPLACE FUNCTION uc_dolphin_T_t_ds_task_instance_A_host() RETURNS void AS $$
+BEGIN
+ IF EXISTS (SELECT 1 FROM information_schema.COLUMNS
+ WHERE TABLE_NAME='t_ds_task_instance'
+ AND COLUMN_NAME ='host')
+ THEN
+ ALTER TABLE t_ds_task_instance ALTER COLUMN host type varchar(135);
+ END IF;
+END;
+$$ LANGUAGE plpgsql;
+d//
+
+delimiter ;
+SELECT uc_dolphin_T_t_ds_task_instance_A_host();
+DROP FUNCTION IF EXISTS uc_dolphin_T_t_ds_task_instance_A_host();
diff --git a/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_dml.sql b/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_dml.sql
new file mode 100644
index 0000000000..38964cc551
--- /dev/null
+++ b/sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_dml.sql
@@ -0,0 +1,16 @@
+/*
+ * 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.
+*/
\ No newline at end of file