Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
25 commits
Select commit Hold shift + click to select a range
3fb7634
close #17748
Feb 27, 2026
e649f3b
fix: prevent circular dependency and wrong startNodes in backfill dep…
Feb 27, 2026
b4e010d
refactor: replace ThreadLocal with explicit parameter for cycle detec…
Feb 27, 2026
ed24efe
[Chore] Apply spotless formatting for backfill workflow executor files
Mar 23, 2026
29905e8
fix review
Mar 23, 2026
1841eaf
fix(api): isolate visited codes per parallel backfill chunk
Mar 25, 2026
cbe8a60
fix(api): remove redundant dependent-trigger testing seam
Mar 25, 2026
e797b01
fix(api): propagate ZonedDateTime chunk into dependent backfill
Mar 26, 2026
47e3d17
test(api): remove unit tests for private dependent backfill
Mar 26, 2026
72a2642
fix(api): align dependent routing and prevent double-splitting
Mar 26, 2026
d2a8c96
test(api): add dependent backfill traversal branches
Mar 26, 2026
c7ba4b9
Update dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler…
det101 Mar 26, 2026
b477d6b
fix(api): aggregate dependent nodes per downstream workflow
Mar 26, 2026
6610bc2
chore(api): initialize visitedCodes at execute entry
Mar 26, 2026
eca546f
Apply suggestion from @Copilot
det101 Mar 26, 2026
f320007
Update dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler…
det101 Mar 26, 2026
7c9acee
Update dolphinscheduler-api/src/main/java/org/apache/dolphinscheduler…
det101 Mar 26, 2026
8b026ac
fix: handle null/zero expectedParallelismNumber to prevent Arithmetic…
Mar 26, 2026
5579fe6
fix: cap parallelism to listDate size and route dependent backfill th…
Mar 26, 2026
af022b9
refactor: revert parallelism guard to original if/else style and shar…
Mar 26, 2026
42086c1
docs: clarify force-SERIAL rationale and recursion depth assumption
Mar 26, 2026
2114213
Refactor: translate all code comments to English and unify test logic…
Mar 26, 2026
8acc714
Merge branch 'dev' into DSIP-95
SbloodyS Mar 27, 2026
ddbbdab
fix spotless
Mar 27, 2026
1363d8b
fix spotless
Mar 27, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,18 @@
package org.apache.dolphinscheduler.api.executor.workflow;

import org.apache.dolphinscheduler.api.exceptions.ServiceException;
import org.apache.dolphinscheduler.api.service.WorkflowLineageService;
import org.apache.dolphinscheduler.api.validator.workflow.BackfillWorkflowDTO;
import org.apache.dolphinscheduler.common.enums.ComplementDependentMode;
import org.apache.dolphinscheduler.common.enums.ExecutionOrder;
import org.apache.dolphinscheduler.common.enums.ReleaseState;
import org.apache.dolphinscheduler.common.enums.RunMode;
import org.apache.dolphinscheduler.common.model.Server;
import org.apache.dolphinscheduler.common.utils.DateUtils;
import org.apache.dolphinscheduler.dao.entity.DependentWorkflowDefinition;
import org.apache.dolphinscheduler.dao.entity.WorkflowDefinition;
import org.apache.dolphinscheduler.dao.repository.CommandDao;
import org.apache.dolphinscheduler.dao.repository.WorkflowDefinitionDao;
import org.apache.dolphinscheduler.extract.base.client.Clients;
import org.apache.dolphinscheduler.extract.master.IWorkflowControlClient;
import org.apache.dolphinscheduler.extract.master.transportor.workflow.WorkflowBackfillTriggerRequest;
Expand All @@ -37,7 +41,10 @@
import java.time.ZonedDateTime;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;

import lombok.extern.slf4j.Slf4j;
Expand All @@ -57,20 +64,32 @@ public class BackfillWorkflowExecutorDelegate implements IExecutorDelegate<Backf
@Autowired
private ProcessService processService;

@Autowired
private WorkflowLineageService workflowLineageService;

@Autowired
private WorkflowDefinitionDao workflowDefinitionDao;

@Autowired
private RegistryClient registryClient;

@Override
public List<Integer> execute(final BackfillWorkflowDTO backfillWorkflowDTO) {
return executeWithVisitedCodes(backfillWorkflowDTO, new HashSet<>());
}

List<Integer> executeWithVisitedCodes(final BackfillWorkflowDTO backfillWorkflowDTO,
final Set<Long> visitedCodes) {
// todo: directly call the master api to do backfill
if (backfillWorkflowDTO.getBackfillParams().getRunMode() == RunMode.RUN_MODE_SERIAL) {
return doSerialBackfillWorkflow(backfillWorkflowDTO);
return doSerialBackfillWorkflow(backfillWorkflowDTO, visitedCodes);
} else {
return doParallelBackfillWorkflow(backfillWorkflowDTO);
return doParallelBackfillWorkflow(backfillWorkflowDTO, visitedCodes);
}
}

private List<Integer> doSerialBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) {
private List<Integer> doSerialBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final Set<Long> visitedCodes) {
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
final List<ZonedDateTime> backfillTimeList = backfillParams.getBackfillDateList();
if (backfillParams.getExecutionOrder() == ExecutionOrder.DESC_ORDER) {
Expand All @@ -79,13 +98,12 @@ private List<Integer> doSerialBackfillWorkflow(final BackfillWorkflowDTO backfil
Collections.sort(backfillTimeList);
}

final Integer workflowInstanceId = doBackfillWorkflow(
backfillWorkflowDTO,
backfillTimeList.stream().map(DateUtils::dateToString).collect(Collectors.toList()));
final Integer workflowInstanceId = doBackfillWorkflow(backfillWorkflowDTO, backfillTimeList, visitedCodes);
return Lists.newArrayList(workflowInstanceId);
}

private List<Integer> doParallelBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO) {
private List<Integer> doParallelBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final Set<Long> visitedCodes) {
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
Integer expectedParallelismNumber = backfillParams.getExpectedParallelismNumber();

Expand All @@ -100,8 +118,7 @@ private List<Integer> doParallelBackfillWorkflow(final BackfillWorkflowDTO backf
final List<Integer> workflowInstanceIdList = Lists.newArrayList();
for (List<ZonedDateTime> stringDate : splitDateTime(listDate, expectedParallelismNumber)) {
final Integer workflowInstanceId = doBackfillWorkflow(
backfillWorkflowDTO,
stringDate.stream().map(DateUtils::dateToString).collect(Collectors.toList()));
backfillWorkflowDTO, stringDate, visitedCodes);
workflowInstanceIdList.add(workflowInstanceId);
}
return workflowInstanceIdList;
Expand Down Expand Up @@ -132,12 +149,16 @@ private List<List<ZonedDateTime>> splitDateTime(List<ZonedDateTime> dateTimeList
}

private Integer doBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final List<String> backfillTimeList) {
final List<ZonedDateTime> backfillDateTimes,
final Set<Long> visitedCodes) {
final Server masterServer = registryClient.getRandomServer(RegistryNodeType.MASTER).orElse(null);
if (masterServer == null) {
throw new ServiceException("no master server available");
}

final List<String> backfillTimeList =
backfillDateTimes.stream().map(DateUtils::dateToString).collect(Collectors.toList());

final WorkflowDefinition workflowDefinition = backfillWorkflowDTO.getWorkflowDefinition();
final WorkflowBackfillTriggerRequest backfillTriggerRequest = WorkflowBackfillTriggerRequest.builder()
.userId(backfillWorkflowDTO.getLoginUser().getId())
Expand All @@ -157,22 +178,144 @@ private Integer doBackfillWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO
.dryRun(backfillWorkflowDTO.getDryRun())
.build();

final WorkflowBackfillTriggerResponse backfillTriggerResponse = Clients
.withService(IWorkflowControlClient.class)
.withHost(masterServer.getHost() + ":" + masterServer.getPort())
.backfillTriggerWorkflow(backfillTriggerRequest);
final WorkflowBackfillTriggerResponse backfillTriggerResponse =
triggerBackfillWorkflow(backfillTriggerRequest, masterServer);
if (!backfillTriggerResponse.isSuccess()) {
throw new ServiceException("Backfill workflow failed: " + backfillTriggerResponse.getMessage());
}
final BackfillWorkflowDTO.BackfillParamsDTO backfillParams = backfillWorkflowDTO.getBackfillParams();
if (backfillParams.getBackfillDependentMode() == ComplementDependentMode.ALL_DEPENDENT) {
doBackfillDependentWorkflow(backfillWorkflowDTO, backfillTimeList);
final Set<Long> effectiveVisitedCodes = visitedCodes == null ? new HashSet<>() : visitedCodes;
effectiveVisitedCodes.add(backfillWorkflowDTO.getWorkflowDefinition().getCode());
doBackfillDependentWorkflow(backfillWorkflowDTO, backfillDateTimes, effectiveVisitedCodes);
}
return backfillTriggerResponse.getWorkflowInstanceId();
}

protected WorkflowBackfillTriggerResponse triggerBackfillWorkflow(final WorkflowBackfillTriggerRequest request,
final Server masterServer) {
return Clients
.withService(IWorkflowControlClient.class)
.withHost(masterServer.getHost() + ":" + masterServer.getPort())
.backfillTriggerWorkflow(request);
}

private void doBackfillDependentWorkflow(final BackfillWorkflowDTO backfillWorkflowDTO,
final List<String> backfillTimeList) {
// todo:
final List<ZonedDateTime> backfillDateTimes,
final Set<Long> visitedCodes) {
// 1) Query downstream dependent workflows for the current workflow
final WorkflowDefinition upstreamWorkflow = backfillWorkflowDTO.getWorkflowDefinition();
final long upstreamWorkflowCode = upstreamWorkflow.getCode();

List<DependentWorkflowDefinition> downstreamDefinitions =
workflowLineageService.queryDownstreamDependentWorkflowDefinitions(upstreamWorkflowCode);

if (downstreamDefinitions == null || downstreamDefinitions.isEmpty()) {
log.info("No downstream dependent workflows found for workflow code {}", upstreamWorkflowCode);
return;
}
// downstreamDefinitions may contain multiple entries for the same downstream workflow code
// (different dependent task lineage). We should only traverse each downstream workflow once
// (visitedCodes check), but trigger all dependent nodes within that downstream workflow by
// aggregating distinct taskDefinitionCodes into startNodes.
final Map<Long, List<DependentWorkflowDefinition>> downstreamDefinitionsByCode =
downstreamDefinitions.stream()
.collect(Collectors.groupingBy(DependentWorkflowDefinition::getWorkflowDefinitionCode));
final Set<Long> downstreamCodes = downstreamDefinitionsByCode.keySet();
final List<WorkflowDefinition> downstreamWorkflowList = workflowDefinitionDao.queryByCodes(downstreamCodes);
// Each workflow code maps to a single WorkflowDefinition (code is unique in t_ds_workflow_definition).
// We still group by code to simplify lookup and keep the code robust if this ever changes.
final Map<Long, List<WorkflowDefinition>> downstreamWorkflowMapByCode = downstreamWorkflowList.stream()
.collect(Collectors.groupingBy(WorkflowDefinition::getCode));

// 2) Reuse upstream business dates for downstream backfill (same instants/zones as the chunk passed to
// doBackfillWorkflow; avoids List<String> -> system-default parse -> dateToString drift)
final List<ZonedDateTime> upstreamBackfillDates = new ArrayList<>(backfillDateTimes);

// 3) Iterate downstream workflows and build/trigger corresponding BackfillWorkflowDTO
for (Map.Entry<Long, List<DependentWorkflowDefinition>> entry : downstreamDefinitionsByCode.entrySet()) {
long downstreamCode = entry.getKey();
// Prevent self-dependency and circular dependency chains.
// We only traverse each downstream workflow once.
if (visitedCodes.contains(downstreamCode)) {
log.warn("Skip already visited dependent workflow {}", downstreamCode);
continue;
}

// Simplification: Downstream backfill is always full, startNodes=null, workerGroup uses
// workflowDefinition's own config
// Only grouping and deduplication are needed, all aggregation logic is omitted

WorkflowDefinition downstreamWorkflow = null;
List<WorkflowDefinition> workflowCandidates = downstreamWorkflowMapByCode.get(downstreamCode);
if (workflowCandidates != null && !workflowCandidates.isEmpty()) {
downstreamWorkflow = workflowCandidates.get(0); // code is unique, just take the first one
}
if (downstreamWorkflow == null) {
log.warn("Skip dependent workflow {}, workflow definition not found", downstreamCode);
continue;
}
if (downstreamWorkflow.getReleaseState() != ReleaseState.ONLINE) {
log.warn("Skip dependent workflow {}, release state is not ONLINE", downstreamCode);
continue;
}

// Currently, reuse the same business date list as upstream for downstream backfill;
// later we can refine the dates based on dependency cycle configuration in dependentWorkflowDefinition
// (taskParams).
BackfillWorkflowDTO.BackfillParamsDTO originalParams = backfillWorkflowDTO.getBackfillParams();
boolean allLevelDependent = originalParams.isAllLevelDependent();
ComplementDependentMode downstreamDependentMode =
allLevelDependent ? originalParams.getBackfillDependentMode() : ComplementDependentMode.OFF_MODE;

BackfillWorkflowDTO.BackfillParamsDTO dependentParams = BackfillWorkflowDTO.BackfillParamsDTO.builder()
// If upstream runs in PARALLEL mode, force downstream to SERIAL to avoid
// re-chunking the already sliced date list; otherwise keep the original runMode.
.runMode(originalParams.getRunMode() == RunMode.RUN_MODE_PARALLEL ? RunMode.RUN_MODE_SERIAL
: originalParams.getRunMode())
.backfillDateList(upstreamBackfillDates)
.expectedParallelismNumber(originalParams.getExpectedParallelismNumber())
// Control whether downstream will continue triggering its own dependencies based on
// allLevelDependent flag
.backfillDependentMode(downstreamDependentMode)
.allLevelDependent(allLevelDependent)
.executionOrder(originalParams.getExecutionOrder())
.build();

// Simplified design notes:
// 1. Downstream backfill is always full, startNodes=null, no more aggregation of dependent nodes
// 2. workerGroup is directly taken from the downstream workflowDefinition's own config, if null then use
// system default workerGroup
// 3. Only grouping deduplication and visitedCodes check, all complex aggregation logic is omitted
// This implementation is the simplest and most controllable, suitable for full backfill and workerGroup
// based on itself
BackfillWorkflowDTO dependentBackfillDTO = BackfillWorkflowDTO.builder()
.loginUser(backfillWorkflowDTO.getLoginUser())
.workflowDefinition(downstreamWorkflow)
.startNodes(null) // Full backfill, simplified design
.failureStrategy(backfillWorkflowDTO.getFailureStrategy())
.taskDependType(backfillWorkflowDTO.getTaskDependType())
.execType(backfillWorkflowDTO.getExecType())
.warningType(backfillWorkflowDTO.getWarningType())
.warningGroupId(downstreamWorkflow.getWarningGroupId())
.runMode(dependentParams.getRunMode())
.workflowInstancePriority(backfillWorkflowDTO.getWorkflowInstancePriority())
.workerGroup(backfillWorkflowDTO.getWorkerGroup())
.tenantCode(backfillWorkflowDTO.getTenantCode())
.environmentCode(backfillWorkflowDTO.getEnvironmentCode())
.startParamList(backfillWorkflowDTO.getStartParamList())
.dryRun(backfillWorkflowDTO.getDryRun())
.backfillParams(dependentParams)
.build();

log.info("Trigger dependent workflow {} for upstream workflow {} with backfill dates {}",
downstreamCode, upstreamWorkflowCode,
backfillDateTimes.stream().map(DateUtils::dateToString).collect(Collectors.toList()));

// Mark as visited to prevent infinite recursion. Actual dependency chains are usually short, recursion is
// safe.
visitedCodes.add(downstreamCode);
executeWithVisitedCodes(dependentBackfillDTO, visitedCodes);
}
}
}
Loading