From 40c80c8fc2c0b9fd9e93a2c3fec26e54cd64673a Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Tue, 3 Feb 2026 17:38:22 +0800
Subject: [PATCH 1/9] remove import and export function
---
.../en/guide/project/workflow-definition.md | 3 -
docs/docs/en/guide/upgrade/incompatible.md | 1 +
.../zh/guide/project/workflow-definition.md | 3 -
docs/docs/zh/guide/upgrade/incompatible.md | 1 +
.../WorkflowDefinitionController.java | 58 --
.../service/WorkflowDefinitionService.java | 41 --
.../impl/WorkflowDefinitionServiceImpl.java | 522 ------------------
.../main/resources/i18n/messages.properties | 1 -
.../resources/i18n/messages_en_US.properties | 3 -
.../resources/i18n/messages_zh_CN.properties | 2 -
.../WorkflowDefinitionServiceTest.java | 175 ------
.../src/main/resources/application.yaml | 4 +-
.../src/locales/en_US/project.ts | 3 -
.../src/locales/zh_CN/project.ts | 3 -
.../modules/workflow-definition/index.ts | 17 -
.../definition/components/import-modal.tsx | 103 ----
.../definition/components/table-action.tsx | 25 -
.../definition/components/use-modal.ts | 27 -
.../definition/components/version-modal.tsx | 15 -
.../projects/workflow/definition/index.tsx | 32 --
.../projects/workflow/definition/use-table.ts | 52 --
21 files changed, 4 insertions(+), 1087 deletions(-)
delete mode 100644 dolphinscheduler-ui/src/views/projects/workflow/definition/components/import-modal.tsx
diff --git a/docs/docs/en/guide/project/workflow-definition.md b/docs/docs/en/guide/project/workflow-definition.md
index 998779c12023..6331d789dfd0 100644
--- a/docs/docs/en/guide/project/workflow-definition.md
+++ b/docs/docs/en/guide/project/workflow-definition.md
@@ -188,6 +188,3 @@ Description of workflow operating parameters:
- View the Schedule Timing:After the scheduled time goes online, you can check the current timing situation through the "Timing" button
. But it cannot be modified, as shown in the below figure.

-## Import Workflow
-
-Click `Project Management -> Workflow -> Workflow Definition` to enter the workflow definition page, click the `Import Workflow` button to import the local workflow file, the workflow definition list displays the imported workflow and the status is offline.
diff --git a/docs/docs/en/guide/upgrade/incompatible.md b/docs/docs/en/guide/upgrade/incompatible.md
index 8431370bfd38..71e075ee2ccd 100644
--- a/docs/docs/en/guide/upgrade/incompatible.md
+++ b/docs/docs/en/guide/upgrade/incompatible.md
@@ -39,4 +39,5 @@ This document records the incompatible updates between each version. You need to
* Remove the default value of `python-gateway.auth-token` at `api-server/application.yaml`. ([#17801])(https://github.com/apache/dolphinscheduler/pull/17801)
* Refactor the task plugins which use ShellCommandExecutor ([#17790])(https://github.com/apache/dolphinscheduler/pull/17790)
* Remove the `Pytorch` from the `Task Plugin` ([#17808])(https://github.com/apache/dolphinscheduler/pull/17808), if you are still using this task type, please delete the data with `task_type = 'PYTORCH'` in `t_ds_task_definition` and `t_ds_task_definition_log` before upgrading.
+* Remove import and export of workflow definition. ([#17940])(https://github.com/apache/dolphinscheduler/issues/17940)
diff --git a/docs/docs/zh/guide/project/workflow-definition.md b/docs/docs/zh/guide/project/workflow-definition.md
index 349897c91668..b2fc73408554 100644
--- a/docs/docs/zh/guide/project/workflow-definition.md
+++ b/docs/docs/zh/guide/project/workflow-definition.md
@@ -170,6 +170,3 @@
- 查看定时:定时上线后可以通过"定时"按钮
,查看当前定时情况。但是不可修改,如图所示。

-## 导入工作流
-
-点击项目管理->工作流->工作流定义,进入工作流定义页面,点击"导入工作流"按钮,导入本地工作流文件,工作流定义列表显示导入的工作流,状态为下线。
diff --git a/docs/docs/zh/guide/upgrade/incompatible.md b/docs/docs/zh/guide/upgrade/incompatible.md
index 90fabf780866..3e1acae5b222 100644
--- a/docs/docs/zh/guide/upgrade/incompatible.md
+++ b/docs/docs/zh/guide/upgrade/incompatible.md
@@ -43,4 +43,5 @@
* 移除 `api-server/application.yaml` 中 `python-gateway.auth-token` 的默认值。 ([#17801])(https://github.com/apache/dolphinscheduler/pull/17801)
* 重构使用 ShellCommandExecutor 的任务插件 ([#17790])(https://github.com/apache/dolphinscheduler/pull/17790)
* 从 `任务插件` 中移除 `Pytorch` 类型 ([#17808])(https://github.com/apache/dolphinscheduler/pull/17808),如果您仍在使用该任务类型,请在升级前删除 `t_ds_task_definition` 和 `t_ds_task_definition_log` 中 `task_type = 'PYTORCH'` 的数据。
+* 移除导入导出工作流([#17940])(https://github.com/apache/dolphinscheduler/issues/17940)
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionController.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionController.java
index 5f203b9f1585..d41ce9e12fc6 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionController.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionController.java
@@ -25,7 +25,6 @@
import static org.apache.dolphinscheduler.api.enums.Status.DELETE_WORKFLOW_DEFINITION_VERSION_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.ENCAPSULATION_TREEVIEW_STRUCTURE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.GET_TASKS_LIST_BY_WORKFLOW_DEFINITION_CODE_ERROR;
-import static org.apache.dolphinscheduler.api.enums.Status.IMPORT_WORKFLOW_DEFINE_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_DETAIL_OF_WORKFLOW_DEFINITION_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_WORKFLOW_DEFINITION_ALL_VARIABLES_ERROR;
import static org.apache.dolphinscheduler.api.enums.Status.QUERY_WORKFLOW_DEFINITION_LIST;
@@ -52,8 +51,6 @@
import java.util.Map;
-import javax.servlet.http.HttpServletResponse;
-
import lombok.extern.slf4j.Slf4j;
import org.springframework.beans.factory.annotation.Autowired;
@@ -66,10 +63,8 @@
import org.springframework.web.bind.annotation.RequestAttribute;
import org.springframework.web.bind.annotation.RequestMapping;
import org.springframework.web.bind.annotation.RequestParam;
-import org.springframework.web.bind.annotation.ResponseBody;
import org.springframework.web.bind.annotation.ResponseStatus;
import org.springframework.web.bind.annotation.RestController;
-import org.springframework.web.multipart.MultipartFile;
import io.swagger.v3.oas.annotations.Operation;
import io.swagger.v3.oas.annotations.Parameter;
@@ -653,32 +648,6 @@ public Result batchDeleteWorkflowDefinitionByCodes(@Parameter(hidden = true) @Re
return returnDataList(result);
}
- /**
- * batch export workflow definition by codes
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param codes workflow definition codes
- * @param response response
- */
- @Operation(summary = "batchExportByCodes", description = "BATCH_EXPORT_WORKFLOW_DEFINITION_BY_CODES_NOTES")
- @Parameters({
- @Parameter(name = "codes", description = "WORKFLOW_DEFINITION_CODE", required = true, schema = @Schema(implementation = String.class))
- })
- @PostMapping(value = "/batch-export")
- @ResponseBody
- @OperatorLog(auditType = AuditType.WORKFLOW_EXPORT)
- public void batchExportWorkflowDefinitionByCodes(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
- @Parameter(name = "projectCode", description = "PROJECT_CODE", required = true) @PathVariable long projectCode,
- @RequestParam("codes") String codes,
- HttpServletResponse response) {
- try {
- workflowDefinitionService.batchExportWorkflowDefinitionByCodes(loginUser, projectCode, codes, response);
- } catch (Exception e) {
- log.error(Status.BATCH_EXPORT_WORKFLOW_DEFINE_BY_IDS_ERROR.getMsg(), e);
- }
- }
-
/**
* query all workflow definition by project code
*
@@ -697,33 +666,6 @@ public Result queryAllWorkflowDefinitionByProjectCode(@Parameter(hidden = true)
return returnDataList(result);
}
- /**
- * import workflow definition
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param file resource file
- * @return import result code
- */
- @Operation(summary = "importWorkflowDefinition", description = "IMPORT_WORKFLOW_DEFINITION_NOTES")
- @Parameters({
- @Parameter(name = "file", description = "RESOURCE_FILE", required = true, schema = @Schema(implementation = MultipartFile.class))
- })
- @PostMapping(value = "/import")
- @ApiException(IMPORT_WORKFLOW_DEFINE_ERROR)
- @OperatorLog(auditType = AuditType.WORKFLOW_IMPORT)
- public Result importWorkflowDefinition(@Parameter(hidden = true) @RequestAttribute(value = Constants.SESSION_USER) User loginUser,
- @Parameter(name = "projectCode", description = "PROJECT_CODE", required = true) @PathVariable long projectCode,
- @RequestParam("file") MultipartFile file) {
- Map result;
- if ("application/zip".equals(file.getContentType())) {
- result = workflowDefinitionService.importSqlWorkflowDefinition(loginUser, projectCode, file);
- } else {
- result = workflowDefinitionService.importWorkflowDefinition(loginUser, projectCode, file);
- }
- return returnDataList(result);
- }
-
/**
* query workflow definition global variables and local variables
*
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionService.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionService.java
index 23e0cdc817da..19cc5e622bd3 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionService.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionService.java
@@ -31,10 +31,6 @@
import java.util.Map;
import java.util.Optional;
-import javax.servlet.http.HttpServletResponse;
-
-import org.springframework.web.multipart.MultipartFile;
-
public interface WorkflowDefinitionService {
/**
@@ -245,43 +241,6 @@ Map batchDeleteWorkflowDefinitionByCodes(User loginUser,
void deleteWorkflowDefinitionByCode(User loginUser, long workflowDefinitionCode);
- /**
- * batch export workflow definition by codes
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param codes workflow definition codes
- * @param response http servlet response
- */
- void batchExportWorkflowDefinitionByCodes(User loginUser,
- long projectCode,
- String codes,
- HttpServletResponse response);
-
- /**
- * import workflow definition
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param file workflow metadata json file
- * @return import workflow
- */
- Map importWorkflowDefinition(User loginUser,
- long projectCode,
- MultipartFile file);
-
- /**
- * import sql workflow definition
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param file sql file, zip
- * @return import workflow
- */
- Map importSqlWorkflowDefinition(User loginUser,
- long projectCode,
- MultipartFile file);
-
/**
* check the workflow task relation json
*
diff --git a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkflowDefinitionServiceImpl.java b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkflowDefinitionServiceImpl.java
index f0c060562035..43e49a952ba9 100644
--- a/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkflowDefinitionServiceImpl.java
+++ b/dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler/api/service/impl/WorkflowDefinitionServiceImpl.java
@@ -24,8 +24,6 @@
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_CREATE;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION_DELETE;
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION_EXPORT;
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_IMPORT;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_ONLINE_OFFLINE;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_SWITCH_TO_THIS_VERSION;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_TREE_VIEW;
@@ -35,13 +33,11 @@
import static org.apache.dolphinscheduler.common.constants.Constants.COPY_SUFFIX;
import static org.apache.dolphinscheduler.common.constants.Constants.DATA_LIST;
import static org.apache.dolphinscheduler.common.constants.Constants.GLOBAL_PARAMS;
-import static org.apache.dolphinscheduler.common.constants.Constants.IMPORT_SUFFIX;
import static org.apache.dolphinscheduler.common.constants.Constants.LOCAL_PARAMS;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.LOCAL_PARAMS_LIST;
import static org.apache.dolphinscheduler.plugin.task.api.TaskConstants.TASK_TYPE;
import static org.apache.dolphinscheduler.plugin.task.api.TaskPluginManager.checkTaskParameters;
-import org.apache.dolphinscheduler.api.dto.DagDataSchedule;
import org.apache.dolphinscheduler.api.dto.TaskCodeVersionDto;
import org.apache.dolphinscheduler.api.dto.treeview.Instance;
import org.apache.dolphinscheduler.api.dto.treeview.TreeViewDto;
@@ -59,15 +55,10 @@
import org.apache.dolphinscheduler.api.service.WorkflowInstanceService;
import org.apache.dolphinscheduler.api.service.WorkflowLineageService;
import org.apache.dolphinscheduler.api.utils.CheckUtils;
-import org.apache.dolphinscheduler.api.utils.FileUtils;
import org.apache.dolphinscheduler.api.utils.PageInfo;
import org.apache.dolphinscheduler.api.utils.Result;
import org.apache.dolphinscheduler.common.constants.Constants;
-import org.apache.dolphinscheduler.common.enums.ConditionType;
-import org.apache.dolphinscheduler.common.enums.Flag;
-import org.apache.dolphinscheduler.common.enums.Priority;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
-import org.apache.dolphinscheduler.common.enums.TimeoutFlag;
import org.apache.dolphinscheduler.common.enums.UserType;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionStatus;
import org.apache.dolphinscheduler.common.enums.WorkflowExecutionTypeEnum;
@@ -78,7 +69,6 @@
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.DagData;
-import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.DependentSimplifyDefinition;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Schedule;
@@ -108,14 +98,10 @@
import org.apache.dolphinscheduler.dao.repository.TaskDefinitionLogDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionLogDao;
-import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
-import org.apache.dolphinscheduler.plugin.task.api.enums.SqlType;
-import org.apache.dolphinscheduler.plugin.task.api.enums.TaskTimeoutStrategy;
import org.apache.dolphinscheduler.plugin.task.api.model.DependentItem;
import org.apache.dolphinscheduler.plugin.task.api.model.DependentTaskModel;
import org.apache.dolphinscheduler.plugin.task.api.model.Property;
import org.apache.dolphinscheduler.plugin.task.api.parameters.DependentParameters;
-import org.apache.dolphinscheduler.plugin.task.api.parameters.SqlParameters;
import org.apache.dolphinscheduler.plugin.task.api.parameters.SwitchParameters;
import org.apache.dolphinscheduler.plugin.task.api.utils.TaskTypeUtils;
import org.apache.dolphinscheduler.service.model.TaskNode;
@@ -124,11 +110,6 @@
import org.apache.commons.collections4.CollectionUtils;
import org.apache.commons.lang3.StringUtils;
-import java.io.BufferedOutputStream;
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.InputStreamReader;
-import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@@ -147,11 +128,6 @@
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.Stream;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipInputStream;
-
-import javax.servlet.ServletOutputStream;
-import javax.servlet.http.HttpServletResponse;
import lombok.NonNull;
import lombok.extern.slf4j.Slf4j;
@@ -159,14 +135,11 @@
import org.jetbrains.annotations.NotNull;
import org.springframework.beans.factory.annotation.Autowired;
import org.springframework.context.annotation.Lazy;
-import org.springframework.http.MediaType;
import org.springframework.stereotype.Service;
import org.springframework.transaction.annotation.Transactional;
-import org.springframework.web.multipart.MultipartFile;
import com.baomidou.mybatisplus.core.metadata.IPage;
import com.baomidou.mybatisplus.extension.plugins.pagination.Page;
-import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.node.ArrayNode;
import com.fasterxml.jackson.databind.node.ObjectNode;
import com.google.common.collect.Lists;
@@ -1125,501 +1098,6 @@ public void deleteWorkflowDefinitionByCode(User loginUser, long code) {
log.info("Success delete workflow definition workflowDefinitionCode: {}", code);
}
- /**
- * batch export workflow definition by codes
- */
- @Override
- public void batchExportWorkflowDefinitionByCodes(User loginUser, long projectCode, String codes,
- HttpServletResponse response) {
- if (StringUtils.isEmpty(codes)) {
- log.warn("workflow definition codes to be exported is empty.");
- return;
- }
- Project project = projectMapper.queryByCode(projectCode);
- // check user access for project
- Map result =
- projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_DEFINITION_EXPORT);
- if (result.get(Constants.STATUS) != Status.SUCCESS) {
- return;
- }
- Set defineCodeSet = Lists.newArrayList(codes.split(Constants.COMMA)).stream().map(Long::parseLong)
- .collect(Collectors.toSet());
- List workflowDefinitionList = workflowDefinitionMapper.queryByCodes(defineCodeSet);
- if (CollectionUtils.isEmpty(workflowDefinitionList)) {
- log.error("workflow definitions to be exported do not exist, workflowDefinitionCodes:{}.", defineCodeSet);
- return;
- }
- // check workflowDefinition exist in project
- List workflowDefinitionListInProject = workflowDefinitionList.stream()
- .filter(o -> projectCode == o.getProjectCode()).collect(Collectors.toList());
- List dagDataSchedules =
- workflowDefinitionListInProject.stream().map(this::exportWorkflowDagData).collect(Collectors.toList());
- if (CollectionUtils.isNotEmpty(dagDataSchedules)) {
- log.info("Start download workflow definition file, workflowDefinitionCodes:{}.", defineCodeSet);
- downloadWorkflowDefinitionFile(response, dagDataSchedules);
- } else {
- log.error("There is no exported workflow dag data.");
- }
- }
-
- /**
- * download the workflow definition file
- */
- protected void downloadWorkflowDefinitionFile(HttpServletResponse response,
- List dagDataSchedules) {
- response.setContentType(MediaType.APPLICATION_JSON_UTF8_VALUE);
- BufferedOutputStream buff = null;
- ServletOutputStream out = null;
- try {
- out = response.getOutputStream();
- buff = new BufferedOutputStream(out);
- buff.write(JSONUtils.toPrettyJsonString(dagDataSchedules).getBytes(StandardCharsets.UTF_8));
- buff.flush();
- buff.close();
- } catch (IOException e) {
- log.warn("Export workflow definition fail", e);
- } finally {
- if (null != buff) {
- try {
- buff.close();
- } catch (Exception e) {
- log.warn("Buffer does not close", e);
- }
- }
- if (null != out) {
- try {
- out.close();
- } catch (Exception e) {
- log.warn("Output stream does not close", e);
- }
- }
- }
- }
-
- /**
- * get export workflow dag data
- *
- * @param workflowDefinition workflow definition
- * @return DagDataSchedule
- */
- public DagDataSchedule exportWorkflowDagData(WorkflowDefinition workflowDefinition) {
- Schedule scheduleObj = scheduleMapper.queryByWorkflowDefinitionCode(workflowDefinition.getCode());
- DagDataSchedule dagDataSchedule = new DagDataSchedule(processService.genDagData(workflowDefinition));
- if (scheduleObj != null) {
- scheduleObj.setReleaseState(ReleaseState.OFFLINE);
- dagDataSchedule.setSchedule(scheduleObj);
- }
- return dagDataSchedule;
- }
-
- /**
- * import workflow definition
- *
- * @param loginUser login user
- * @param projectCode project code
- * @param file workflow metadata json file
- * @return import workflow
- */
- @Override
- @Transactional
- public Map importWorkflowDefinition(User loginUser, long projectCode, MultipartFile file) {
- Map result;
- String dagDataScheduleJson = FileUtils.file2String(file);
- List dagDataScheduleList = JSONUtils.toList(dagDataScheduleJson, DagDataSchedule.class);
- Project project = projectMapper.queryByCode(projectCode);
- result = projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_IMPORT);
- if (result.get(Constants.STATUS) != Status.SUCCESS) {
- return result;
- }
- // check file content
- if (CollectionUtils.isEmpty(dagDataScheduleList)) {
- log.warn("workflow definition file content is empty.");
- putMsg(result, Status.DATA_IS_NULL, "fileContent");
- return result;
- }
- for (DagDataSchedule dagDataSchedule : dagDataScheduleList) {
- if (!checkAndImport(loginUser, projectCode, result, dagDataSchedule)) {
- return result;
- }
- }
- return result;
- }
-
- @Override
- @Transactional
- public Map importSqlWorkflowDefinition(User loginUser, long projectCode, MultipartFile file) {
- Map result;
- Project project = projectMapper.queryByCode(projectCode);
- result = projectService.checkProjectAndAuth(loginUser, project, projectCode, WORKFLOW_IMPORT);
- if (result.get(Constants.STATUS) != Status.SUCCESS) {
- return result;
- }
- String workflowDefinitionName =
- file.getOriginalFilename() == null ? file.getName() : file.getOriginalFilename();
- int index = workflowDefinitionName.lastIndexOf(".");
- if (index > 0) {
- workflowDefinitionName = workflowDefinitionName.substring(0, index);
- }
- workflowDefinitionName = getNewName(workflowDefinitionName, IMPORT_SUFFIX);
-
- WorkflowDefinition workflowDefinition;
- List taskDefinitionList = new ArrayList<>();
- List workflowTaskRelationLogList = new ArrayList<>();
-
- // for Zip Bomb Attack
- final int THRESHOLD_ENTRIES = 10000;
- final int THRESHOLD_SIZE = 1000000000; // 1 GB
- final double THRESHOLD_RATIO = 10;
- int totalEntryArchive = 0;
- int totalSizeEntry = 0;
- // In most cases, there will be only one data source
- Map dataSourceCache = new HashMap<>(1);
- Map taskNameToCode = new HashMap<>(16);
- Map> taskNameToUpstream = new HashMap<>(16);
- try (
- ZipInputStream zIn = new ZipInputStream(file.getInputStream());
- BufferedReader bufferedReader = new BufferedReader(new InputStreamReader(zIn))) {
- // build workflow definition
- workflowDefinition = new WorkflowDefinition(projectCode,
- workflowDefinitionName,
- CodeGenerateUtils.genCode(),
- "",
- "[]", null,
- 0, loginUser.getId());
-
- ZipEntry entry;
- while ((entry = zIn.getNextEntry()) != null) {
- totalEntryArchive++;
- int totalSizeArchive = 0;
- if (!entry.isDirectory()) {
- StringBuilder sql = new StringBuilder();
- String taskName = null;
- String datasourceName = null;
- List upstreams = Collections.emptyList();
- String line;
- while ((line = bufferedReader.readLine()) != null) {
- int nBytes = line.getBytes(StandardCharsets.UTF_8).length;
- totalSizeEntry += nBytes;
- totalSizeArchive += nBytes;
- long compressionRatio = totalSizeEntry / entry.getCompressedSize();
- if (compressionRatio > THRESHOLD_RATIO) {
- throw new IllegalStateException(
- "Ratio between compressed and uncompressed data is highly suspicious, looks like a Zip Bomb Attack.");
- }
- int commentIndex = line.indexOf("-- ");
- if (commentIndex >= 0) {
- int colonIndex = line.indexOf(":", commentIndex);
- if (colonIndex > 0) {
- String key = line.substring(commentIndex + 3, colonIndex).trim().toLowerCase();
- String value = line.substring(colonIndex + 1).trim();
- switch (key) {
- case "name":
- taskName = value;
- line = line.substring(0, commentIndex);
- break;
- case "upstream":
- upstreams = Arrays.stream(value.split(",")).map(String::trim)
- .filter(s -> !"".equals(s)).collect(Collectors.toList());
- line = line.substring(0, commentIndex);
- break;
- case "datasource":
- datasourceName = value;
- line = line.substring(0, commentIndex);
- break;
- default:
- break;
- }
- }
- }
- if (!"".equals(line)) {
- sql.append(line).append("\n");
- }
- }
- // import/sql1.sql -> sql1
- if (taskName == null) {
- taskName = entry.getName();
- index = taskName.indexOf("/");
- if (index > 0) {
- taskName = taskName.substring(index + 1);
- }
- index = taskName.lastIndexOf(".");
- if (index > 0) {
- taskName = taskName.substring(0, index);
- }
- }
- DataSource dataSource = dataSourceCache.get(datasourceName);
- if (dataSource == null) {
- dataSource = queryDatasourceByNameAndUser(datasourceName, loginUser);
- }
- if (dataSource == null) {
- log.error("Datasource does not found, may be its name is illegal.");
- putMsg(result, Status.DATASOURCE_NAME_ILLEGAL);
- return result;
- }
- dataSourceCache.put(datasourceName, dataSource);
-
- TaskDefinitionLog taskDefinition =
- buildNormalSqlTaskDefinition(taskName, dataSource, sql.substring(0, sql.length() - 1));
-
- taskDefinitionList.add(taskDefinition);
- taskNameToCode.put(taskDefinition.getName(), taskDefinition.getCode());
- taskNameToUpstream.put(taskDefinition.getName(), upstreams);
- }
-
- if (totalSizeArchive > THRESHOLD_SIZE) {
- throw new IllegalStateException(
- "the uncompressed data size is too much for the application resource capacity");
- }
-
- if (totalEntryArchive > THRESHOLD_ENTRIES) {
- throw new IllegalStateException(
- "too much entries in this archive, can lead to inodes exhaustion of the system");
- }
- }
- } catch (Exception e) {
- log.error("Import workflow definition error.", e);
- putMsg(result, Status.IMPORT_WORKFLOW_DEFINE_ERROR);
- return result;
- }
-
- // build task relation
- for (Map.Entry entry : taskNameToCode.entrySet()) {
- List upstreams = taskNameToUpstream.get(entry.getKey());
- if (CollectionUtils.isEmpty(upstreams)
- || (upstreams.size() == 1 && upstreams.contains("root") && !taskNameToCode.containsKey("root"))) {
- WorkflowTaskRelationLog workflowTaskRelationLog = buildNormalTaskRelation(0, entry.getValue());
- workflowTaskRelationLogList.add(workflowTaskRelationLog);
- continue;
- }
- for (String upstream : upstreams) {
- WorkflowTaskRelationLog workflowTaskRelationLog =
- buildNormalTaskRelation(taskNameToCode.get(upstream), entry.getValue());
- workflowTaskRelationLogList.add(workflowTaskRelationLog);
- }
- }
-
- return createDagDefine(loginUser, workflowTaskRelationLogList, workflowDefinition, taskDefinitionList);
- }
-
- private WorkflowTaskRelationLog buildNormalTaskRelation(long preTaskCode, long postTaskCode) {
- WorkflowTaskRelationLog workflowTaskRelationLog = new WorkflowTaskRelationLog();
- workflowTaskRelationLog.setPreTaskCode(preTaskCode);
- workflowTaskRelationLog.setPreTaskVersion(0);
- workflowTaskRelationLog.setPostTaskCode(postTaskCode);
- workflowTaskRelationLog.setPostTaskVersion(0);
- workflowTaskRelationLog.setConditionType(ConditionType.NONE);
- workflowTaskRelationLog.setName("");
- return workflowTaskRelationLog;
- }
-
- private DataSource queryDatasourceByNameAndUser(String datasourceName, User loginUser) {
- if (isAdmin(loginUser)) {
- List dataSources = dataSourceMapper.queryDataSourceByName(datasourceName);
- if (CollectionUtils.isNotEmpty(dataSources)) {
- return dataSources.get(0);
- }
- } else {
- return dataSourceMapper.queryDataSourceByNameAndUserId(loginUser.getId(), datasourceName);
- }
- return null;
- }
-
- private TaskDefinitionLog buildNormalSqlTaskDefinition(String taskName, DataSource dataSource,
- String sql) {
- TaskDefinitionLog taskDefinition = new TaskDefinitionLog();
- taskDefinition.setName(taskName);
- taskDefinition.setFlag(Flag.YES);
- SqlParameters sqlParameters = new SqlParameters();
- sqlParameters.setType(dataSource.getType().name());
- sqlParameters.setDatasource(dataSource.getId());
- sqlParameters.setSql(sql.substring(0, sql.length() - 1));
- // it may be a query type, but it can only be determined by parsing SQL
- sqlParameters.setSqlType(SqlType.NON_QUERY.ordinal());
- sqlParameters.setLocalParams(Collections.emptyList());
- taskDefinition.setTaskParams(JSONUtils.toJsonString(sqlParameters));
- taskDefinition.setCode(CodeGenerateUtils.genCode());
- taskDefinition.setTaskType("SQL");
- taskDefinition.setFailRetryTimes(0);
- taskDefinition.setFailRetryInterval(0);
- taskDefinition.setTimeoutFlag(TimeoutFlag.CLOSE);
- taskDefinition.setWorkerGroup(WorkerGroupUtils.getDefaultWorkerGroup());
- taskDefinition.setTaskPriority(Priority.MEDIUM);
- taskDefinition.setEnvironmentCode(-1);
- taskDefinition.setTimeout(0);
- taskDefinition.setDelayTime(0);
- taskDefinition.setTimeoutNotifyStrategy(TaskTimeoutStrategy.WARN);
- taskDefinition.setVersion(0);
- taskDefinition.setResourceIds("");
- return taskDefinition;
- }
-
- /**
- * check and import
- */
- protected boolean checkAndImport(User loginUser,
- long projectCode,
- Map result,
- DagDataSchedule dagDataSchedule) {
- if (!checkImportanceParams(dagDataSchedule, result)) {
- return false;
- }
- WorkflowDefinition workflowDefinition = dagDataSchedule.getWorkflowDefinition();
-
- // generate import workflowDefinitionName
- String workflowDefinitionName = recursionWorkflowDefinitionName(projectCode, workflowDefinition.getName(), 1);
- String importWorkflowDefinitionName = getNewName(workflowDefinitionName, IMPORT_SUFFIX);
- // unique check
- Map checkResult =
- verifyWorkflowDefinitionName(loginUser, projectCode, importWorkflowDefinitionName, 0);
- if (Status.SUCCESS.equals(checkResult.get(Constants.STATUS))) {
- putMsg(result, Status.SUCCESS);
- } else {
- result.putAll(checkResult);
- return false;
- }
- workflowDefinition.setName(importWorkflowDefinitionName);
- workflowDefinition.setId(null);
- workflowDefinition.setProjectCode(projectCode);
- workflowDefinition.setUserId(loginUser.getId());
- workflowDefinition.setCode(CodeGenerateUtils.genCode());
-
- List taskDefinitionList = dagDataSchedule.getTaskDefinitionList();
- Map taskCodeMap = new HashMap<>();
- Date now = new Date();
- List taskDefinitionLogList = new ArrayList<>();
- for (TaskDefinition taskDefinition : taskDefinitionList) {
- TaskDefinitionLog taskDefinitionLog = new TaskDefinitionLog(taskDefinition);
- taskDefinitionLog.setName(taskDefinitionLog.getName());
- taskDefinitionLog.setProjectCode(projectCode);
- taskDefinitionLog.setUserId(loginUser.getId());
- taskDefinitionLog.setVersion(Constants.VERSION_FIRST);
- taskDefinitionLog.setCreateTime(now);
- taskDefinitionLog.setUpdateTime(now);
- taskDefinitionLog.setOperator(loginUser.getId());
- taskDefinitionLog.setOperateTime(now);
- long code = CodeGenerateUtils.genCode();
- taskCodeMap.put(taskDefinitionLog.getCode(), code);
- taskDefinitionLog.setCode(code);
-
- taskDefinitionLogList.add(taskDefinitionLog);
- }
- int insert = taskDefinitionMapper.batchInsert(taskDefinitionLogList);
- int logInsert = taskDefinitionLogMapper.batchInsert(taskDefinitionLogList);
- if ((logInsert & insert) == 0) {
- log.error("Save task definition error, projectCode:{}, workflowDefinitionCode:{}", projectCode,
- workflowDefinition.getCode());
- putMsg(result, Status.CREATE_TASK_DEFINITION_ERROR);
- throw new ServiceException(Status.CREATE_TASK_DEFINITION_ERROR);
- }
-
- List taskRelationList = dagDataSchedule.getWorkflowTaskRelationList();
- List taskRelationLogList = new ArrayList<>();
- for (WorkflowTaskRelation workflowTaskRelation : taskRelationList) {
- WorkflowTaskRelationLog workflowTaskRelationLog = new WorkflowTaskRelationLog(workflowTaskRelation);
- if (taskCodeMap.containsKey(workflowTaskRelationLog.getPreTaskCode())) {
- workflowTaskRelationLog.setPreTaskCode(taskCodeMap.get(workflowTaskRelationLog.getPreTaskCode()));
- }
- if (taskCodeMap.containsKey(workflowTaskRelationLog.getPostTaskCode())) {
- workflowTaskRelationLog.setPostTaskCode(taskCodeMap.get(workflowTaskRelationLog.getPostTaskCode()));
- }
- workflowTaskRelationLog.setPreTaskVersion(Constants.VERSION_FIRST);
- workflowTaskRelationLog.setPostTaskVersion(Constants.VERSION_FIRST);
- taskRelationLogList.add(workflowTaskRelationLog);
- }
- if (StringUtils.isNotEmpty(workflowDefinition.getLocations())
- && JSONUtils.checkJsonValid(workflowDefinition.getLocations())) {
- ArrayNode arrayNode = JSONUtils.parseArray(workflowDefinition.getLocations());
- ArrayNode newArrayNode = JSONUtils.createArrayNode();
- for (int i = 0; i < arrayNode.size(); i++) {
- ObjectNode newObjectNode = newArrayNode.addObject();
- JsonNode jsonNode = arrayNode.get(i);
- Long taskCode = taskCodeMap.get(jsonNode.get("taskCode").asLong());
- if (Objects.nonNull(taskCode)) {
- newObjectNode.put("taskCode", taskCode);
- newObjectNode.set("x", jsonNode.get("x"));
- newObjectNode.set("y", jsonNode.get("y"));
- }
- }
- workflowDefinition.setLocations(newArrayNode.toString());
- }
- workflowDefinition.setCreateTime(new Date());
- workflowDefinition.setUpdateTime(new Date());
- Map createDagResult =
- createDagDefine(loginUser, taskRelationLogList, workflowDefinition, Lists.newArrayList());
- if (Status.SUCCESS.equals(createDagResult.get(Constants.STATUS))) {
- putMsg(createDagResult, Status.SUCCESS);
- } else {
- result.putAll(createDagResult);
- log.error("Import workflow definition error, projectCode:{}, workflowDefinitionCode:{}.", projectCode,
- workflowDefinition.getCode());
- throw new ServiceException(Status.IMPORT_WORKFLOW_DEFINE_ERROR);
- }
-
- Schedule schedule = dagDataSchedule.getSchedule();
- if (null != schedule) {
- WorkflowDefinition newWorkflowDefinition =
- workflowDefinitionMapper.queryByCode(workflowDefinition.getCode());
- schedule.setWorkflowDefinitionCode(newWorkflowDefinition.getCode());
- schedule.setId(null);
- schedule.setUserId(loginUser.getId());
- schedule.setCreateTime(now);
- schedule.setUpdateTime(now);
- // not allow to import an online schedule
- schedule.setReleaseState(ReleaseState.OFFLINE);
- int scheduleInsert = scheduleMapper.insert(schedule);
- if (0 == scheduleInsert) {
- log.error(
- "Import workflow definition error due to save schedule fail, projectCode:{}, workflowDefinitionCode:{}.",
- projectCode, workflowDefinition.getCode());
- putMsg(result, Status.IMPORT_WORKFLOW_DEFINE_ERROR);
- throw new ServiceException(Status.IMPORT_WORKFLOW_DEFINE_ERROR);
- }
- }
-
- result.put(Constants.DATA_LIST, workflowDefinition);
- log.info("Import workflow definition complete, projectCode:{}, workflowDefinitionCode:{}.", projectCode,
- workflowDefinition.getCode());
- return true;
- }
-
- /**
- * check importance params
- */
- private boolean checkImportanceParams(DagDataSchedule dagDataSchedule, Map result) {
- if (dagDataSchedule.getWorkflowDefinition() == null) {
- log.warn("workflow definition is null.");
- putMsg(result, Status.DATA_IS_NULL, "WorkflowDefinition");
- return false;
- }
- if (CollectionUtils.isEmpty(dagDataSchedule.getTaskDefinitionList())) {
- log.warn("Task definition list is null.");
- putMsg(result, Status.DATA_IS_NULL, "TaskDefinitionList");
- return false;
- }
- if (CollectionUtils.isEmpty(dagDataSchedule.getWorkflowTaskRelationList())) {
- log.warn("workflow task relation list is null.");
- putMsg(result, Status.DATA_IS_NULL, "WorkflowTaskRelationList");
- return false;
- }
- return true;
- }
-
- private String recursionWorkflowDefinitionName(long projectCode, String workflowDefinitionName, int num) {
- WorkflowDefinition workflowDefinition =
- workflowDefinitionMapper.queryByDefineName(projectCode, workflowDefinitionName);
- if (workflowDefinition != null) {
- if (num > 1) {
- String str = workflowDefinitionName.substring(0, workflowDefinitionName.length() - 3);
- workflowDefinitionName = str + "(" + num + ")";
- } else {
- workflowDefinitionName = workflowDefinition.getName() + "(" + num + ")";
- }
- } else {
- return workflowDefinitionName;
- }
- return recursionWorkflowDefinitionName(projectCode, workflowDefinitionName, num + 1);
- }
-
/**
* check the workflow task relation json
*
diff --git a/dolphinscheduler-api/src/main/resources/i18n/messages.properties b/dolphinscheduler-api/src/main/resources/i18n/messages.properties
index b11a591d734b..7f56c1eb8d39 100644
--- a/dolphinscheduler-api/src/main/resources/i18n/messages.properties
+++ b/dolphinscheduler-api/src/main/resources/i18n/messages.properties
@@ -292,7 +292,6 @@ QUERY_UNAUTHORIZED_NAMESPACE_NOTES=query unauthorized k8s namespace
QUERY_AUTHORIZED_NAMESPACE_NOTES=query authorized k8s namespace
QUERY_AVAILABLE_NAMESPACE_LIST_NOTES=query available k8s namespace
-BATCH_EXPORT_WORKFLOW_DEFINITION_BY_CODES_NOTES=batch export workflow definition by code list
QUERY_WORKFLOW_DEFINITION_All_BY_PROJECT_CODE_NOTES=query all workflow definition by project code
CREATE_WORKFLOW_TASK_RELATION_NOTES=create workflow task relation
diff --git a/dolphinscheduler-api/src/main/resources/i18n/messages_en_US.properties b/dolphinscheduler-api/src/main/resources/i18n/messages_en_US.properties
index 65416ca87c53..71e5dc091d83 100644
--- a/dolphinscheduler-api/src/main/resources/i18n/messages_en_US.properties
+++ b/dolphinscheduler-api/src/main/resources/i18n/messages_en_US.properties
@@ -158,10 +158,8 @@ VERIFY_WORKFLOW_DEFINITION_NAME_NOTES=verify process definition name
LOGIN_NOTES=user login
SSO_LOGIN_NOTES=user sso login
UPDATE_WORKFLOW_DEFINITION_NOTES=update process definition
-WORKFLOW_DEFINITION_ID=workflow definition id
WORKFLOW_DEFINITION_CODE=workflow definition code
WORKFLOW_DEFINITION_CODE_LIST=workflow definition code list
-IMPORT_WORKFLOW_DEFINITION_NOTES=import workflow definition
RELEASE_WORKFLOW_DEFINITION_NOTES=release workflow definition
QUERY_WORKFLOW_DEFINITION_LIST_NOTES=query workflow definition list
QUERY_WORKFLOW_DEFINITION_LIST_PAGING_NOTES=query workflow definition list paging
@@ -324,7 +322,6 @@ QUERY_UNAUTHORIZED_NAMESPACE_NOTES=query unauthorized k8s namespace
QUERY_AUTHORIZED_NAMESPACE_NOTES=query authorized k8s namespace
QUERY_AVAILABLE_NAMESPACE_LIST_NOTES=query available k8s namespace
-BATCH_EXPORT_WORKFLOW_DEFINITION_BY_CODES_NOTES=batch export process definition by code list
QUERY_WORKFLOW_DEFINITION_All_BY_PROJECT_CODE_NOTES=query all process definition by project code
CREATE_WORKFLOW_TASK_RELATION_NOTES=create process task relation
diff --git a/dolphinscheduler-api/src/main/resources/i18n/messages_zh_CN.properties b/dolphinscheduler-api/src/main/resources/i18n/messages_zh_CN.properties
index 3fc58dcb4124..243c41c0685c 100644
--- a/dolphinscheduler-api/src/main/resources/i18n/messages_zh_CN.properties
+++ b/dolphinscheduler-api/src/main/resources/i18n/messages_zh_CN.properties
@@ -175,7 +175,6 @@ PAGE_SIZE=\u9875\u5927\u5C0F
LIMIT=\u663E\u793A\u591A\u5C11\u6761
RESOURCE_CURRENTDIR=\u5F53\u524D\u8D44\u6E90\u76EE\u5F55
VIEW_TREE_NOTES=\u6811\u72B6\u56FE
-IMPORT_WORKFLOW_DEFINITION_NOTES=\u5BFC\u5165\u6D41\u7A0B\u5B9A\u4E49
BATCH_DELETE_WORKFLOW_DEFINITION_BY_IDS_NOTES=\u901A\u8FC7\u6D41\u7A0B\u5B9A\u4E49ID\u96C6\u5408\u6279\u91CF\u5220\u9664\u6D41\u7A0B\u5B9A\u4E49
BATCH_DELETE_WORKFLOW_INSTANCE_BY_IDS_NOTES=\u901A\u8FC7\u6D41\u7A0B\u5B9E\u4F8BID\u96C6\u5408\u6279\u91CF\u5220\u9664\u6D41\u7A0B\u5B9E\u4F8B
DELETE_WORKFLOW_DEFINITION_BY_ID_NOTES=\u901A\u8FC7\u6D41\u7A0B\u5B9A\u4E49ID\u5220\u9664\u6D41\u7A0B\u5B9A\u4E49
@@ -322,7 +321,6 @@ QUERY_UNAUTHORIZED_NAMESPACE_NOTES=\u67E5\u8BE2\u672A\u6388\u6743\u547D\u540D\u7
QUERY_AUTHORIZED_NAMESPACE_NOTES=\u67E5\u8BE2\u6388\u6743\u547D\u540D\u7A7A\u95F4
QUERY_AVAILABLE_NAMESPACE_LIST_NOTES=\u67E5\u8BE2\u53EF\u7528\u547D\u540D\u7A7A\u95F4\u5217\u8868
-BATCH_EXPORT_WORKFLOW_DEFINITION_BY_CODES_NOTES=\u901A\u8FC7\u4EE3\u7801\u5217\u8868\u6279\u91CF\u5BFC\u51FA\u5DE5\u4F5C\u91CF\u5B9A\u4E49
QUERY_WORKFLOW_DEFINITION_All_BY_PROJECT_CODE_NOTES=\u901A\u8FC7\u9879\u76EE\u4EE3\u7801\u67E5\u8BE2\u6240\u6709\u5DE5\u4F5C\u91CF\u5B9A\u4E49
CREATE_WORKFLOW_TASK_RELATION_NOTES=\u521B\u5EFA\u5DE5\u4F5C\u6D41\u4EFB\u52A1\u5173\u7CFB
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionServiceTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionServiceTest.java
index d11f1ef9b5db..30664ede4d94 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionServiceTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/service/WorkflowDefinitionServiceTest.java
@@ -22,14 +22,11 @@
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_CREATE;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_DEFINITION_DELETE;
-import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_IMPORT;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_TREE_VIEW;
import static org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant.WORKFLOW_UPDATE;
import static org.apache.dolphinscheduler.common.constants.Constants.EMPTY_STRING;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyList;
-import static org.mockito.ArgumentMatchers.anyLong;
-import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.ArgumentMatchers.isA;
import static org.mockito.Mockito.doNothing;
@@ -38,7 +35,6 @@
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
-import org.apache.dolphinscheduler.api.constants.ApiFuncIdentificationConstant;
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowCreateRequest;
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowFilterRequest;
import org.apache.dolphinscheduler.api.dto.workflow.WorkflowUpdateRequest;
@@ -58,7 +54,6 @@
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.common.utils.JSONUtils;
import org.apache.dolphinscheduler.dao.entity.DagData;
-import org.apache.dolphinscheduler.dao.entity.DataSource;
import org.apache.dolphinscheduler.dao.entity.Project;
import org.apache.dolphinscheduler.dao.entity.Schedule;
import org.apache.dolphinscheduler.dao.entity.TaskDefinitionLog;
@@ -82,14 +77,11 @@
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionLogDao;
import org.apache.dolphinscheduler.dao.utils.WorkerGroupUtils;
import org.apache.dolphinscheduler.service.process.ProcessService;
-import org.apache.dolphinscheduler.spi.enums.DbType;
import org.apache.commons.lang3.StringUtils;
-import java.io.ByteArrayOutputStream;
import java.io.IOException;
import java.net.URISyntaxException;
-import java.nio.charset.StandardCharsets;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
@@ -104,10 +96,6 @@
import java.util.Optional;
import java.util.Set;
import java.util.stream.Collectors;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipOutputStream;
-
-import javax.servlet.http.HttpServletResponse;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
@@ -891,71 +879,6 @@ public void testUpdateWorkflowDefinition() {
}
}
- @Test
- public void testBatchExportWorkflowDefinitionByCodes() {
- processDefinitionService.batchExportWorkflowDefinitionByCodes(null, 1L, null, null);
- Project project = getProject(projectCode);
-
- Map result = new HashMap<>();
- putMsg(result, Status.PROJECT_NOT_FOUND);
- when(projectMapper.queryByCode(projectCode)).thenReturn(getProject(projectCode));
- processDefinitionService.batchExportWorkflowDefinitionByCodes(user, projectCode, "1", null);
-
- WorkflowDefinition workflowDefinition = new WorkflowDefinition();
- workflowDefinition.setId(1);
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- HttpServletResponse response = Mockito.mock(HttpServletResponse.class);
-
- DagData dagData = new DagData(getWorkflowDefinition(), null, null);
- when(processService.genDagData(any())).thenReturn(dagData);
- processDefinitionService.batchExportWorkflowDefinitionByCodes(user, projectCode, "1", response);
- Assertions.assertNotNull(processDefinitionService.exportWorkflowDagData(workflowDefinition));
- }
-
- @Test
- public void testImportSqlWorkflowDefinition() throws Exception {
- ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream();
- ZipOutputStream outputStream = new ZipOutputStream(byteArrayOutputStream);
- outputStream.putNextEntry(new ZipEntry("import_sql/"));
-
- outputStream.putNextEntry(new ZipEntry("import_sql/a.sql"));
- outputStream.write(
- "-- upstream: start_auto_dag\n-- datasource: mysql_1\nselect 1;".getBytes(StandardCharsets.UTF_8));
-
- outputStream.putNextEntry(new ZipEntry("import_sql/b.sql"));
- outputStream
- .write("-- name: start_auto_dag\n-- datasource: mysql_1\nselect 1;".getBytes(StandardCharsets.UTF_8));
-
- outputStream.close();
-
- MockMultipartFile mockMultipartFile =
- new MockMultipartFile("import_sql.zip", byteArrayOutputStream.toByteArray());
-
- DataSource dataSource = Mockito.mock(DataSource.class);
- when(dataSource.getId()).thenReturn(1);
- when(dataSource.getType()).thenReturn(DbType.MYSQL);
-
- when(dataSourceMapper.queryDataSourceByNameAndUserId(user.getId(), "mysql_1")).thenReturn(dataSource);
-
- Project project = getProject(projectCode);
- Map result = new HashMap<>();
- result.put(Constants.STATUS, Status.SUCCESS);
- when(projectMapper.queryByCode(projectCode)).thenReturn(getProject(projectCode));
- when(projectService.checkProjectAndAuth(user, project, projectCode, WORKFLOW_IMPORT))
- .thenReturn(result);
- when(processService.saveTaskDefine(Mockito.same(user), eq(projectCode), Mockito.notNull(),
- Mockito.anyBoolean())).thenReturn(2);
- when(processService.saveWorkflowDefine(Mockito.same(user), Mockito.notNull(), Mockito.notNull(),
- Mockito.anyBoolean())).thenReturn(1);
- when(
- processService.saveTaskRelation(Mockito.same(user), eq(projectCode), anyLong(),
- eq(1), Mockito.notNull(), Mockito.notNull(), Mockito.anyBoolean()))
- .thenReturn(0);
- result = processDefinitionService.importSqlWorkflowDefinition(user, projectCode, mockMultipartFile);
-
- Assertions.assertEquals(result.get(Constants.STATUS), Status.SUCCESS);
- }
-
@Test
public void testGetNewProcessName() {
String processName1 = "test_copy_" + DateUtils.getCurrentTimeStamp();
@@ -1281,104 +1204,6 @@ private List getTaskMainInfo() {
return taskMainInfos;
}
- @Test
- public void testImportWorkflowDefinitionWithoutProjectAuth() {
- Project project = this.getProject(projectCode);
- Map successResult = new HashMap<>();
- putMsg(successResult, Status.SUCCESS);
- MultipartFile file = new MockMultipartFile(
- "file", "", "application/json", "".getBytes());
- Map checkProjectPermResult1 = new HashMap<>();
- putMsg(checkProjectPermResult1, Status.USER_NO_OPERATION_PROJECT_PERM);
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_IMPORT))
- .thenReturn(checkProjectPermResult1);
- Map checkProjectPermResult = processDefinitionService.importWorkflowDefinition(
- user, projectCode, file);
- Assertions.assertEquals(
- checkProjectPermResult.get(Constants.STATUS), checkProjectPermResult1.get(Constants.STATUS));
- }
-
- @Test
- public void testImportWorkflowDefinitionWithEmptyFileContent() {
- Project project = this.getProject(projectCode);
- Map successResult = new HashMap<>();
- putMsg(successResult, Status.SUCCESS);
- MultipartFile file = new MockMultipartFile("file", "", "application/json", "".getBytes());
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_IMPORT))
- .thenReturn(successResult);
- Map result = processDefinitionService.importWorkflowDefinition(user, projectCode, file);
- Assertions.assertEquals(Status.DATA_IS_NULL, result.get(Constants.STATUS));
- }
-
- @Test
- public void testImportWorkflowDefinitionWhenMissImportanceParams() throws URISyntaxException, IOException {
- Project project = this.getProject(projectCode);
- Map successResult = new HashMap<>();
- putMsg(successResult, Status.SUCCESS);
- // miss workflowTaskRelationList
- MultipartFile checkImportanceParamsFile = createMultipartFile("workflowImport/check_importance_params.json");
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_IMPORT))
- .thenReturn(successResult);
- Map checkImportanceParamsResult = processDefinitionService.importWorkflowDefinition(
- user, projectCode, checkImportanceParamsFile);
- Assertions.assertEquals(Status.DATA_IS_NULL, checkImportanceParamsResult.get(Constants.STATUS));
- }
-
- @Test
- public void testImportWorkflowDefinitionWhenNameExist() throws URISyntaxException, IOException {
- Project project = this.getProject(projectCode);
- Map successResult = new HashMap<>();
- putMsg(successResult, Status.SUCCESS);
- MultipartFile checkDuplicateNameFile = createMultipartFile("workflowImport/check_duplicate_name.json");
- Map verifyNameResult = new HashMap<>();
- putMsg(verifyNameResult, Status.WORKFLOW_DEFINITION_NAME_EXIST);
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_IMPORT))
- .thenReturn(successResult);
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_CREATE))
- .thenReturn(successResult);
- WorkflowDefinition workflowDefinition = new WorkflowDefinition();
- workflowDefinition.setCode(2);
- workflowDefinition.setName("workflow1");
- when(workflowDefinitionMapper.verifyByDefineName(eq(projectCode), anyString()))
- .thenReturn(workflowDefinition);
- Map checkDuplicateNameResult = processDefinitionService.importWorkflowDefinition(
- user, projectCode, checkDuplicateNameFile);
- Assertions.assertEquals(Status.WORKFLOW_DEFINITION_NAME_EXIST, checkDuplicateNameResult.get(Constants.STATUS));
- }
-
- @Test
- public void testImportWorkflowDefinitionSuccessful() throws URISyntaxException, IOException {
- Project project = this.getProject(projectCode);
- Map successResult = new HashMap<>();
- putMsg(successResult, Status.SUCCESS);
- MultipartFile successfulFile = createMultipartFile("workflowImport/check_successful.json");
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(),
- ApiFuncIdentificationConstant.WORKFLOW_IMPORT))
- .thenReturn(successResult);
- when(projectMapper.queryByCode(projectCode)).thenReturn(project);
- when(projectService.checkProjectAndAuth(user, project, project.getCode(), WORKFLOW_CREATE))
- .thenReturn(successResult);
- when(workflowDefinitionMapper.verifyByDefineName(eq(projectCode), anyString()))
- .thenReturn(null);
- when(taskDefinitionMapper.batchInsert(anyList())).thenReturn(1);
- when(taskDefinitionLogMapper.batchInsert(anyList())).thenReturn(1);
- WorkflowDefinition successWorkflowDef = new WorkflowDefinition();
- successWorkflowDef.setCode(123);
- when(workflowDefinitionMapper.queryByCode(anyLong())).thenReturn(successWorkflowDef);
- when(scheduleMapper.insert(any())).thenReturn(1);
- when(processService.saveWorkflowDefine(eq(user), any(), eq(true), eq(true)))
- .thenReturn(Constants.VERSION_FIRST);
- Map successfulResul = processDefinitionService.importWorkflowDefinition(
- user, 1L, successfulFile);
- Assertions.assertEquals(Status.SUCCESS, successfulResul.get(Constants.STATUS));
- }
-
private MultipartFile createMultipartFile(String filePath) throws URISyntaxException, IOException {
Path path = Paths.get(getClass().getClassLoader().getResource(filePath).toURI());
byte[] content = Files.readAllBytes(path);
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index 71c309e6d732..a0f0b9386ba8 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -169,7 +169,7 @@ master:
# master heartbeat interval
max-heartbeat-interval: 10s
server-load-protection:
- enabled: true
+ enabled: false
# Master max system cpu usage, when the master's system cpu usage is smaller then this value, master server can execute workflow.
max-system-cpu-usage-percentage-thresholds: 0.8
# Master max jvm cpu usage, when the master's jvm cpu usage is smaller then this value, master server can execute workflow.
@@ -205,7 +205,7 @@ worker:
# worker group name. If it is not set, the default value is default.
group: default
server-load-protection:
- enabled: true
+ enabled: false
# Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks.
max-system-cpu-usage-percentage-thresholds: 0.8
# Worker max jvm cpu usage, when the worker's jvm cpu usage is smaller then this value, worker server can be dispatched tasks.
diff --git a/dolphinscheduler-ui/src/locales/en_US/project.ts b/dolphinscheduler-ui/src/locales/en_US/project.ts
index 916893b02a0c..85519adf015b 100644
--- a/dolphinscheduler-ui/src/locales/en_US/project.ts
+++ b/dolphinscheduler-ui/src/locales/en_US/project.ts
@@ -50,7 +50,6 @@ export default {
workflow_relation: 'Workflow Relation',
create_workflow: 'Create Workflow',
create_workflow_dynamic: 'Create Workflow (Dynamic)',
- import_workflow: 'Import Workflow',
workflow_name: 'Workflow Name',
workflow_instance_name: 'Workflow Instance Name',
current_selection: 'Current Selection',
@@ -92,10 +91,8 @@ export default {
delete: 'Delete',
tree_view: 'Tree View',
tree_limit: 'Limit Size',
- export: 'Export',
batch_copy: 'Batch Copy',
batch_delete: 'Batch Delete',
- batch_export: 'Batch Export',
version_info: 'Version Info',
version: 'Version',
file_upload: 'File Upload',
diff --git a/dolphinscheduler-ui/src/locales/zh_CN/project.ts b/dolphinscheduler-ui/src/locales/zh_CN/project.ts
index 46fd12e348fd..8de4df985ca3 100644
--- a/dolphinscheduler-ui/src/locales/zh_CN/project.ts
+++ b/dolphinscheduler-ui/src/locales/zh_CN/project.ts
@@ -50,7 +50,6 @@ export default {
workflow_relation: '工作流关系',
create_workflow: '创建工作流',
create_workflow_dynamic: '创建工作流 (动态)',
- import_workflow: '导入工作流',
workflow_name: '工作流名称',
workflow_instance_name: '工作流实例名称',
current_selection: '当前选择',
@@ -92,10 +91,8 @@ export default {
delete: '删除',
tree_view: '工作流树形图',
tree_limit: '限制大小',
- export: '导出',
batch_copy: '批量复制',
batch_delete: '批量删除',
- batch_export: '批量导出',
version_info: '版本信息',
version: '版本',
file_upload: '文件上传',
diff --git a/dolphinscheduler-ui/src/service/modules/workflow-definition/index.ts b/dolphinscheduler-ui/src/service/modules/workflow-definition/index.ts
index 7e6900617f68..d6d7b0a23cf1 100644
--- a/dolphinscheduler-ui/src/service/modules/workflow-definition/index.ts
+++ b/dolphinscheduler-ui/src/service/modules/workflow-definition/index.ts
@@ -73,15 +73,6 @@ export function batchDeleteByCodes(data: CodesReq, code: number): any {
})
}
-export function batchExportByCodes(data: CodesReq, code: number): any {
- return axios({
- url: `/projects/${code}/workflow-definition/batch-export`,
- method: 'post',
- responseType: 'blob',
- data
- })
-}
-
export function batchMoveByCodes(
data: TargetCodeReq & CodesReq,
code: CodeReq
@@ -104,14 +95,6 @@ export function getTaskListByDefinitionCodes(
})
}
-export function importWorkflowDefinition(data: FormData, code: number): any {
- return axios({
- url: `/projects/${code}/workflow-definition/import`,
- method: 'post',
- data
- })
-}
-
export function queryList(code: CodeReq): any {
return axios({
url: `/projects/${code}/workflow-definition/list`,
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/import-modal.tsx b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/import-modal.tsx
deleted file mode 100644
index b81f081e04bc..000000000000
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/import-modal.tsx
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * 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.
- */
-
-import { defineComponent, getCurrentInstance, PropType, toRefs } from 'vue'
-import { useI18n } from 'vue-i18n'
-import Modal from '@/components/modal'
-import { useForm } from './use-form'
-import { useModal } from './use-modal'
-import { NForm, NFormItem, NButton, NUpload, NIcon, NInput } from 'naive-ui'
-import { CloudUploadOutlined } from '@vicons/antd'
-
-const props = {
- show: {
- type: Boolean as PropType,
- default: false
- }
-}
-
-export default defineComponent({
- name: 'workflowDefinitionImport',
- props,
- emits: ['update:show', 'update:row', 'updateList'],
- setup(props, ctx) {
- const { importState } = useForm()
- const { handleImportDefinition } = useModal(importState, ctx)
- const hideModal = () => {
- ctx.emit('update:show')
- }
-
- const handleImport = () => {
- handleImportDefinition()
- }
-
- const customRequest = ({ file }: any) => {
- importState.importForm.name = file.name
- importState.importForm.file = file.file
- }
- const trim = getCurrentInstance()?.appContext.config.globalProperties.trim
-
- return {
- hideModal,
- handleImport,
- customRequest,
- ...toRefs(importState),
- trim
- }
- },
-
- render() {
- const { t } = useI18n()
-
- return (
-
-
-
-
-
-
- {t('project.workflow.upload')}
-
-
-
-
-
-
-
-
-
-
-
-
- )
- }
-})
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/table-action.tsx b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/table-action.tsx
index 61499a898c97..aa7b313d92bc 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/table-action.tsx
+++ b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/table-action.tsx
@@ -25,7 +25,6 @@ import {
PlayCircleOutlined,
ClockCircleOutlined,
CopyOutlined,
- ExportOutlined,
ApartmentOutlined,
UploadOutlined,
ArrowUpOutlined,
@@ -53,7 +52,6 @@ export default defineComponent({
'releaseWorkflow',
'releaseScheduler',
'copyWorkflow',
- 'exportWorkflow',
'gotoWorkflowTree'
],
setup(props, ctx) {
@@ -85,10 +83,6 @@ export default defineComponent({
ctx.emit('copyWorkflow')
}
- const handleExportWorkflow = () => {
- ctx.emit('exportWorkflow')
- }
-
const handleGotoWorkflowTree = () => {
ctx.emit('gotoWorkflowTree')
}
@@ -105,7 +99,6 @@ export default defineComponent({
handleDeleteWorkflow,
handleReleaseWorkflow,
handleCopyWorkflow,
- handleExportWorkflow,
handleGotoWorkflowTree,
handleReleaseScheduler,
...toRefs(props)
@@ -316,24 +309,6 @@ export default defineComponent({
)
}}
-
- {{
- default: () => t('project.workflow.export'),
- trigger: () => (
-
-
-
-
-
- )
- }}
-
{{
default: () => t('project.workflow.version_info'),
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/use-modal.ts b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/use-modal.ts
index 44f25dde46eb..4baac0f1d1dc 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/use-modal.ts
+++ b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/use-modal.ts
@@ -23,7 +23,6 @@ import type { Router } from 'vue-router'
import { format } from 'date-fns'
import {
batchCopyByCodes,
- importWorkflowDefinition,
queryWorkflowDefinitionByCode
} from '@/service/modules/workflow-definition'
import { queryAllEnvironmentList } from '@/service/modules/environment'
@@ -62,31 +61,6 @@ export function useModal(
[key: string]: { prop: string; value: string }[]
}
- const resetImportForm = () => {
- state.importForm.name = ''
- state.importForm.file = ''
- }
-
- const handleImportDefinition = async () => {
- await state.importFormRef.validate()
-
- if (state.saving) return
- state.saving = true
- try {
- const formData = new FormData()
- formData.append('file', state.importForm.file)
- const code = Number(router.currentRoute.value.params.projectCode)
- await importWorkflowDefinition(formData, code)
- window.$message.success(t('project.workflow.success'))
- state.saving = false
- ctx.emit('updateList')
- ctx.emit('update:show')
- resetImportForm()
- } catch (err) {
- state.saving = false
- }
- }
-
const handleStartDefinition = async (code: number, version: number) => {
await state.startFormRef.validate()
@@ -300,7 +274,6 @@ export function useModal(
return {
variables,
- handleImportDefinition,
handleStartDefinition,
handleCreateTiming,
handleUpdateTiming,
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/version-modal.tsx b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/version-modal.tsx
index bbcf70bb7ada..728b0193bf47 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/components/version-modal.tsx
+++ b/dolphinscheduler-ui/src/views/projects/workflow/definition/components/version-modal.tsx
@@ -17,8 +17,6 @@
import { defineComponent, PropType, toRefs, watch } from 'vue'
import { useI18n } from 'vue-i18n'
-import { useForm } from './use-form'
-import { useModal } from './use-modal'
import { useTable } from './use-table'
import { NDataTable, NPagination, NSpace } from 'naive-ui'
import Modal from '@/components/modal'
@@ -45,8 +43,6 @@ export default defineComponent({
emits: ['update:show', 'update:row', 'updateList'],
setup(props, ctx) {
const { variables, createColumns, getTableData } = useTable(ctx)
- const { importState } = useForm()
- const { handleImportDefinition } = useModal(importState, ctx)
const requestData = () => {
if (props.show && props.row?.code) {
@@ -58,15 +54,6 @@ export default defineComponent({
ctx.emit('update:show')
}
- const handleImport = () => {
- handleImportDefinition()
- }
-
- const customRequest = ({ file }: any) => {
- importState.importForm.name = file.name
- importState.importForm.file = file.file
- }
-
watch(
() => props.show,
() => {
@@ -81,8 +68,6 @@ export default defineComponent({
return {
hideModal,
- handleImport,
- customRequest,
requestData,
...toRefs(variables)
}
diff --git a/dolphinscheduler-ui/src/views/projects/workflow/definition/index.tsx b/dolphinscheduler-ui/src/views/projects/workflow/definition/index.tsx
index 6d0a28cba433..34605f519c0a 100644
--- a/dolphinscheduler-ui/src/views/projects/workflow/definition/index.tsx
+++ b/dolphinscheduler-ui/src/views/projects/workflow/definition/index.tsx
@@ -37,7 +37,6 @@ import { useTable } from './use-table'
import { useRouter, useRoute } from 'vue-router'
import { useUISettingStore } from '@/store/ui-setting/ui-setting'
import Card from '@/components/card'
-import ImportModal from './components/import-modal'
import StartModal from './components/start-modal'
import TimingModal from './components/timing-modal'
import VersionModal from './components/version-modal'
@@ -60,7 +59,6 @@ export default defineComponent({
createColumns,
getTableData,
batchDeleteWorkflow,
- batchExportWorkflow,
batchCopyWorkflow
} = useTable()
@@ -135,7 +133,6 @@ export default defineComponent({
createDefinitionDynamic,
handleChangePageSize,
batchDeleteWorkflow,
- batchExportWorkflow,
batchCopyWorkflow,
handleCopyUpdateList,
...toRefs(variables),
@@ -169,14 +166,6 @@ export default defineComponent({
{t('project.workflow.create_workflow_dynamic')}
)}
- (this.showRef = true)}
- >
- {t('project.workflow.import_workflow')}
-
-
- {{
- default: () => t('project.workflow.batch_export'),
- trigger: () => (
-
- {t('project.workflow.batch_export')}
-
- )
- }}
-
{{
default: () => t('project.workflow.batch_copy'),
@@ -279,10 +251,6 @@ export default defineComponent({
-
releaseWorkflow(row),
onReleaseScheduler: () => releaseScheduler(row),
onCopyWorkflow: () => copyWorkflow(row),
- onExportWorkflow: () => exportWorkflow(row),
onGotoWorkflowTree: () => gotoWorkflowTree(row)
})
}
@@ -341,19 +339,6 @@ export function useTable() {
})
}
- const batchExportWorkflow = () => {
- const fileName = 'workflow_' + new Date().getTime()
- const data = {
- codes: _.join(variables.checkedRowKeys, ',')
- }
-
- batchExportByCodes(data, variables.projectCode).then((res: any) => {
- downloadBlob(res, fileName)
- window.$message.success(t('project.workflow.success'))
- variables.checkedRowKeys = []
- })
- }
-
const batchCopyWorkflow = () => {}
const confirmToOfflineWorkflow = () => {
@@ -511,42 +496,6 @@ export function useTable() {
})
}
- const downloadBlob = (data: any, fileNameS = 'json') => {
- if (!data) {
- return
- }
- const blob = new Blob([data])
- const fileName = `${fileNameS}.json`
- if ('download' in document.createElement('a')) {
- // Not IE
- const url = window.URL.createObjectURL(blob)
- const link = document.createElement('a')
- link.style.display = 'none'
- link.href = url
- link.setAttribute('download', fileName)
- document.body.appendChild(link)
- link.click()
- document.body.removeChild(link) // remove element after downloading is complete.
- window.URL.revokeObjectURL(url) // release blob object
- } else {
- // IE 10+
- if (window.navigator.msSaveBlob) {
- window.navigator.msSaveBlob(blob, fileName)
- }
- }
- }
-
- const exportWorkflow = (row: any) => {
- const fileName = 'workflow_' + new Date().getTime()
-
- const data = {
- codes: String(row.code)
- }
- batchExportByCodes(data, variables.projectCode).then((res: any) => {
- downloadBlob(res, fileName)
- })
- }
-
const gotoWorkflowTree = (row: any) => {
router.push({
name: 'workflow-definition-tree',
@@ -576,7 +525,6 @@ export function useTable() {
createColumns,
getTableData,
batchDeleteWorkflow,
- batchExportWorkflow,
batchCopyWorkflow
}
}
From a0da458980bef64d51a7dfb6d2338d655da1455d Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Tue, 3 Feb 2026 17:39:13 +0800
Subject: [PATCH 2/9] remove import and export function
---
.../src/main/resources/application.yaml | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index a0f0b9386ba8..71c309e6d732 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -169,7 +169,7 @@ master:
# master heartbeat interval
max-heartbeat-interval: 10s
server-load-protection:
- enabled: false
+ enabled: true
# Master max system cpu usage, when the master's system cpu usage is smaller then this value, master server can execute workflow.
max-system-cpu-usage-percentage-thresholds: 0.8
# Master max jvm cpu usage, when the master's jvm cpu usage is smaller then this value, master server can execute workflow.
@@ -205,7 +205,7 @@ worker:
# worker group name. If it is not set, the default value is default.
group: default
server-load-protection:
- enabled: false
+ enabled: true
# Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks.
max-system-cpu-usage-percentage-thresholds: 0.8
# Worker max jvm cpu usage, when the worker's jvm cpu usage is smaller then this value, worker server can be dispatched tasks.
From c7a3dc4e845b1e4d7e53770ae90b4fbdb3a2a26d Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Tue, 3 Feb 2026 18:06:33 +0800
Subject: [PATCH 3/9] remove import and export function
---
.../test/cases/WorkflowDefinitionAPITest.java | 203 ------------------
.../workflow/WorkflowDefinitionPage.java | 10 -
2 files changed, 213 deletions(-)
delete mode 100644 dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
deleted file mode 100644
index bd514adc48e7..000000000000
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
+++ /dev/null
@@ -1,203 +0,0 @@
-/*
- * 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.api.test.cases;
-
-import org.apache.dolphinscheduler.api.test.core.DolphinScheduler;
-import org.apache.dolphinscheduler.api.test.entity.HttpResponse;
-import org.apache.dolphinscheduler.api.test.entity.LoginResponseData;
-import org.apache.dolphinscheduler.api.test.pages.LoginPage;
-import org.apache.dolphinscheduler.api.test.pages.project.ProjectPage;
-import org.apache.dolphinscheduler.api.test.pages.workflow.WorkflowDefinitionPage;
-import org.apache.dolphinscheduler.api.test.utils.JSONUtils;
-import org.apache.dolphinscheduler.common.enums.ReleaseState;
-import org.apache.dolphinscheduler.common.enums.UserType;
-import org.apache.dolphinscheduler.dao.entity.User;
-
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.util.EntityUtils;
-
-import java.io.File;
-import java.util.LinkedHashMap;
-import java.util.List;
-
-import lombok.extern.slf4j.Slf4j;
-
-import org.junit.jupiter.api.AfterAll;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.BeforeAll;
-import org.junit.jupiter.api.Order;
-import org.junit.jupiter.api.Test;
-import org.junitpioneer.jupiter.DisableIfTestFails;
-
-@DolphinScheduler(composeFiles = "docker/basic/docker-compose.yaml")
-@Slf4j
-@DisableIfTestFails
-public class WorkflowDefinitionAPITest {
-
- private static final String username = "admin";
-
- private static final String password = "dolphinscheduler123";
-
- private static String sessionId;
-
- private static User loginUser;
-
- private static WorkflowDefinitionPage workflowDefinitionPage;
-
- private static ProjectPage projectPage;
-
- private static long projectCode;
-
- private static long workflowDefinitionCode;
-
- private static String workflowDefinitionName;
-
- @BeforeAll
- public static void setup() {
- LoginPage loginPage = new LoginPage();
- HttpResponse loginHttpResponse = loginPage.login(username, password);
- sessionId =
- JSONUtils.convertValue(loginHttpResponse.getBody().getData(), LoginResponseData.class).getSessionId();
- workflowDefinitionPage = new WorkflowDefinitionPage(sessionId);
- projectPage = new ProjectPage(sessionId);
- loginUser = new User();
- loginUser.setId(123);
- loginUser.setUserType(UserType.GENERAL_USER);
- }
-
- @AfterAll
- public static void cleanup() {
- log.info("success cleanup");
- }
-
- @Test
- @Order(1)
- public void testImportWorkflowDefinition() {
- try {
- HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test");
- HttpResponse queryAllProjectListResponse = projectPage.queryAllProjectList(loginUser);
- Assertions.assertTrue(queryAllProjectListResponse.getBody().getSuccess());
-
- projectCode = (long) ((LinkedHashMap) ((List) queryAllProjectListResponse
- .getBody().getData()).get(0)).get("code");
- ClassLoader classLoader = getClass().getClassLoader();
- File file = new File(classLoader.getResource("workflow-json/test.json").getFile());
- CloseableHttpResponse importWorkflowDefinitionResponse = workflowDefinitionPage
- .importWorkflowDefinition(loginUser, projectCode, file);
- String data = EntityUtils.toString(importWorkflowDefinitionResponse.getEntity());
- Assertions.assertTrue(data.contains("\"success\":true"));
- } catch (Exception e) {
- log.error("failed", e);
- Assertions.fail();
- }
- }
-
- @Test
- @Order(2)
- public void testQueryAllWorkflowDefinitionByProjectCode() {
- HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
- workflowDefinitionPage.queryAllWorkflowDefinitionByProjectCode(loginUser, projectCode);
- Assertions.assertTrue(queryAllWorkflowDefinitionByProjectCodeResponse.getBody().getSuccess());
- Assertions.assertTrue(
- queryAllWorkflowDefinitionByProjectCodeResponse.getBody().getData().toString().contains("hello world"));
- workflowDefinitionCode =
- (long) ((LinkedHashMap) ((LinkedHashMap) ((List) queryAllWorkflowDefinitionByProjectCodeResponse
- .getBody().getData()).get(0)).get("workflowDefinition")).get("code");
- workflowDefinitionName =
- (String) ((LinkedHashMap) ((LinkedHashMap) ((List) queryAllWorkflowDefinitionByProjectCodeResponse
- .getBody().getData()).get(0)).get("workflowDefinition")).get("name");
- }
-
- @Test
- @Order(3)
- public void testQueryWorkflowDefinitionByCode() {
- HttpResponse queryWorkflowDefinitionByCodeResponse =
- workflowDefinitionPage.queryWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
- Assertions.assertTrue(queryWorkflowDefinitionByCodeResponse.getBody().getSuccess());
- Assertions.assertTrue(
- queryWorkflowDefinitionByCodeResponse.getBody().getData().toString().contains("hello world"));
- }
-
- @Test
- @Order(4)
- public void testGetWorkflowListByProjectCode() {
- HttpResponse getWorkflowListByProjectCodeResponse =
- workflowDefinitionPage.getWorkflowListByProjectCode(loginUser, projectCode);
- Assertions.assertTrue(getWorkflowListByProjectCodeResponse.getBody().getSuccess());
- Assertions
- .assertTrue(
- getWorkflowListByProjectCodeResponse.getBody().getData().toString().contains("test_import"));
- }
-
- @Test
- @Order(5)
- public void testQueryWorkflowDefinitionByName() {
- HttpResponse queryWorkflowDefinitionByNameResponse =
- workflowDefinitionPage.queryWorkflowDefinitionByName(loginUser, projectCode, workflowDefinitionName);
- Assertions.assertTrue(queryWorkflowDefinitionByNameResponse.getBody().getSuccess());
- Assertions.assertTrue(
- queryWorkflowDefinitionByNameResponse.getBody().getData().toString().contains("hello world"));
- }
-
- @Test
- @Order(6)
- public void testQueryWorkflowDefinitionList() {
- HttpResponse queryWorkflowDefinitionListResponse =
- workflowDefinitionPage.queryWorkflowDefinitionList(loginUser, projectCode);
- Assertions.assertTrue(queryWorkflowDefinitionListResponse.getBody().getSuccess());
- Assertions
- .assertTrue(queryWorkflowDefinitionListResponse.getBody().getData().toString().contains("hello world"));
- }
-
- @Test
- @Order(7)
- public void testReleaseWorkflowDefinition() {
- HttpResponse releaseWorkflowDefinitionResponse = workflowDefinitionPage.releaseWorkflowDefinition(loginUser,
- projectCode, workflowDefinitionCode, ReleaseState.ONLINE);
- Assertions.assertTrue(releaseWorkflowDefinitionResponse.getBody().getSuccess());
-
- HttpResponse queryWorkflowDefinitionByCodeResponse =
- workflowDefinitionPage.queryWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
- Assertions.assertTrue(queryWorkflowDefinitionByCodeResponse.getBody().getSuccess());
- Assertions.assertTrue(
- queryWorkflowDefinitionByCodeResponse.getBody().getData().toString().contains("releaseState=ONLINE"));
- }
-
- @Test
- @Order(8)
- public void testDeleteWorkflowDefinitionByCode() {
- HttpResponse deleteWorkflowDefinitionByCodeResponse =
- workflowDefinitionPage.deleteWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
- Assertions.assertFalse(deleteWorkflowDefinitionByCodeResponse.getBody().getSuccess());
-
- HttpResponse releaseWorkflowDefinitionResponse = workflowDefinitionPage.releaseWorkflowDefinition(loginUser,
- projectCode, workflowDefinitionCode, ReleaseState.OFFLINE);
- Assertions.assertTrue(releaseWorkflowDefinitionResponse.getBody().getSuccess());
-
- deleteWorkflowDefinitionByCodeResponse =
- workflowDefinitionPage.deleteWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
- Assertions.assertTrue(deleteWorkflowDefinitionByCodeResponse.getBody().getSuccess());
-
- HttpResponse queryWorkflowDefinitionListResponse =
- workflowDefinitionPage.queryWorkflowDefinitionList(loginUser, projectCode);
- Assertions.assertTrue(queryWorkflowDefinitionListResponse.getBody().getSuccess());
- Assertions
- .assertFalse(
- queryWorkflowDefinitionListResponse.getBody().getData().toString().contains("hello world"));
- }
-}
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
index cdfa44083eb7..6944f076470f 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
@@ -38,16 +38,6 @@ public class WorkflowDefinitionPage {
private String sessionId;
- public CloseableHttpResponse importWorkflowDefinition(User loginUser, long projectCode, File file) {
- Map params = new HashMap<>();
- params.put("loginUser", loginUser);
- Map headers = new HashMap<>();
- headers.put(Constants.SESSION_ID_KEY, sessionId);
- RequestClient requestClient = new RequestClient();
- String url = String.format("/projects/%s/workflow-definition/import", projectCode);
- return requestClient.postWithFile(url, headers, params, file);
- }
-
public HttpResponse queryAllWorkflowDefinitionByProjectCode(User loginUser, long projectCode) {
Map params = new HashMap<>();
params.put("loginUser", loginUser);
From 36631590b97b2adf04d934dbb527f9ef7af4095f Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Tue, 3 Feb 2026 18:06:55 +0800
Subject: [PATCH 4/9] remove import and export function
---
.../api/test/pages/workflow/WorkflowDefinitionPage.java | 3 ---
1 file changed, 3 deletions(-)
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
index 6944f076470f..531935935c41 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
@@ -23,9 +23,6 @@
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.http.client.methods.CloseableHttpResponse;
-
-import java.io.File;
import java.util.HashMap;
import java.util.Map;
From 4518058ca25add24253124476b97455148bcbf53 Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Tue, 3 Feb 2026 18:17:41 +0800
Subject: [PATCH 5/9] remove import and export function
---
.github/workflows/api-test.yml | 2 --
1 file changed, 2 deletions(-)
diff --git a/.github/workflows/api-test.yml b/.github/workflows/api-test.yml
index b1668aad9169..867f3cb0892b 100644
--- a/.github/workflows/api-test.yml
+++ b/.github/workflows/api-test.yml
@@ -103,8 +103,6 @@ jobs:
class: org.apache.dolphinscheduler.api.test.cases.WorkerGroupAPITest
- name: ProjectAPITest
class: org.apache.dolphinscheduler.api.test.cases.ProjectAPITest
- - name: WorkflowDefinitionAPITest
- class: org.apache.dolphinscheduler.api.test.cases.WorkflowDefinitionAPITest
- name: SchedulerAPITest
class: org.apache.dolphinscheduler.api.test.cases.SchedulerAPITest
- name: ExecutorAPITest
From fda977a7c552e06e185f48f8b742ddfea7b0f959 Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Wed, 4 Feb 2026 10:54:34 +0800
Subject: [PATCH 6/9] remove import and export function
---
.github/workflows/api-test.yml | 2 +
.../api/test/cases/ExecutorAPITest.java | 12 +-
.../api/test/cases/SchedulerAPITest.java | 4 +-
.../test/cases/WorkflowDefinitionAPITest.java | 202 ++++++++++++++++++
.../test/cases/WorkflowInstanceAPITest.java | 26 +--
.../api/test/cases/tasks/GrpcTaskAPITest.java | 19 +-
.../workflow/WorkflowDefinitionPage.java | 38 ++++
.../task-grpc/grpcFailedWorkflow.json | 106 +++------
.../task-grpc/grpcSuccessWorkflow.json | 106 +++------
.../test/resources/workflow-json/test.json | 124 ++++-------
10 files changed, 378 insertions(+), 261 deletions(-)
create mode 100644 dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
diff --git a/.github/workflows/api-test.yml b/.github/workflows/api-test.yml
index 867f3cb0892b..b1668aad9169 100644
--- a/.github/workflows/api-test.yml
+++ b/.github/workflows/api-test.yml
@@ -103,6 +103,8 @@ jobs:
class: org.apache.dolphinscheduler.api.test.cases.WorkerGroupAPITest
- name: ProjectAPITest
class: org.apache.dolphinscheduler.api.test.cases.ProjectAPITest
+ - name: WorkflowDefinitionAPITest
+ class: org.apache.dolphinscheduler.api.test.cases.WorkflowDefinitionAPITest
- name: SchedulerAPITest
class: org.apache.dolphinscheduler.api.test.cases.SchedulerAPITest
- name: ExecutorAPITest
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java
index bf1d05691ac9..667016f5b8a8 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/ExecutorAPITest.java
@@ -31,9 +31,6 @@
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.util.EntityUtils;
-
import java.io.File;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -75,6 +72,8 @@ public class ExecutorAPITest {
private static List workflowInstanceIds;
+ private static String workflowDefinitionName = "test" + System.currentTimeMillis();
+
@BeforeAll
public static void setup() {
LoginPage loginPage = new LoginPage();
@@ -109,10 +108,9 @@ public void testStartWorkflowInstance() {
// upload test workflow definition json
ClassLoader classLoader = getClass().getClassLoader();
File file = new File(classLoader.getResource("workflow-json/test.json").getFile());
- CloseableHttpResponse importWorkflowDefinitionResponse = workflowDefinitionPage
- .importWorkflowDefinition(loginUser, projectCode, file);
- String data = EntityUtils.toString(importWorkflowDefinitionResponse.getEntity());
- Assertions.assertTrue(data.contains("\"success\":true"));
+ HttpResponse createWorkflowDefinitionResponse = workflowDefinitionPage
+ .createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
+ Assertions.assertTrue(createWorkflowDefinitionResponse.getBody().getSuccess());
// get workflow definition code
HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/SchedulerAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/SchedulerAPITest.java
index 495236dc4b46..2919b0a66f08 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/SchedulerAPITest.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/SchedulerAPITest.java
@@ -67,6 +67,8 @@ public class SchedulerAPITest {
private static int scheduleId;
+ private static String workflowDefinitionName = "test" + System.currentTimeMillis();
+
@BeforeAll
public static void setup() {
LoginPage loginPage = new LoginPage();
@@ -98,7 +100,7 @@ public void testCreateSchedule() {
.getBody().getData()).get(0)).get("code");
ClassLoader classLoader = getClass().getClassLoader();
File file = new File(classLoader.getResource("workflow-json/test.json").getFile());
- workflowDefinitionPage.importWorkflowDefinition(loginUser, projectCode, file);
+ workflowDefinitionPage.createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
workflowDefinitionPage.queryAllWorkflowDefinitionByProjectCode(loginUser, projectCode);
Assertions.assertTrue(queryAllWorkflowDefinitionByProjectCodeResponse.getBody().getSuccess());
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
new file mode 100644
index 000000000000..b58e268e6ca0
--- /dev/null
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowDefinitionAPITest.java
@@ -0,0 +1,202 @@
+/*
+ * 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.api.test.cases;
+
+import org.apache.dolphinscheduler.api.test.core.DolphinScheduler;
+import org.apache.dolphinscheduler.api.test.entity.HttpResponse;
+import org.apache.dolphinscheduler.api.test.entity.LoginResponseData;
+import org.apache.dolphinscheduler.api.test.pages.LoginPage;
+import org.apache.dolphinscheduler.api.test.pages.project.ProjectPage;
+import org.apache.dolphinscheduler.api.test.pages.workflow.WorkflowDefinitionPage;
+import org.apache.dolphinscheduler.api.test.utils.JSONUtils;
+import org.apache.dolphinscheduler.common.enums.ReleaseState;
+import org.apache.dolphinscheduler.common.enums.UserType;
+import org.apache.dolphinscheduler.dao.entity.User;
+
+import java.io.File;
+import java.util.LinkedHashMap;
+import java.util.List;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Order;
+import org.junit.jupiter.api.Test;
+import org.junitpioneer.jupiter.DisableIfTestFails;
+
+@DolphinScheduler(composeFiles = "docker/basic/docker-compose.yaml")
+@Slf4j
+@DisableIfTestFails
+public class WorkflowDefinitionAPITest {
+
+ private static final String username = "admin";
+
+ private static final String password = "dolphinscheduler123";
+
+ private static String sessionId;
+
+ private static User loginUser;
+
+ private static WorkflowDefinitionPage workflowDefinitionPage;
+
+ private static ProjectPage projectPage;
+
+ private static long projectCode;
+
+ private static long workflowDefinitionCode;
+
+ private static String projectName = "project-test" + System.currentTimeMillis();
+
+ private static String workflowDefinitionName = "test" + System.currentTimeMillis();
+
+ @BeforeAll
+ public static void setup() {
+ LoginPage loginPage = new LoginPage();
+ HttpResponse loginHttpResponse = loginPage.login(username, password);
+ sessionId =
+ JSONUtils.convertValue(loginHttpResponse.getBody().getData(), LoginResponseData.class).getSessionId();
+ workflowDefinitionPage = new WorkflowDefinitionPage(sessionId);
+ projectPage = new ProjectPage(sessionId);
+ loginUser = new User();
+ loginUser.setId(123);
+ loginUser.setUserType(UserType.GENERAL_USER);
+ }
+
+ @AfterAll
+ public static void cleanup() {
+ log.info("success cleanup");
+ }
+
+ @Test
+ @Order(1)
+ public void testCreateWorkflowDefinition() {
+ try {
+ projectPage.createProject(loginUser, projectName);
+ HttpResponse queryAllProjectListResponse = projectPage.queryAllProjectList(loginUser);
+ Assertions.assertTrue(queryAllProjectListResponse.getBody().getSuccess());
+
+ projectCode = (long) ((LinkedHashMap) ((List) queryAllProjectListResponse
+ .getBody().getData()).get(0)).get("code");
+ ClassLoader classLoader = getClass().getClassLoader();
+ File file = new File(classLoader.getResource("workflow-json/test.json").getFile());
+ HttpResponse createWorkflowDefinitionResponse = workflowDefinitionPage
+ .createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
+ Boolean successFlag = createWorkflowDefinitionResponse.getBody().getSuccess();
+ Assertions.assertTrue(successFlag);
+ } catch (Exception e) {
+ log.error("failed", e);
+ Assertions.fail();
+ }
+ }
+
+ @Test
+ @Order(2)
+ public void testQueryAllWorkflowDefinitionByProjectCode() {
+ HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
+ workflowDefinitionPage.queryAllWorkflowDefinitionByProjectCode(loginUser, projectCode);
+ Assertions.assertTrue(queryAllWorkflowDefinitionByProjectCodeResponse.getBody().getSuccess());
+ Assertions.assertTrue(
+ queryAllWorkflowDefinitionByProjectCodeResponse.getBody().getData().toString().contains("hello world"));
+ workflowDefinitionCode =
+ (long) ((LinkedHashMap) ((LinkedHashMap) ((List) queryAllWorkflowDefinitionByProjectCodeResponse
+ .getBody().getData()).get(0)).get("workflowDefinition")).get("code");
+ workflowDefinitionName =
+ (String) ((LinkedHashMap) ((LinkedHashMap) ((List) queryAllWorkflowDefinitionByProjectCodeResponse
+ .getBody().getData()).get(0)).get("workflowDefinition")).get("name");
+ }
+
+ @Test
+ @Order(3)
+ public void testQueryWorkflowDefinitionByCode() {
+ HttpResponse queryWorkflowDefinitionByCodeResponse =
+ workflowDefinitionPage.queryWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
+ Assertions.assertTrue(queryWorkflowDefinitionByCodeResponse.getBody().getSuccess());
+ Assertions.assertTrue(
+ queryWorkflowDefinitionByCodeResponse.getBody().getData().toString().contains("hello world"));
+ }
+
+ @Test
+ @Order(4)
+ public void testGetWorkflowListByProjectCode() {
+ HttpResponse getWorkflowListByProjectCodeResponse =
+ workflowDefinitionPage.getWorkflowListByProjectCode(loginUser, projectCode);
+ Assertions.assertTrue(getWorkflowListByProjectCodeResponse.getBody().getSuccess());
+ Assertions
+ .assertTrue(
+ getWorkflowListByProjectCodeResponse.getBody().getData().toString().contains("test"));
+ }
+
+ @Test
+ @Order(5)
+ public void testQueryWorkflowDefinitionByName() {
+ HttpResponse queryWorkflowDefinitionByNameResponse =
+ workflowDefinitionPage.queryWorkflowDefinitionByName(loginUser, projectCode, workflowDefinitionName);
+ Assertions.assertTrue(queryWorkflowDefinitionByNameResponse.getBody().getSuccess());
+ Assertions.assertTrue(
+ queryWorkflowDefinitionByNameResponse.getBody().getData().toString().contains(workflowDefinitionName));
+ }
+
+ @Test
+ @Order(6)
+ public void testQueryWorkflowDefinitionList() {
+ HttpResponse queryWorkflowDefinitionListResponse =
+ workflowDefinitionPage.queryWorkflowDefinitionList(loginUser, projectCode);
+ Assertions.assertTrue(queryWorkflowDefinitionListResponse.getBody().getSuccess());
+ Assertions
+ .assertTrue(queryWorkflowDefinitionListResponse.getBody().getData().toString().contains("hello world"));
+ }
+
+ @Test
+ @Order(7)
+ public void testReleaseWorkflowDefinition() {
+ HttpResponse releaseWorkflowDefinitionResponse = workflowDefinitionPage.releaseWorkflowDefinition(loginUser,
+ projectCode, workflowDefinitionCode, ReleaseState.ONLINE);
+ Assertions.assertTrue(releaseWorkflowDefinitionResponse.getBody().getSuccess());
+
+ HttpResponse queryWorkflowDefinitionByCodeResponse =
+ workflowDefinitionPage.queryWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
+ Assertions.assertTrue(queryWorkflowDefinitionByCodeResponse.getBody().getSuccess());
+ Assertions.assertTrue(
+ queryWorkflowDefinitionByCodeResponse.getBody().getData().toString().contains("releaseState=ONLINE"));
+ }
+
+ @Test
+ @Order(8)
+ public void testDeleteWorkflowDefinitionByCode() {
+ HttpResponse deleteWorkflowDefinitionByCodeResponse =
+ workflowDefinitionPage.deleteWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
+ Assertions.assertFalse(deleteWorkflowDefinitionByCodeResponse.getBody().getSuccess());
+
+ HttpResponse releaseWorkflowDefinitionResponse = workflowDefinitionPage.releaseWorkflowDefinition(loginUser,
+ projectCode, workflowDefinitionCode, ReleaseState.OFFLINE);
+ Assertions.assertTrue(releaseWorkflowDefinitionResponse.getBody().getSuccess());
+
+ deleteWorkflowDefinitionByCodeResponse =
+ workflowDefinitionPage.deleteWorkflowDefinitionByCode(loginUser, projectCode, workflowDefinitionCode);
+ Assertions.assertTrue(deleteWorkflowDefinitionByCodeResponse.getBody().getSuccess());
+
+ HttpResponse queryWorkflowDefinitionListResponse =
+ workflowDefinitionPage.queryWorkflowDefinitionList(loginUser, projectCode);
+ Assertions.assertTrue(queryWorkflowDefinitionListResponse.getBody().getSuccess());
+ Assertions
+ .assertFalse(
+ queryWorkflowDefinitionListResponse.getBody().getData().toString().contains("hello world"));
+ }
+}
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowInstanceAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowInstanceAPITest.java
index ded0bdf85386..a13019df8f19 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowInstanceAPITest.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/WorkflowInstanceAPITest.java
@@ -35,9 +35,6 @@
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.util.EntityUtils;
-
import java.io.File;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -83,6 +80,10 @@ public class WorkflowInstanceAPITest {
private static int workflowInstanceId;
+ private static String projectName = "project-test" + System.currentTimeMillis();
+
+ private static String workflowDefinitionName = "test" + System.currentTimeMillis();
+
@BeforeAll
public static void setup() {
LoginPage loginPage = new LoginPage();
@@ -109,7 +110,7 @@ public static void cleanup() {
public void testQueryWorkflowInstancesByWorkflowInstanceId() {
try {
// create test project
- HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test");
+ HttpResponse createProjectResponse = projectPage.createProject(loginUser, projectName);
HttpResponse queryAllProjectListResponse = projectPage.queryAllProjectList(loginUser);
assertTrue(queryAllProjectListResponse.getBody().getSuccess());
projectCode = (long) ((LinkedHashMap) ((List) queryAllProjectListResponse
@@ -118,10 +119,9 @@ public void testQueryWorkflowInstancesByWorkflowInstanceId() {
// upload test workflow definition json
ClassLoader classLoader = getClass().getClassLoader();
File file = new File(classLoader.getResource("workflow-json/test.json").getFile());
- CloseableHttpResponse importWorkflowDefinitionResponse = workflowDefinitionPage
- .importWorkflowDefinition(loginUser, projectCode, file);
- String data = EntityUtils.toString(importWorkflowDefinitionResponse.getEntity());
- assertTrue(data.contains("\"success\":true"));
+ HttpResponse createWorkflowDefinitionResponse = workflowDefinitionPage
+ .createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
+ assertTrue(createWorkflowDefinitionResponse.getBody().getSuccess());
// get workflow definition code
HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
@@ -176,7 +176,7 @@ public void testQueryWorkflowInstanceList() {
HttpResponse queryWorkflowInstanceListResponse =
workflowInstancePage.queryWorkflowInstanceList(loginUser, projectCode, 1, 10);
assertTrue(queryWorkflowInstanceListResponse.getBody().getSuccess());
- assertTrue(queryWorkflowInstanceListResponse.getBody().getData().toString().contains("test_import"));
+ assertTrue(queryWorkflowInstanceListResponse.getBody().getData().toString().contains(workflowDefinitionName));
}
@Test
@@ -185,7 +185,8 @@ public void testQueryTaskListByWorkflowInstanceId() {
HttpResponse queryTaskListByWorkflowInstanceIdResponse =
workflowInstancePage.queryTaskListByWorkflowInstanceId(loginUser, projectCode, workflowInstanceId);
assertTrue(queryTaskListByWorkflowInstanceIdResponse.getBody().getSuccess());
- assertTrue(queryTaskListByWorkflowInstanceIdResponse.getBody().getData().toString().contains("test_import"));
+ assertTrue(queryTaskListByWorkflowInstanceIdResponse.getBody().getData().toString()
+ .contains(workflowDefinitionName));
}
@Test
@@ -194,7 +195,7 @@ public void testQueryWorkflowInstanceById() {
HttpResponse queryWorkflowInstanceByIdResponse =
workflowInstancePage.queryWorkflowInstanceById(loginUser, projectCode, workflowInstanceId);
assertTrue(queryWorkflowInstanceByIdResponse.getBody().getSuccess());
- assertTrue(queryWorkflowInstanceByIdResponse.getBody().getData().toString().contains("test_import"));
+ assertTrue(queryWorkflowInstanceByIdResponse.getBody().getData().toString().contains(workflowDefinitionName));
}
@Test
@@ -208,7 +209,8 @@ public void testDeleteWorkflowInstanceById() {
workflowInstancePage.queryWorkflowInstanceList(loginUser, projectCode, 1, 10);
assertTrue(queryWorkflowInstanceListResponse.getBody().getSuccess());
Assertions
- .assertFalse(queryWorkflowInstanceListResponse.getBody().getData().toString().contains("test_import"));
+ .assertFalse(queryWorkflowInstanceListResponse.getBody().getData().toString()
+ .contains(workflowDefinitionName));
}
}
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
index bd7cb3f918d1..3b637757ba37 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
@@ -31,9 +31,6 @@
import org.apache.dolphinscheduler.common.enums.WarningType;
import org.apache.dolphinscheduler.dao.entity.User;
-import org.apache.http.client.methods.CloseableHttpResponse;
-import org.apache.http.util.EntityUtils;
-
import java.io.File;
import java.text.SimpleDateFormat;
import java.util.Date;
@@ -73,6 +70,8 @@ public class GrpcTaskAPITest {
private static List workflowInstanceIds;
+ private static String workflowDefinitionName = "test" + System.currentTimeMillis();
+
@BeforeAll
public static void setup() {
LoginPage loginPage = new LoginPage();
@@ -106,10 +105,9 @@ public void testGrpcFailedWorkflowInstance() {
// upload test workflow definition json
ClassLoader classLoader = getClass().getClassLoader();
File file = new File(classLoader.getResource("workflow-json/task-grpc/grpcFailedWorkflow.json").getFile());
- CloseableHttpResponse importWorkflowDefinitionResponse = workflowDefinitionPage
- .importWorkflowDefinition(loginUser, projectCode, file);
- String data = EntityUtils.toString(importWorkflowDefinitionResponse.getEntity());
- Assertions.assertTrue(data.contains("\"success\":true"));
+ HttpResponse createWorkflowDefinitionResponse = workflowDefinitionPage
+ .createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
+ Assertions.assertTrue(createWorkflowDefinitionResponse.getBody().getSuccess());
// get workflow definition code
HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
@@ -155,10 +153,9 @@ public void testGrpcSuccessWorkflowInstance() {
// upload test workflow definition json
ClassLoader classLoader = getClass().getClassLoader();
File file = new File(classLoader.getResource("workflow-json/task-grpc/grpcSuccessWorkflow.json").getFile());
- CloseableHttpResponse importWorkflowDefinitionResponse = workflowDefinitionPage
- .importWorkflowDefinition(loginUser, projectCode, file);
- String data = EntityUtils.toString(importWorkflowDefinitionResponse.getEntity());
- Assertions.assertTrue(data.contains("\"success\":true"));
+ HttpResponse createWorkflowDefinitionResponse = workflowDefinitionPage
+ .createWorkflowDefinition(loginUser, projectCode, file, workflowDefinitionName);
+ Assertions.assertTrue(createWorkflowDefinitionResponse.getBody().getSuccess());
// get workflow definition code
HttpResponse queryAllWorkflowDefinitionByProjectCodeResponse =
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
index 531935935c41..e64c89893317 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/pages/workflow/WorkflowDefinitionPage.java
@@ -19,22 +19,60 @@
import org.apache.dolphinscheduler.api.test.core.Constants;
import org.apache.dolphinscheduler.api.test.entity.HttpResponse;
+import org.apache.dolphinscheduler.api.test.utils.JSONUtils;
import org.apache.dolphinscheduler.api.test.utils.RequestClient;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.dao.entity.User;
+import java.io.File;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
import java.util.HashMap;
+import java.util.List;
import java.util.Map;
import lombok.AllArgsConstructor;
+import lombok.SneakyThrows;
import lombok.extern.slf4j.Slf4j;
+import com.fasterxml.jackson.core.type.TypeReference;
+
@Slf4j
@AllArgsConstructor
public class WorkflowDefinitionPage {
private String sessionId;
+ @SneakyThrows
+ public HttpResponse createWorkflowDefinition(User loginUser,
+ long projectCode,
+ File file,
+ String workflowDefinitionName) {
+ Map params = new HashMap<>();
+ params.put("loginUser", loginUser);
+
+ String fileContent = new String(Files.readAllBytes(file.toPath()), StandardCharsets.UTF_8);
+ Map fileContentMap = JSONUtils.parseObject(fileContent, new TypeReference<>() {
+ });
+ if (fileContentMap == null) {
+ throw new RuntimeException("file content parse error");
+ }
+ fileContentMap.replaceAll((key, value) -> {
+ if (value instanceof List) {
+ return JSONUtils.toJsonString(value);
+ }
+ return value;
+ });
+ params.putAll(fileContentMap);
+ params.put("name", workflowDefinitionName);
+
+ Map headers = new HashMap<>();
+ headers.put(Constants.SESSION_ID_KEY, sessionId);
+ RequestClient requestClient = new RequestClient();
+ String url = String.format("/projects/%s/workflow-definition", projectCode);
+ return requestClient.post(url, headers, params);
+ }
+
public HttpResponse queryAllWorkflowDefinitionByProjectCode(User loginUser, long projectCode) {
Map params = new HashMap<>();
params.put("loginUser", loginUser);
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcFailedWorkflow.json b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcFailedWorkflow.json
index 9f82811715fb..1d2c9ed18b94 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcFailedWorkflow.json
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcFailedWorkflow.json
@@ -1,58 +1,14 @@
-[ {
- "workflowDefinition": {
- "id": 1,
- "code": 152040369702561,
- "name": "grpcSuccessWorkflow",
- "version": 1,
- "releaseState": "OFFLINE",
- "projectCode": 152040056703649,
- "description": "",
- "globalParams": "[]",
- "globalParamList": [],
- "globalParamMap": {},
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "flag": "YES",
- "userId": 1,
- "userName": null,
- "projectName": null,
- "locations": "[{\"taskCode\":152040321402528,\"x\":369,\"y\":264}]",
- "scheduleReleaseState": null,
- "schedule": null,
- "timeout": 0,
- "modifyBy": null,
- "warningGroupId": null,
- "executionType": "PARALLEL"
- },
- "workflowTaskRelationList": [
+{
+ "taskDefinitionJson": [
{
- "id": 1,
- "name": "",
- "workflowDefinitionVersion": 1,
- "projectCode": 152040056703649,
- "workflowDefinitionCode": 152040369702561,
- "preTaskCode": 0,
- "preTaskVersion": 0,
- "postTaskCode": 152040321402529,
- "postTaskVersion": 1,
- "conditionType": "NONE",
- "conditionParams": {},
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "operator": 1,
- "operateTime": "2025-09-15 12:35:23"
- }
- ],
- "taskDefinitionList": [
- {
- "id": 1,
"code": 152040321402529,
- "name": "grpc_example",
- "version": 1,
+ "delayTime": "0",
"description": "",
- "projectCode": 152040056703649,
- "userId": 1,
- "taskType": "GRPC",
+ "environmentCode": -1,
+ "failRetryInterval": "1",
+ "failRetryTimes": "0",
+ "flag": "YES",
+ "name": "grpc_example",
"taskParams": {
"localParams": [],
"resourceList": [],
@@ -66,31 +22,31 @@
"grpcConnectTimeoutMs": 60000,
"socketTimeout": 60000
},
- "taskParamList": [],
- "taskParamMap": null,
- "flag": "YES",
"taskPriority": "MEDIUM",
- "userName": null,
- "projectName": null,
- "workerGroup": "default",
- "environmentCode": -1,
- "failRetryTimes": 0,
- "failRetryInterval": 1,
- "timeoutFlag": "CLOSE",
- "timeoutNotifyStrategy": null,
+ "taskType": "GRPC",
"timeout": 0,
- "delayTime": 0,
- "resourceIds": null,
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "modifyBy": null,
- "taskGroupId": 0,
- "taskGroupPriority": 0,
+ "timeoutFlag": "CLOSE",
+ "timeoutNotifyStrategy": "",
+ "workerGroup": "default",
"cpuQuota": -1,
"memoryMax": -1,
- "taskExecuteType": "BATCH",
- "operator": 1,
- "operateTime": "2025-09-15 12:35:23"
+ "taskExecuteType": "BATCH"
}
- ]
-} ]
\ No newline at end of file
+ ],
+ "taskRelationJson": [
+ {
+ "name": "",
+ "preTaskCode": 0,
+ "preTaskVersion": 0,
+ "postTaskCode": 152040321402529,
+ "postTaskVersion": 1,
+ "conditionType": "NONE",
+ "conditionParams": {}
+ }
+ ],
+ "name": "grpcSuccessWorkflow",
+ "executionType": "PARALLEL",
+ "description": "",
+ "globalParams": [],
+ "timeout": 0
+}
\ No newline at end of file
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcSuccessWorkflow.json b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcSuccessWorkflow.json
index 68bc083e0898..8c13c5e63841 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcSuccessWorkflow.json
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/task-grpc/grpcSuccessWorkflow.json
@@ -1,58 +1,14 @@
-[ {
- "workflowDefinition": {
- "id": 1,
- "code": 152040369702560,
- "name": "grpcSuccessWorkflow",
- "version": 1,
- "releaseState": "OFFLINE",
- "projectCode": 152040056703648,
- "description": "",
- "globalParams": "[]",
- "globalParamList": [],
- "globalParamMap": {},
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "flag": "YES",
- "userId": 1,
- "userName": null,
- "projectName": null,
- "locations": "[{\"taskCode\":152040321402528,\"x\":369,\"y\":264}]",
- "scheduleReleaseState": null,
- "schedule": null,
- "timeout": 0,
- "modifyBy": null,
- "warningGroupId": null,
- "executionType": "PARALLEL"
- },
- "workflowTaskRelationList": [
+{
+ "taskDefinitionJson": [
{
- "id": 1,
- "name": "",
- "workflowDefinitionVersion": 1,
- "projectCode": 152040056703648,
- "workflowDefinitionCode": 152040369702560,
- "preTaskCode": 0,
- "preTaskVersion": 0,
- "postTaskCode": 152040321402528,
- "postTaskVersion": 1,
- "conditionType": "NONE",
- "conditionParams": {},
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "operator": 1,
- "operateTime": "2025-09-15 12:35:23"
- }
- ],
- "taskDefinitionList": [
- {
- "id": 1,
"code": 152040321402528,
- "name": "grpc_example",
- "version": 1,
+ "delayTime": "0",
"description": "",
- "projectCode": 152040056703648,
- "userId": 1,
- "taskType": "GRPC",
+ "environmentCode": -1,
+ "failRetryInterval": "1",
+ "failRetryTimes": "0",
+ "flag": "YES",
+ "name": "grpc_example",
"taskParams": {
"localParams": [],
"resourceList": [],
@@ -66,31 +22,31 @@
"grpcConnectTimeoutMs": 60000,
"socketTimeout": 60000
},
- "taskParamList": [],
- "taskParamMap": null,
- "flag": "YES",
"taskPriority": "MEDIUM",
- "userName": null,
- "projectName": null,
- "workerGroup": "default",
- "environmentCode": -1,
- "failRetryTimes": 0,
- "failRetryInterval": 1,
- "timeoutFlag": "CLOSE",
- "timeoutNotifyStrategy": null,
+ "taskType": "GRPC",
"timeout": 0,
- "delayTime": 0,
- "resourceIds": null,
- "createTime": "2025-09-15 12:35:23",
- "updateTime": "2025-09-15 12:35:23",
- "modifyBy": null,
- "taskGroupId": 0,
- "taskGroupPriority": 0,
+ "timeoutFlag": "CLOSE",
+ "timeoutNotifyStrategy": "",
+ "workerGroup": "default",
"cpuQuota": -1,
"memoryMax": -1,
- "taskExecuteType": "BATCH",
- "operator": 1,
- "operateTime": "2025-09-15 12:35:23"
+ "taskExecuteType": "BATCH"
}
- ]
-} ]
\ No newline at end of file
+ ],
+ "taskRelationJson": [
+ {
+ "name": "",
+ "preTaskCode": 0,
+ "preTaskVersion": 0,
+ "postTaskCode": 152040321402528,
+ "postTaskVersion": 1,
+ "conditionType": "NONE",
+ "conditionParams": {}
+ }
+ ],
+ "name": "grpcSuccessWorkflow",
+ "executionType": "PARALLEL",
+ "description": "",
+ "globalParams": [],
+ "timeout": 0
+}
\ No newline at end of file
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
index c3cb8a5741e6..54c5ec735136 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
@@ -1,80 +1,44 @@
-[ {
- "workflowDefinition" : {
- "id" : 1,
- "code" : 9752686452032,
- "name" : "test",
- "version" : 1,
- "releaseState" : "OFFLINE",
- "projectCode" : 9752680865600,
- "description" : "",
- "globalParams" : "[]",
- "globalParamList" : [ ],
- "globalParamMap" : { },
- "createTime" : "2023-06-01 20:41:02",
- "updateTime" : "2023-06-01 20:41:02",
- "flag" : "YES",
- "userId" : 1,
- "userName" : null,
- "projectName" : null,
- "locations" : "[{\"taskCode\":9752684723008,\"x\":132,\"y\":112}]",
- "scheduleReleaseState" : null,
- "timeout" : 0,
- "modifyBy" : null,
- "warningGroupId" : null,
- "executionType" : "PARALLEL"
- },
- "workflowTaskRelationList" : [ {
- "id" : 1,
- "name" : "",
- "workflowDefinitionVersion" : 1,
- "projectCode" : 9752680865600,
- "workflowDefinitionCode" : 9752686452032,
- "preTaskCode" : 0,
- "preTaskVersion" : 0,
- "postTaskCode" : 9752684723008,
- "postTaskVersion" : 1,
- "conditionType" : "NONE",
- "conditionParams" : {},
- "createTime" : "2023-06-01 20:41:02",
- "updateTime" : "2023-06-01 20:41:02",
- "operator" : 1,
- "operateTime" : "2023-06-01 20:41:02"
- } ],
- "taskDefinitionList" : [ {
- "id" : 1,
- "code" : 9752684723008,
- "name" : "test",
- "version" : 1,
- "description" : "",
- "projectCode" : 9752680865600,
- "userId" : 1,
- "taskType" : "SHELL",
- "taskParams" : {"localParams":[],"rawScript":"echo \"hello world\"","resourceList":[]},
- "taskParamList" : [ ],
- "taskParamMap" : null,
- "flag" : "YES",
- "taskPriority" : "MEDIUM",
- "userName" : null,
- "projectName" : null,
- "workerGroup" : "default",
- "environmentCode" : -1,
- "failRetryTimes" : 0,
- "failRetryInterval" : 1,
- "timeoutFlag" : "CLOSE",
- "timeoutNotifyStrategy" : null,
- "timeout" : 0,
- "delayTime" : 0,
- "resourceIds" : null,
- "createTime" : "2023-06-01 20:41:02",
- "updateTime" : "2023-06-01 20:41:02",
- "modifyBy" : null,
- "taskGroupId" : 0,
- "taskGroupPriority" : 0,
- "cpuQuota" : -1,
- "memoryMax" : -1,
- "taskExecuteType" : "BATCH",
- "operator" : 1,
- "operateTime" : "2023-06-01 20:41:02"
- } ],
- "schedule" : null
-} ]
+{
+ "taskDefinitionJson" : [
+ {
+ "code": 164597693773600,
+ "delayTime": "0",
+ "description": "",
+ "environmentCode": -1,
+ "failRetryInterval": "1",
+ "failRetryTimes": "0",
+ "flag": "YES",
+ "name": "test1",
+ "taskParams": {
+ "localParams": [],
+ "rawScript": "hello world",
+ "resourceList": []
+ },
+ "taskPriority": "MEDIUM",
+ "taskType": "SHELL",
+ "timeout": 0,
+ "timeoutFlag": "CLOSE",
+ "timeoutNotifyStrategy": "",
+ "workerGroup": "default",
+ "cpuQuota": -1,
+ "memoryMax": -1,
+ "taskExecuteType": "BATCH"
+ }
+ ],
+ "taskRelationJson" : [
+ {
+ "name": "",
+ "preTaskCode": 0,
+ "preTaskVersion": 0,
+ "postTaskCode": 164597693773600,
+ "postTaskVersion": 0,
+ "conditionType": "NONE",
+ "conditionParams": {}
+ }
+ ],
+ "name" : "test1",
+ "executionType": "PARALLEL",
+ "description": "",
+ "globalParams": [],
+ "timeout": 0
+}
From 9c157a11fb016de9812b3f8e98b67ced2416da1a Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Wed, 4 Feb 2026 11:32:04 +0800
Subject: [PATCH 7/9] remove import and export function
---
.../api/test/cases/tasks/GrpcTaskAPITest.java | 9 ++++++---
.../WorkflowDefinitionControllerTest.java | 14 --------------
.../src/main/resources/application.yaml | 4 ++--
.../src/main/resources/logback-spring.xml | 1 +
4 files changed, 9 insertions(+), 19 deletions(-)
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
index 3b637757ba37..df7ceb2fa178 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/java/org/apache/dolphinscheduler/api/test/cases/tasks/GrpcTaskAPITest.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.dolphinscheduler.api.test.cases;
+package org.apache.dolphinscheduler.api.test.cases.tasks;
import org.apache.dolphinscheduler.api.test.core.DolphinScheduler;
import org.apache.dolphinscheduler.api.test.entity.HttpResponse;
@@ -42,6 +42,7 @@
import org.junit.jupiter.api.AfterAll;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Order;
import org.junit.jupiter.api.Test;
import org.junitpioneer.jupiter.DisableIfTestFails;
@@ -70,8 +71,6 @@ public class GrpcTaskAPITest {
private static List workflowInstanceIds;
- private static String workflowDefinitionName = "test" + System.currentTimeMillis();
-
@BeforeAll
public static void setup() {
LoginPage loginPage = new LoginPage();
@@ -93,8 +92,10 @@ public static void cleanup() {
}
@Test
+ @Order(1)
public void testGrpcFailedWorkflowInstance() {
try {
+ String workflowDefinitionName = "test_failed" + System.currentTimeMillis();
// create test project
HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test");
HttpResponse queryAllProjectListResponse = projectPage.queryAllProjectList(loginUser);
@@ -141,8 +142,10 @@ public void testGrpcFailedWorkflowInstance() {
}
@Test
+ @Order(10)
public void testGrpcSuccessWorkflowInstance() {
try {
+ String workflowDefinitionName = "test_success" + System.currentTimeMillis();
// create test project
HttpResponse createProjectResponse = projectPage.createProject(loginUser, "project-test");
HttpResponse queryAllProjectListResponse = projectPage.queryAllProjectList(loginUser);
diff --git a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionControllerTest.java b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionControllerTest.java
index 7217fed1a8fe..5b62bb76c166 100644
--- a/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionControllerTest.java
+++ b/dolphinscheduler-api/src/test/java/org/apache/dolphinscheduler/api/controller/WorkflowDefinitionControllerTest.java
@@ -37,8 +37,6 @@
import java.util.List;
import java.util.Map;
-import javax.servlet.http.HttpServletResponse;
-
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -47,7 +45,6 @@
import org.mockito.Mock;
import org.mockito.Mockito;
import org.mockito.junit.jupiter.MockitoExtension;
-import org.springframework.mock.web.MockHttpServletResponse;
/**
* process definition controller test
@@ -385,17 +382,6 @@ public void testQueryWorkflowDefinitionListPaging() {
Assertions.assertTrue(response != null && response.isSuccess());
}
- @Test
- public void testBatchExportWorkflowDefinitionByCodes() {
- String processDefinitionIds = "1,2";
- long projectCode = 1L;
- HttpServletResponse response = new MockHttpServletResponse();
- doNothing().when(this.processDefinitionService).batchExportWorkflowDefinitionByCodes(user, projectCode,
- processDefinitionIds, response);
- workflowDefinitionController.batchExportWorkflowDefinitionByCodes(user, projectCode, processDefinitionIds,
- response);
- }
-
@Test
public void testQueryWorkflowDefinitionVersions() {
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index 71c309e6d732..a0f0b9386ba8 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -169,7 +169,7 @@ master:
# master heartbeat interval
max-heartbeat-interval: 10s
server-load-protection:
- enabled: true
+ enabled: false
# Master max system cpu usage, when the master's system cpu usage is smaller then this value, master server can execute workflow.
max-system-cpu-usage-percentage-thresholds: 0.8
# Master max jvm cpu usage, when the master's jvm cpu usage is smaller then this value, master server can execute workflow.
@@ -205,7 +205,7 @@ worker:
# worker group name. If it is not set, the default value is default.
group: default
server-load-protection:
- enabled: true
+ enabled: false
# Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks.
max-system-cpu-usage-percentage-thresholds: 0.8
# Worker max jvm cpu usage, when the worker's jvm cpu usage is smaller then this value, worker server can be dispatched tasks.
diff --git a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
index 1380995f9032..357a4d9cfd59 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
+++ b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
@@ -81,6 +81,7 @@
+
From 8a121ec431f509fd8edbb5b337cc0d9e82d525fb Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Wed, 4 Feb 2026 11:51:46 +0800
Subject: [PATCH 8/9] remove import and export function
---
.../src/test/resources/workflow-json/test.json | 2 +-
.../src/main/resources/application.yaml | 4 ++--
.../src/main/resources/logback-spring.xml | 1 -
3 files changed, 3 insertions(+), 4 deletions(-)
diff --git a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
index 54c5ec735136..1a160d747422 100644
--- a/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
+++ b/dolphinscheduler-api-test/dolphinscheduler-api-test-case/src/test/resources/workflow-json/test.json
@@ -11,7 +11,7 @@
"name": "test1",
"taskParams": {
"localParams": [],
- "rawScript": "hello world",
+ "rawScript": "echo 'hello world'",
"resourceList": []
},
"taskPriority": "MEDIUM",
diff --git a/dolphinscheduler-standalone-server/src/main/resources/application.yaml b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
index a0f0b9386ba8..71c309e6d732 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/application.yaml
+++ b/dolphinscheduler-standalone-server/src/main/resources/application.yaml
@@ -169,7 +169,7 @@ master:
# master heartbeat interval
max-heartbeat-interval: 10s
server-load-protection:
- enabled: false
+ enabled: true
# Master max system cpu usage, when the master's system cpu usage is smaller then this value, master server can execute workflow.
max-system-cpu-usage-percentage-thresholds: 0.8
# Master max jvm cpu usage, when the master's jvm cpu usage is smaller then this value, master server can execute workflow.
@@ -205,7 +205,7 @@ worker:
# worker group name. If it is not set, the default value is default.
group: default
server-load-protection:
- enabled: false
+ enabled: true
# Worker max system cpu usage, when the worker's system cpu usage is smaller then this value, worker server can be dispatched tasks.
max-system-cpu-usage-percentage-thresholds: 0.8
# Worker max jvm cpu usage, when the worker's jvm cpu usage is smaller then this value, worker server can be dispatched tasks.
diff --git a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
index 357a4d9cfd59..1380995f9032 100644
--- a/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
+++ b/dolphinscheduler-standalone-server/src/main/resources/logback-spring.xml
@@ -81,7 +81,6 @@
-
From f29d80f8eabe0216aecdbc82a5c9e5f27f5f92cb Mon Sep 17 00:00:00 2001
From: SbloodyS <460888207@qq.com>
Date: Fri, 6 Feb 2026 09:39:03 +0800
Subject: [PATCH 9/9] remove import and export function
---
docs/docs/en/guide/upgrade/incompatible.md | 3 +++
docs/docs/zh/guide/upgrade/incompatible.md | 3 +++
2 files changed, 6 insertions(+)
diff --git a/docs/docs/en/guide/upgrade/incompatible.md b/docs/docs/en/guide/upgrade/incompatible.md
index 71e075ee2ccd..0f60fcb9986a 100644
--- a/docs/docs/en/guide/upgrade/incompatible.md
+++ b/docs/docs/en/guide/upgrade/incompatible.md
@@ -39,5 +39,8 @@ This document records the incompatible updates between each version. You need to
* Remove the default value of `python-gateway.auth-token` at `api-server/application.yaml`. ([#17801])(https://github.com/apache/dolphinscheduler/pull/17801)
* Refactor the task plugins which use ShellCommandExecutor ([#17790])(https://github.com/apache/dolphinscheduler/pull/17790)
* Remove the `Pytorch` from the `Task Plugin` ([#17808])(https://github.com/apache/dolphinscheduler/pull/17808), if you are still using this task type, please delete the data with `task_type = 'PYTORCH'` in `t_ds_task_definition` and `t_ds_task_definition_log` before upgrading.
+
+## 3.4.1
+
* Remove import and export of workflow definition. ([#17940])(https://github.com/apache/dolphinscheduler/issues/17940)
diff --git a/docs/docs/zh/guide/upgrade/incompatible.md b/docs/docs/zh/guide/upgrade/incompatible.md
index 3e1acae5b222..2f0c4c044ff8 100644
--- a/docs/docs/zh/guide/upgrade/incompatible.md
+++ b/docs/docs/zh/guide/upgrade/incompatible.md
@@ -43,5 +43,8 @@
* 移除 `api-server/application.yaml` 中 `python-gateway.auth-token` 的默认值。 ([#17801])(https://github.com/apache/dolphinscheduler/pull/17801)
* 重构使用 ShellCommandExecutor 的任务插件 ([#17790])(https://github.com/apache/dolphinscheduler/pull/17790)
* 从 `任务插件` 中移除 `Pytorch` 类型 ([#17808])(https://github.com/apache/dolphinscheduler/pull/17808),如果您仍在使用该任务类型,请在升级前删除 `t_ds_task_definition` 和 `t_ds_task_definition_log` 中 `task_type = 'PYTORCH'` 的数据。
+
+## 3.4.1
+
* 移除导入导出工作流([#17940])(https://github.com/apache/dolphinscheduler/issues/17940)