You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dolphinscheduler.apache.org by le...@apache.org on 2021/01/20 06:39:23 UTC

[incubator-dolphinscheduler] branch json_split updated: [Feature][JsonSplit] task mapper and postgre (#4498)

This is an automated email from the ASF dual-hosted git repository.

leonbao pushed a commit to branch json_split
in repository https://gitbox.apache.org/repos/asf/incubator-dolphinscheduler.git


The following commit(s) were added to refs/heads/json_split by this push:
     new c184b89  [Feature][JsonSplit] task mapper and postgre (#4498)
c184b89 is described below

commit c184b89f3e622034ebfe6d7a8ccdcf4e59a985be
Author: JinyLeeChina <42...@users.noreply.github.com>
AuthorDate: Wed Jan 20 14:39:09 2021 +0800

    [Feature][JsonSplit] task mapper and postgre (#4498)
    
    * task and relation entity
    
    * task and relation entity
    
    * task and relation entity
    
    * task and relation entity
    
    * task and relation entity
    
    * add timeout flag
    
    * task mapper and postgre
    
    * task mapper and postgre
    
    * 'postgre'
    
    Co-authored-by: JinyLeeChina <29...@qq.com>
---
 .../dao/entity/TaskDefinition.java                 |  13 ++
 .../dao/entity/TaskDefinitionLog.java              |  13 ++
 .../dao/mapper/TaskDefinitionLogMapper.java        |  50 +++++++
 .../dao/mapper/TaskDefinitionMapper.java           | 103 ++++++++++++++
 .../dao/mapper/TaskDefinitionLogMapper.xml         |  42 ++++++
 .../dao/mapper/TaskDefinitionMapper.xml            |  95 +++++++++++++
 sql/dolphinscheduler-postgre.sql                   | 156 +++++++++++++++++++--
 sql/dolphinscheduler_mysql.sql                     |   2 +
 8 files changed, 463 insertions(+), 11 deletions(-)

diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
index 3789fed..1158aae 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinition.java
@@ -151,6 +151,11 @@ public class TaskDefinition {
     private int timeout;
 
     /**
+     * resource ids
+     */
+    private String resourceIds;
+
+    /**
      * create time
      */
     @JsonFormat(pattern = "yyyy-MM-dd HH:mm:ss", timezone = "GMT+8")
@@ -356,4 +361,12 @@ public class TaskDefinition {
     public void setTimeoutFlag(TimeoutFlag timeoutFlag) {
         this.timeoutFlag = timeoutFlag;
     }
+
+    public String getResourceIds() {
+        return resourceIds;
+    }
+
+    public void setResourceIds(String resourceIds) {
+        this.resourceIds = resourceIds;
+    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java
index d9dd8a4..1791935 100644
--- a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/entity/TaskDefinitionLog.java
@@ -150,6 +150,11 @@ public class TaskDefinitionLog {
     private int timeout;
 
     /**
+     * resource ids
+     */
+    private String resourceIds;
+
+    /**
      * operator user id
      */
     private int operator;
@@ -382,4 +387,12 @@ public class TaskDefinitionLog {
     public void setTimeoutFlag(TimeoutFlag timeoutFlag) {
         this.timeoutFlag = timeoutFlag;
     }
+
+    public String getResourceIds() {
+        return resourceIds;
+    }
+
+    public void setResourceIds(String resourceIds) {
+        this.resourceIds = resourceIds;
+    }
 }
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.java
new file mode 100644
index 0000000..b4878a0
--- /dev/null
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.java
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.dao.mapper;
+
+import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
+
+import org.apache.ibatis.annotations.Param;
+
+import java.util.List;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+/**
+ * task definition log mapper interface
+ */
+public interface TaskDefinitionLogMapper extends BaseMapper<TaskDefinitionLog> {
+
+    /**
+     * query task definition log by name
+     *
+     * @param projectCode projectCode
+     * @param name name
+     * @return task definition log list
+     */
+    List<TaskDefinitionLog> queryByDefinitionName(@Param("projectCode") Long projectCode,
+                                        @Param("taskDefinitionName") String name);
+
+    /**
+     * query task definition log list
+     *
+     * @param taskDefinitionCode taskDefinitionCode
+     * @return task definition log list
+     */
+    List<TaskDefinitionLog> queryByDefinitionCode(@Param("taskDefinitionCode") long taskDefinitionCode);
+}
diff --git a/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
new file mode 100644
index 0000000..f1e9a78
--- /dev/null
+++ b/dolphinscheduler-dao/src/main/java/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.java
@@ -0,0 +1,103 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.dolphinscheduler.dao.mapper;
+
+import org.apache.dolphinscheduler.dao.entity.DefinitionGroupByUser;
+import org.apache.dolphinscheduler.dao.entity.TaskDefinition;
+
+import org.apache.ibatis.annotations.MapKey;
+import org.apache.ibatis.annotations.Param;
+
+import java.util.List;
+import java.util.Map;
+
+import com.baomidou.mybatisplus.core.mapper.BaseMapper;
+
+/**
+ * task definition mapper interface
+ */
+public interface TaskDefinitionMapper extends BaseMapper<TaskDefinition> {
+
+    /**
+     * verify task definition by name
+     *
+     * @param projectCode projectCode
+     * @param name name
+     * @return task definition
+     */
+    TaskDefinition verifyByDefineName(@Param("projectCode") Long projectCode,
+                                      @Param("taskDefinitionName") String name);
+
+    /**
+     * query task definition by name
+     *
+     * @param projectCode projectCode
+     * @param name name
+     * @return task definition
+     */
+    TaskDefinition queryByDefinitionName(@Param("projectCode") Long projectCode,
+                                         @Param("taskDefinitionName") String name);
+
+    /**
+     * query task definition by id
+     *
+     * @param taskDefinitionId taskDefinitionId
+     * @return task definition
+     */
+    TaskDefinition queryByDefinitionId(@Param("taskDefinitionId") int taskDefinitionId);
+
+    /**
+     * query all task definition list
+     *
+     * @param projectCode projectCode
+     * @return task definition list
+     */
+    List<TaskDefinition> queryAllDefinitionList(@Param("projectCode") Long projectCode);
+
+    /**
+     * query task definition by ids
+     *
+     * @param ids ids
+     * @return task definition list
+     */
+    List<TaskDefinition> queryDefinitionListByIdList(@Param("ids") Integer[] ids);
+
+    /**
+     * count task definition group by user
+     *
+     * @param projectCodes projectCodes
+     * @return task definition list
+     */
+    List<DefinitionGroupByUser> countDefinitionGroupByUser(@Param("projectCodes") Long[] projectCodes);
+
+    /**
+     * list all resource ids
+     *
+     * @return task ids list
+     */
+    @MapKey("id")
+    List<Map<String, Object>> listResources();
+
+    /**
+     * list all resource ids by user id
+     *
+     * @return resource ids list
+     */
+    @MapKey("id")
+    List<Map<String, Object>> listResourcesByUser(@Param("userId") Integer userId);
+}
diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
new file mode 100644
index 0000000..814ed70
--- /dev/null
+++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionLogMapper.xml
@@ -0,0 +1,42 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
+<mapper namespace="org.apache.dolphinscheduler.dao.mapper.TaskDefinitionLogMapper">
+    <select id="queryByDefinitionName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog">
+        select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
+        td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag,
+        td.timeout_notify_strategy, td.timeout, td.resource_ids, td.operator,td.operate_time, td.create_time, td.update_time,
+        u.user_name,p.name as project_name
+        from t_ds_task_definition_log td
+        JOIN t_ds_user u ON td.user_id = u.id
+        JOIN  t_ds_project p ON td.project_code = p.code
+        WHERE p.code = #{projectCode}
+        and td.name = #{taskDefinitionName}
+    </select>
+    <select id="queryByDefinitionCode" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog">
+        select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
+        td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag,
+        td.timeout_notify_strategy, td.timeout, td.resource_ids, td.operator,td.operate_time, td.create_time, td.update_time,
+        u.user_name,p.name as project_name
+        from t_ds_task_definition_log td
+        JOIN t_ds_user u ON td.user_id = u.id
+        JOIN  t_ds_project p ON td.project_code = p.code
+        WHERE td.code = #{taskDefinitionCode}
+    </select>
+</mapper>
diff --git a/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
new file mode 100644
index 0000000..1456789
--- /dev/null
+++ b/dolphinscheduler-dao/src/main/resources/org/apache/dolphinscheduler/dao/mapper/TaskDefinitionMapper.xml
@@ -0,0 +1,95 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+  ~ 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.
+  -->
+
+<!DOCTYPE mapper PUBLIC "-//mybatis.org//DTD Mapper 3.0//EN" "http://mybatis.org/dtd/mybatis-3-mapper.dtd" >
+<mapper namespace="org.apache.dolphinscheduler.dao.mapper.TaskDefinitionMapper">
+    <sql id="baseSql">
+        id, code, `name`, version, description, project_code, user_id, task_type, task_params, flag, task_priority,
+        worker_group, fail_retry_times, fail_retry_interval, timeout_flag, timeout_notify_strategy, timeout,
+        resource_ids, create_time, update_time
+    </sql>
+    <select id="verifyByDefineName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
+        select
+        <include refid="baseSql"/>
+        from t_ds_task_definition
+        WHERE projectCode = #{projectCode}
+        and `name` = #{taskDefinitionName}
+    </select>
+    <select id="queryByDefinitionName" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
+        select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
+        td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag,
+        td.timeout_notify_strategy, td.timeout, td.resource_ids, td.create_time, td.update_time, u.user_name,p.name as project_name
+        from t_ds_task_definition td
+        JOIN t_ds_user u ON td.user_id = u.id
+        JOIN  t_ds_project p ON td.project_code = p.code
+        WHERE p.code = #{projectCode}
+        and td.name = #{taskDefinitionName}
+    </select>
+
+    <select id="queryAllDefinitionList" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
+        select
+        <include refid="baseSql"/>
+        from t_ds_task_definition
+        where project_code = #{projectCode}
+        order by create_time desc
+    </select>
+
+    <select id="queryDefinitionListByIdList" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
+        select
+        <include refid="baseSql"/>
+        from t_ds_task_definition
+        where id in
+        <foreach collection="ids" index="index" item="i" open="(" separator="," close=")">
+            #{i}
+        </foreach>
+    </select>
+
+    <select id="countDefinitionGroupByUser" resultType="org.apache.dolphinscheduler.dao.entity.DefinitionGroupByUser">
+        SELECT td.user_id as user_id, tu.user_name as user_name, count(0) as count
+        FROM t_ds_task_definition td
+        JOIN t_ds_user tu on tu.id=td.user_id
+        where 1 = 1
+        <if test="projectCodes != null and projectCodes.length != 0">
+            and td.project_id in
+            <foreach collection="projectCodes" index="index" item="i" open="(" separator="," close=")">
+                #{i}
+            </foreach>
+        </if>
+        group by td.user_id,tu.user_name
+    </select>
+    <select id="queryByDefinitionId" resultType="org.apache.dolphinscheduler.dao.entity.TaskDefinition">
+        select td.id, td.code, td.name, td.version, td.description, td.project_code, td.user_id, td.task_type, td.task_params,
+        td.flag, td.task_priority, td.worker_group, td.fail_retry_times, td.fail_retry_interval, td.timeout_flag,
+        td.timeout_notify_strategy, td.timeout, td.resource_ids, td.create_time, td.update_time, u.user_name,p.name as project_name
+        from t_ds_task_definition td
+        JOIN t_ds_user u ON td.user_id = u.id
+        JOIN t_ds_project p ON td.project_code = p.code
+        WHERE td.id = #{taskDefinitionId}
+    </select>
+    <select id="listResources" resultType="java.util.HashMap">
+        SELECT id,resource_ids
+        FROM t_ds_task_definition
+        WHERE  resource_ids is not null and resource_ids != ''
+    </select>
+
+    <select id="listResourcesByUser" resultType="java.util.HashMap">
+        SELECT id,resource_ids
+        FROM t_ds_task_definition
+        WHERE user_id = #{userId} resource_ids is not null and resource_ids != ''
+    </select>
+</mapper>
diff --git a/sql/dolphinscheduler-postgre.sql b/sql/dolphinscheduler-postgre.sql
index e91527e..f98110c 100644
--- a/sql/dolphinscheduler-postgre.sql
+++ b/sql/dolphinscheduler-postgre.sql
@@ -290,30 +290,146 @@ CREATE TABLE t_ds_error_command (
 DROP TABLE IF EXISTS t_ds_process_definition;
 CREATE TABLE t_ds_process_definition (
   id int NOT NULL  ,
+  code bigint NOT NULL,
   name varchar(255) DEFAULT NULL ,
   version int DEFAULT NULL ,
+  description text ,
+  project_code bigint DEFAULT NULL ,
   release_state int DEFAULT NULL ,
-  project_id int DEFAULT NULL ,
   user_id int DEFAULT NULL ,
-  process_definition_json text ,
-  description text ,
   global_params text ,
-  flag int DEFAULT NULL ,
   locations text ,
-  connects text ,
+  flag int DEFAULT NULL ,
   receivers text ,
   receivers_cc text ,
+  timeout int DEFAULT '0' ,
+  tenant_id int NOT NULL DEFAULT '-1' ,
   create_time timestamp DEFAULT NULL ,
+  update_time timestamp DEFAULT NULL ,
+  PRIMARY KEY (id),
+  CONSTRAINT process_definition_unique UNIQUE (name, project_id)
+) ;
+
+create index process_definition_index on t_ds_process_definition (project_code,id);
+
+DROP TABLE IF EXISTS t_ds_process_definition_log;
+CREATE TABLE t_ds_process_definition_log (
+  id int NOT NULL  ,
+  code bigint NOT NULL,
+  name varchar(255) DEFAULT NULL ,
+  version int DEFAULT NULL ,
+  description text ,
+  project_code bigint DEFAULT NULL ,
+  release_state int DEFAULT NULL ,
+  user_id int DEFAULT NULL ,
+  global_params text ,
+  locations text ,
+  flag int DEFAULT NULL ,
+  receivers text ,
+  receivers_cc text ,
   timeout int DEFAULT '0' ,
   tenant_id int NOT NULL DEFAULT '-1' ,
+  operator int DEFAULT NULL ,
+  operate_time timestamp DEFAULT NULL ,
+  create_time timestamp DEFAULT NULL ,
   update_time timestamp DEFAULT NULL ,
-  modify_by varchar(36) DEFAULT '' ,
-  resource_ids varchar(64),
   PRIMARY KEY (id),
   CONSTRAINT process_definition_unique UNIQUE (name, project_id)
 ) ;
 
-create index process_definition_index on t_ds_process_definition (project_id,id);
+DROP TABLE IF EXISTS t_ds_task_definition;
+CREATE TABLE t_ds_task_definition (
+  id int NOT NULL  ,
+  code bigint NOT NULL,
+  name varchar(255) DEFAULT NULL ,
+  version int DEFAULT NULL ,
+  description text ,
+  project_code bigint DEFAULT NULL ,
+  user_id int DEFAULT NULL ,
+  task_type int DEFAULT NULL ,
+  task_params text ,
+  flag int DEFAULT NULL ,
+  task_priority int DEFAULT NULL ,
+  worker_group varchar(255) DEFAULT NULL ,
+  fail_retry_times int DEFAULT NULL ,
+  fail_retry_interval int DEFAULT NULL ,
+  timeout_flag int DEFAULT NULL ,
+  timeout_notify_strategy int DEFAULT NULL ,
+  timeout int DEFAULT '0' ,
+  resource_ids varchar(255) DEFAULT NULL ,
+  create_time timestamp DEFAULT NULL ,
+  update_time timestamp DEFAULT NULL ,
+  PRIMARY KEY (id),
+  CONSTRAINT task_definition_unique UNIQUE (name, project_code)
+) ;
+
+create index task_definition_index on t_ds_task_definition (project_code,id);
+
+DROP TABLE IF EXISTS t_ds_task_definition_log;
+CREATE TABLE t_ds_task_definition_log (
+  id int NOT NULL  ,
+  code bigint NOT NULL,
+  name varchar(255) DEFAULT NULL ,
+  version int DEFAULT NULL ,
+  description text ,
+  project_code bigint DEFAULT NULL ,
+  user_id int DEFAULT NULL ,
+  task_type int DEFAULT NULL ,
+  task_params text ,
+  flag int DEFAULT NULL ,
+  task_priority int DEFAULT NULL ,
+  worker_group varchar(255) DEFAULT NULL ,
+  fail_retry_times int DEFAULT NULL ,
+  fail_retry_interval int DEFAULT NULL ,
+  timeout_flag int DEFAULT NULL ,
+  timeout_notify_strategy int DEFAULT NULL ,
+  timeout int DEFAULT '0' ,
+  resource_ids varchar(255) DEFAULT NULL ,
+  operator int DEFAULT NULL ,
+  operate_time timestamp DEFAULT NULL ,
+  create_time timestamp DEFAULT NULL ,
+  update_time timestamp DEFAULT NULL ,
+  PRIMARY KEY (id),
+  CONSTRAINT task_definition_unique UNIQUE (name, project_id)
+) ;
+
+DROP TABLE IF EXISTS t_ds_process_task_relation;
+CREATE TABLE t_ds_process_task_relation (
+  id int NOT NULL  ,
+  name varchar(255) DEFAULT NULL ,
+  version int DEFAULT NULL ,
+  project_code bigint DEFAULT NULL ,
+  process_definition_code bigint DEFAULT NULL ,
+  pre_project_code bigint DEFAULT NULL ,
+  pre_task_code bigint DEFAULT NULL ,
+  post_project_code bigint DEFAULT NULL ,
+  post_task_code bigint DEFAULT NULL ,
+  condition_type int DEFAULT NULL ,
+  condition_params text ,
+  create_time timestamp DEFAULT NULL ,
+  update_time timestamp DEFAULT NULL ,
+  PRIMARY KEY (id)
+) ;
+
+DROP TABLE IF EXISTS t_ds_process_task_relation_log;
+CREATE TABLE t_ds_process_task_relation_log (
+  id int NOT NULL  ,
+  name varchar(255) DEFAULT NULL ,
+  version int DEFAULT NULL ,
+  project_code bigint DEFAULT NULL ,
+  process_definition_code bigint DEFAULT NULL ,
+  pre_project_code bigint DEFAULT NULL ,
+  pre_task_code bigint DEFAULT NULL ,
+  post_project_code bigint DEFAULT NULL ,
+  post_task_code bigint DEFAULT NULL ,
+  condition_type int DEFAULT NULL ,
+  condition_params text ,
+  operator int DEFAULT NULL ,
+  operate_time timestamp DEFAULT NULL ,
+  create_time timestamp DEFAULT NULL ,
+  update_time timestamp DEFAULT NULL ,
+  PRIMARY KEY (id)
+) ;
 
 --
 -- Table structure for table t_ds_process_definition_version
@@ -347,7 +463,8 @@ DROP TABLE IF EXISTS t_ds_process_instance;
 CREATE TABLE t_ds_process_instance (
   id int NOT NULL  ,
   name varchar(255) DEFAULT NULL ,
-  process_definition_id int DEFAULT NULL ,
+  process_definition_version int DEFAULT NULL ,
+  process_definition_code bigint DEFAULT NULL ,
   state int DEFAULT NULL ,
   recovery int DEFAULT NULL ,
   start_time timestamp DEFAULT NULL ,
@@ -380,7 +497,7 @@ CREATE TABLE t_ds_process_instance (
   var_pool text ,
   PRIMARY KEY (id)
 ) ;
-  create index process_instance_index on t_ds_process_instance (process_definition_id,id);
+  create index process_instance_index on t_ds_process_instance (process_definition_code,id);
   create index start_time_index on t_ds_process_instance (start_time);
 
 --
@@ -391,6 +508,7 @@ DROP TABLE IF EXISTS t_ds_project;
 CREATE TABLE t_ds_project (
   id int NOT NULL  ,
   name varchar(100) DEFAULT NULL ,
+  code bigint NOT NULL,
   description varchar(200) DEFAULT NULL ,
   user_id int DEFAULT NULL ,
   flag int DEFAULT '1' ,
@@ -574,7 +692,8 @@ CREATE TABLE t_ds_task_instance (
   id int NOT NULL  ,
   name varchar(255) DEFAULT NULL ,
   task_type varchar(64) DEFAULT NULL ,
-  process_definition_id int DEFAULT NULL ,
+  task_definition_version int DEFAULT NULL ,
+  process_definition_code bigint DEFAULT NULL ,
   process_instance_id int DEFAULT NULL ,
   task_json text ,
   state int DEFAULT NULL ,
@@ -719,6 +838,21 @@ ALTER TABLE t_ds_datasource ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_datasource
 DROP SEQUENCE IF EXISTS t_ds_process_definition_id_sequence;
 CREATE SEQUENCE  t_ds_process_definition_id_sequence;
 ALTER TABLE t_ds_process_definition ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_process_definition_id_sequence');
+DROP SEQUENCE IF EXISTS t_ds_process_definition_log_id_sequence;
+CREATE SEQUENCE  t_ds_process_definition_log_id_sequence;
+ALTER TABLE t_ds_process_definition_log ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_process_definition_log_id_sequence');
+DROP SEQUENCE IF EXISTS t_ds_task_definition_id_sequence;
+CREATE SEQUENCE  t_ds_task_definition_id_sequence;
+ALTER TABLE t_ds_task_definition ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_task_definition_id_sequence');
+DROP SEQUENCE IF EXISTS t_ds_task_definition_log_id_sequence;
+CREATE SEQUENCE  t_ds_task_definition_log_id_sequence;
+ALTER TABLE t_ds_task_definition_log ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_task_definition_log_id_sequence');
+DROP SEQUENCE IF EXISTS t_ds_process_task_relation_id_sequence;
+CREATE SEQUENCE  t_ds_process_task_relation_id_sequence;
+ALTER TABLE t_ds_process_task_relation ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_process_task_relation_id_sequence');
+DROP SEQUENCE IF EXISTS t_ds_process_task_relation_log_id_sequence;
+CREATE SEQUENCE  t_ds_process_task_relation_log_id_sequence;
+ALTER TABLE t_ds_process_task_relation_log ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_process_task_relation_log_id_sequence');
 DROP SEQUENCE IF EXISTS t_ds_process_definition_version_id_sequence;
 CREATE SEQUENCE  t_ds_process_definition_version_id_sequence;
 ALTER TABLE t_ds_process_definition_version ALTER COLUMN id SET DEFAULT NEXTVAL('t_ds_process_definition_version_id_sequence');
diff --git a/sql/dolphinscheduler_mysql.sql b/sql/dolphinscheduler_mysql.sql
index da6bfa2..4b4596d 100644
--- a/sql/dolphinscheduler_mysql.sql
+++ b/sql/dolphinscheduler_mysql.sql
@@ -466,6 +466,7 @@ CREATE TABLE `t_ds_task_definition` (
   `timeout_flag` tinyint(1) DEFAULT '0' COMMENT 'timeout flag:0 close, 1 open',
   `timeout_notify_strategy` tinyint(4) DEFAULT NULL COMMENT 'timeout notification policy: 0 warning, 1 fail',
   `timeout` int(11) DEFAULT '0' COMMENT 'timeout length,unit: minute',
+  `resource_ids` varchar(255) DEFAULT NULL COMMENT 'resource id, separated by comma',
   `create_time` datetime NOT NULL COMMENT 'create time',
   `update_time` datetime DEFAULT NULL COMMENT 'update time',
   PRIMARY KEY (`id`,`code`),
@@ -493,6 +494,7 @@ CREATE TABLE `t_ds_task_definition_log` (
   `timeout_flag` tinyint(1) DEFAULT '0' COMMENT 'timeout flag:0 close, 1 open',
   `timeout_notify_strategy` tinyint(4) DEFAULT NULL COMMENT 'timeout notification policy: 0 warning, 1 fail',
   `timeout` int(11) DEFAULT '0' COMMENT 'timeout length,unit: minute',
+  `resource_ids` varchar(255) DEFAULT NULL COMMENT 'resource id, separated by comma',
   `operator` int(11) DEFAULT NULL COMMENT 'operator user id',
   `operate_time` datetime DEFAULT NULL COMMENT 'operate time',
   `create_time` datetime NOT NULL COMMENT 'create time',