Browse Source

[1.3.5-prepare][Fix-3298][K8s]Fix task log disappear after recreating or upgrading helm release (#4703)

* [1.3.5-prepare][Improvement][K8s] Alter column host varchar(15) to varchar(135) for long host

* [1.3.5-prepare][Fix][K8s] Fix task log disappear after recreating or upgrading helm release
Shiwen Cheng 4 years ago committed by GitHub
parent
commit
2722bffc91
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
  1. 7
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/Constants.java
  2. 30
      dolphinscheduler-common/src/main/java/org/apache/dolphinscheduler/common/utils/OSUtils.java
  3. 6
      dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java
  4. 5
      dolphinscheduler-remote/pom.xml
  5. 8
      dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java
  6. 5
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistry.java
  7. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/ConditionsTaskExecThread.java
  8. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/DependentTaskExecThread.java
  9. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/master/runner/MasterSchedulerService.java
  10. 2
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/processor/TaskExecuteProcessor.java
  11. 4
      dolphinscheduler-server/src/main/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistry.java
  12. 4
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java
  13. 2
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/host/RoundRobinHostManagerTest.java
  14. 3
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java
  15. 4
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java
  16. 2
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java
  17. 2
      dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java
  18. 4
      sql/dolphinscheduler-postgre.sql
  19. 4
      sql/dolphinscheduler_mysql.sql
  20. 58
      sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_ddl.sql
  21. 16
      sql/upgrade/1.3.5_schema/mysql/dolphinscheduler_dml.sql
  22. 52
      sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_ddl.sql
  23. 16
      sql/upgrade/1.3.5_schema/postgresql/dolphinscheduler_dml.sql

7
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.enums.ExecutionStatus;
import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils;
import java.util.regex.Pattern; import java.util.regex.Pattern;
@ -1003,4 +1004,10 @@ public final class Constants {
*/ */
public static final String PSTREE = "pstree"; 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"));
} }

30
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.GlobalMemory;
import oshi.hardware.HardwareAbstractionLayer; import oshi.hardware.HardwareAbstractionLayer;
import org.apache.commons.configuration.Configuration;
import java.lang.management.OperatingSystemMXBean; import java.lang.management.OperatingSystemMXBean;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.FileInputStream; import java.io.FileInputStream;
@ -406,19 +404,45 @@ public class OSUtils {
return Integer.parseInt(runtimeMXBean.getName().split("@")[0]); 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 * get local host
* @return host * @return host
*/ */
public static String getHost(){ public static String getHost(){
try { try {
return InetAddress.getLocalHost().getHostAddress(); return getHost(InetAddress.getLocalHost());
} catch (UnknownHostException e) { } catch (UnknownHostException e) {
logger.error(e.getMessage(),e); logger.error(e.getMessage(),e);
} }
return null; 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 * whether is macOS

6
dolphinscheduler-common/src/test/java/org/apache/dolphinscheduler/common/utils/OSUtilsTest.java

@ -88,6 +88,12 @@ public class OSUtilsTest {
Assert.assertNotEquals(0, processId); Assert.assertNotEquals(0, processId);
} }
@Test @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(){ public void getHost(){
String host = OSUtils.getHost(); String host = OSUtils.getHost();
Assert.assertNotNull(host); Assert.assertNotNull(host);

5
dolphinscheduler-remote/pom.xml

@ -34,6 +34,10 @@
</properties> </properties>
<dependencies> <dependencies>
<dependency>
<groupId>org.apache.dolphinscheduler</groupId>
<artifactId>dolphinscheduler-common</artifactId>
</dependency>
<dependency> <dependency>
<groupId>io.netty</groupId> <groupId>io.netty</groupId>
<artifactId>netty-all</artifactId> <artifactId>netty-all</artifactId>
@ -51,6 +55,5 @@
<artifactId>junit</artifactId> <artifactId>junit</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
</dependencies> </dependencies>
</project> </project>

8
dolphinscheduler-remote/src/main/java/org/apache/dolphinscheduler/remote/utils/ChannelUtils.java

@ -16,6 +16,8 @@
*/ */
package org.apache.dolphinscheduler.remote.utils; package org.apache.dolphinscheduler.remote.utils;
import org.apache.dolphinscheduler.common.utils.OSUtils;
import io.netty.channel.Channel; import io.netty.channel.Channel;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
@ -32,7 +34,7 @@ public class ChannelUtils {
* @return local address * @return local address
*/ */
public static String getLocalAddress(Channel channel){ 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 * @return remote address
*/ */
public static String getRemoteAddress(Channel channel){ 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){ public static Host toAddress(Channel channel){
InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress()); InetSocketAddress socketAddress = ((InetSocketAddress)channel.remoteAddress());
return new Host(socketAddress.getAddress().getHostAddress(), socketAddress.getPort()); return new Host(OSUtils.getHost(socketAddress.getAddress()), socketAddress.getPort());
} }
} }

5
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.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener; 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.DateUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory; import org.apache.dolphinscheduler.remote.utils.NamedThreadFactory;
@ -40,8 +39,6 @@ import org.springframework.stereotype.Service;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import static org.apache.dolphinscheduler.remote.utils.Constants.COMMA;
/** /**
* master registry * master registry
*/ */
@ -135,7 +132,7 @@ public class MasterRegistry {
* @return * @return
*/ */
private String getLocalAddress(){ private String getLocalAddress(){
return OSUtils.getHost() + Constants.COLON + masterConfig.getListenPort(); return OSUtils.getAddr(masterConfig.getListenPort());
} }
} }

2
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() { private void initTaskParameters() {
this.taskInstance.setLogPath(getTaskLogPath(taskInstance)); 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.setState(ExecutionStatus.RUNNING_EXEUTION);
taskInstance.setStartTime(new Date()); taskInstance.setStartTime(new Date());
this.processService.saveTaskInstance(taskInstance); this.processService.saveTaskInstance(taskInstance);

2
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() { private void initTaskParameters() {
taskInstance.setLogPath(getTaskLogPath(taskInstance)); taskInstance.setLogPath(getTaskLogPath(taskInstance));
taskInstance.setHost(OSUtils.getHost() + Constants.COLON + masterConfig.getListenPort()); taskInstance.setHost(OSUtils.getAddr(masterConfig.getListenPort()));
taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION); taskInstance.setState(ExecutionStatus.RUNNING_EXEUTION);
taskInstance.setStartTime(new Date()); taskInstance.setStartTime(new Date());
processService.updateTaskInstance(taskInstance); processService.updateTaskInstance(taskInstance);

2
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(){ private String getLocalAddress(){
return OSUtils.getHost() + ":" + masterConfig.getListenPort(); return OSUtils.getAddr(masterConfig.getListenPort());
} }
} }

2
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.getProcessInstanceId(),
taskExecutionContext.getTaskInstanceId())); taskExecutionContext.getTaskInstanceId()));
taskExecutionContext.setHost(OSUtils.getHost() + ":" + workerConfig.getListenPort()); taskExecutionContext.setHost(OSUtils.getAddr(workerConfig.getListenPort()));
taskExecutionContext.setStartTime(new Date()); taskExecutionContext.setStartTime(new Date());
taskExecutionContext.setLogPath(getTaskLogPath(taskExecutionContext)); taskExecutionContext.setLogPath(getTaskLogPath(taskExecutionContext));

4
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.CuratorFramework;
import org.apache.curator.framework.state.ConnectionState; import org.apache.curator.framework.state.ConnectionState;
import org.apache.curator.framework.state.ConnectionStateListener; 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.DateUtils;
import org.apache.dolphinscheduler.common.utils.OSUtils; import org.apache.dolphinscheduler.common.utils.OSUtils;
import org.apache.dolphinscheduler.common.utils.StringUtils; import org.apache.dolphinscheduler.common.utils.StringUtils;
@ -44,7 +43,6 @@ import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.stereotype.Service; import org.springframework.stereotype.Service;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.google.common.collect.Sets;
/** /**
@ -164,7 +162,7 @@ public class WorkerRegistry {
* @return local address * @return local address
*/ */
private String getLocalAddress(){ private String getLocalAddress(){
return OSUtils.getHost() + Constants.COLON + workerConfig.getListenPort(); return OSUtils.getAddr(workerConfig.getListenPort());
} }
} }

4
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/dispatch/executor/NettyExecutorManagerTest.java

@ -78,7 +78,7 @@ public class NettyExecutorManagerTest {
.buildProcessDefinitionRelatedInfo(processDefinition) .buildProcessDefinitionRelatedInfo(processDefinition)
.create(); .create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER); 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); Boolean execute = nettyExecutorManager.execute(executionContext);
Assert.assertTrue(execute); Assert.assertTrue(execute);
nettyRemotingServer.close(); nettyRemotingServer.close();
@ -97,7 +97,7 @@ public class NettyExecutorManagerTest {
.buildProcessDefinitionRelatedInfo(processDefinition) .buildProcessDefinitionRelatedInfo(processDefinition)
.create(); .create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER); 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); nettyExecutorManager.execute(executionContext);
} }

2
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); ExecutionContext context = ExecutionContextTestUtils.getExecutionContext(10000);
Host host = roundRobinHostManager.select(context); Host host = roundRobinHostManager.select(context);
Assert.assertTrue(StringUtils.isNotEmpty(host.getAddress())); Assert.assertTrue(StringUtils.isNotEmpty(host.getAddress()));
Assert.assertTrue(host.getAddress().equalsIgnoreCase(OSUtils.getHost() + ":" + workerConfig.getListenPort())); Assert.assertTrue(host.getAddress().equalsIgnoreCase(OSUtils.getAddr(workerConfig.getListenPort())));
} }
} }

3
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/master/registry/MasterRegistryTest.java

@ -17,6 +17,7 @@
package org.apache.dolphinscheduler.server.master.registry; 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.remote.utils.Constants;
import org.apache.dolphinscheduler.server.master.config.MasterConfig; import org.apache.dolphinscheduler.server.master.config.MasterConfig;
import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter; import org.apache.dolphinscheduler.server.registry.ZookeeperRegistryCenter;
@ -55,7 +56,7 @@ public class MasterRegistryTest {
masterRegistry.registry(); masterRegistry.registry();
String masterPath = zookeeperRegistryCenter.getMasterPath(); String masterPath = zookeeperRegistryCenter.getMasterPath();
TimeUnit.SECONDS.sleep(masterConfig.getMasterHeartbeatInterval() + 2); //wait heartbeat info write into zk node 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); String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(masterNodePath);
Assert.assertEquals(HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH, heartbeat.split(",").length); Assert.assertEquals(HEARTBEAT_FOR_ZOOKEEPER_INFO_LENGTH, heartbeat.split(",").length);
} }

4
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/registry/ZookeeperNodeManagerTest.java

@ -74,7 +74,7 @@ public class ZookeeperNodeManagerTest {
Set<String> masterNodes = zookeeperNodeManager.getMasterNodes(); Set<String> masterNodes = zookeeperNodeManager.getMasterNodes();
Assert.assertTrue(CollectionUtils.isNotEmpty(masterNodes)); Assert.assertTrue(CollectionUtils.isNotEmpty(masterNodes));
Assert.assertEquals(1, masterNodes.size()); Assert.assertEquals(1, masterNodes.size());
Assert.assertEquals(OSUtils.getHost() + ":" + masterConfig.getListenPort(), masterNodes.iterator().next()); Assert.assertEquals(OSUtils.getAddr(masterConfig.getListenPort()), masterNodes.iterator().next());
} }
@Test @Test
@ -102,6 +102,6 @@ public class ZookeeperNodeManagerTest {
Set<String> workerNodes = zookeeperNodeManager.getWorkerGroupNodes("default"); Set<String> workerNodes = zookeeperNodeManager.getWorkerGroupNodes("default");
Assert.assertTrue(CollectionUtils.isNotEmpty(workerNodes)); Assert.assertTrue(CollectionUtils.isNotEmpty(workerNodes));
Assert.assertEquals(1, workerNodes.size()); Assert.assertEquals(1, workerNodes.size());
Assert.assertEquals(OSUtils.getHost() + ":" + workerConfig.getListenPort(), workerNodes.iterator().next()); Assert.assertEquals(OSUtils.getAddr(workerConfig.getListenPort()), workerNodes.iterator().next());
} }
} }

2
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/utils/ExecutionContextTestUtils.java

@ -47,7 +47,7 @@ public class ExecutionContextTestUtils {
.buildProcessDefinitionRelatedInfo(processDefinition) .buildProcessDefinitionRelatedInfo(processDefinition)
.create(); .create();
ExecutionContext executionContext = new ExecutionContext(context.toCommand(), ExecutorType.WORKER); 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; return executionContext;
} }

2
dolphinscheduler-server/src/test/java/org/apache/dolphinscheduler/server/worker/registry/WorkerRegistryTest.java

@ -113,7 +113,7 @@ public class WorkerRegistryTest {
int i = 0; int i = 0;
for (String workerGroup : workerConfig.getWorkerGroups()) { 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); String heartbeat = zookeeperRegistryCenter.getZookeeperCachedOperator().get(workerZkPath);
if (0 == i) { if (0 == i) {
Assert.assertTrue(workerZkPath.startsWith("/dolphinscheduler/nodes/worker/test/")); Assert.assertTrue(workerZkPath.startsWith("/dolphinscheduler/nodes/worker/test/"));

4
sql/dolphinscheduler-postgre.sql

@ -329,7 +329,7 @@ CREATE TABLE t_ds_process_instance (
start_time timestamp DEFAULT NULL , start_time timestamp DEFAULT NULL ,
end_time timestamp DEFAULT NULL , end_time timestamp DEFAULT NULL ,
run_times int DEFAULT NULL , run_times int DEFAULT NULL ,
host varchar(45) DEFAULT NULL , host varchar(150) DEFAULT NULL ,
command_type int DEFAULT NULL , command_type int DEFAULT NULL ,
command_param text , command_param text ,
task_depend_type int DEFAULT NULL , task_depend_type int DEFAULT NULL ,
@ -556,7 +556,7 @@ CREATE TABLE t_ds_task_instance (
submit_time timestamp DEFAULT NULL , submit_time timestamp DEFAULT NULL ,
start_time timestamp DEFAULT NULL , start_time timestamp DEFAULT NULL ,
end_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 , execute_path varchar(200) DEFAULT NULL ,
log_path varchar(200) DEFAULT NULL , log_path varchar(200) DEFAULT NULL ,
alert_flag int DEFAULT NULL , alert_flag int DEFAULT NULL ,

4
sql/dolphinscheduler_mysql.sql

@ -436,7 +436,7 @@ CREATE TABLE `t_ds_process_instance` (
`start_time` datetime DEFAULT NULL COMMENT 'process instance start time', `start_time` datetime DEFAULT NULL COMMENT 'process instance start time',
`end_time` datetime DEFAULT NULL COMMENT 'process instance end time', `end_time` datetime DEFAULT NULL COMMENT 'process instance end time',
`run_times` int(11) DEFAULT NULL COMMENT 'process instance run times', `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_type` tinyint(4) DEFAULT NULL COMMENT 'command type',
`command_param` text COMMENT 'json command parameters', `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', `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', `submit_time` datetime DEFAULT NULL COMMENT 'task submit time',
`start_time` datetime DEFAULT NULL COMMENT 'task start time', `start_time` datetime DEFAULT NULL COMMENT 'task start time',
`end_time` datetime DEFAULT NULL COMMENT 'task end 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', `execute_path` varchar(200) DEFAULT NULL COMMENT 'task execute path in the host',
`log_path` varchar(200) DEFAULT NULL COMMENT 'task log path', `log_path` varchar(200) DEFAULT NULL COMMENT 'task log path',
`alert_flag` tinyint(4) DEFAULT NULL COMMENT 'whether alert', `alert_flag` tinyint(4) DEFAULT NULL COMMENT 'whether alert',

58
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;

16
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.
*/

52
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();

16
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.
*/
Loading…
Cancel
Save