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