Skip to content

Commit 96b36c4

Browse files
committed
add a new status SKIPPED for skipped jobs and flows
fix merge conflicts
1 parent 45ad13e commit 96b36c4

File tree

19 files changed

+246
-29
lines changed

19 files changed

+246
-29
lines changed

gobblin-cluster/src/main/java/org/apache/gobblin/cluster/HelixRetriggeringJobCallable.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -279,6 +279,7 @@ private void runJobExecutionLauncher() throws JobException {
279279

280280
try {
281281
if (planningJobIdFromStore.isPresent() && !canRun(planningJobIdFromStore.get(), planningJobHelixManager)) {
282+
// todo it should emit SKIPPED_JOB event that sets the job status SKIPPED rather than CANCELLED
282283
TimingEvent timer = new TimingEvent(eventSubmitter, TimingEvent.JOB_SKIPPED_TIME);
283284
HashMap<String, String> metadata = new HashMap<>(Tag.toMap(Tag.tagValuesToString(
284285
HelixUtils.initBaseEventTags(jobProps, Lists.newArrayList()))));

gobblin-metrics-libs/gobblin-metrics-base/src/main/java/org/apache/gobblin/metrics/event/TimingEvent.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ public static class LauncherTimings {
4343
public static final String JOB_PENDING_RESUME = "JobPendingResume";
4444
public static final String JOB_ORCHESTRATED = "JobOrchestrated";
4545
public static final String JOB_PREPARE = "JobPrepareTimer";
46+
public static final String JOB_SKIPPED = "JobSkipped";
4647
public static final String JOB_START = "JobStartTimer";
4748
public static final String JOB_RUN = "JobRunTimer";
4849
public static final String JOB_COMMIT = "JobCommitTimer";

gobblin-modules/gobblin-kafka-09/src/test/java/org/apache/gobblin/runtime/KafkaAvroJobStatusMonitorTest.java

Lines changed: 40 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -308,7 +308,7 @@ public void testProcessMessageForSkippedFlow() throws IOException, ReflectiveOpe
308308
ImmutableList.of(
309309
createFlowCompiledEvent(),
310310
createJobOrchestratedEvent(1, 2),
311-
createJobSkippedEvent()
311+
createJobSkippedTimeEvent()
312312
).forEach(event -> {
313313
context.submitEvent(event);
314314
kafkaReporter.report();
@@ -836,6 +836,40 @@ public void testObservabilityEventFlowFailed() throws IOException, ReflectiveOpe
836836
jobStatusMonitor.shutDown();
837837
}
838838

839+
@Test// (dependsOnMethods = "testObservabilityEventFlowFailed")
840+
public void testProcessMessageForSkippedEvent() throws IOException, ReflectiveOperationException {
841+
DagManagementStateStore dagManagementStateStore = mock(DagManagementStateStore.class);
842+
KafkaEventReporter kafkaReporter = builder.build("localhost:0000", "topic8");
843+
844+
//Submit GobblinTrackingEvents to Kafka
845+
ImmutableList.of(
846+
createJobSkippedEvent()
847+
).forEach(event -> {
848+
context.submitEvent(event);
849+
kafkaReporter.report();
850+
});
851+
852+
try {
853+
Thread.sleep(1000);
854+
} catch(InterruptedException ex) {
855+
Thread.currentThread().interrupt();
856+
}
857+
858+
MockKafkaAvroJobStatusMonitor jobStatusMonitor = createMockKafkaAvroJobStatusMonitor(new AtomicBoolean(false),
859+
ConfigFactory.empty(), new NoopGaaSJobObservabilityEventProducer(), dagManagementStateStore);
860+
jobStatusMonitor.buildMetricsContextAndMetrics();
861+
Iterator<DecodeableKafkaRecord<byte[], byte[]>> recordIterator = Iterators.transform(
862+
this.kafkaTestHelper.getIteratorForTopic(TOPIC),
863+
this::convertMessageAndMetadataToDecodableKafkaRecord);
864+
865+
State state = getNextJobStatusState(jobStatusMonitor, recordIterator, this.jobGroup, this.jobName);
866+
Assert.assertEquals(state.getProp(JobStatusRetriever.EVENT_NAME_FIELD), ExecutionStatus.SKIPPED.name());
867+
Mockito.verify(dagManagementStateStore, Mockito.times(1)).addJobDagAction(
868+
any(), any(), anyLong(), any(), eq(DagActionStore.DagActionType.REEVALUATE));
869+
870+
jobStatusMonitor.shutDown();
871+
}
872+
839873
private State getNextJobStatusState(MockKafkaAvroJobStatusMonitor jobStatusMonitor, Iterator<DecodeableKafkaRecord<byte[], byte[]>> recordIterator,
840874
String jobGroup, String jobName) throws IOException {
841875
jobStatusMonitor.processMessage(recordIterator.next());
@@ -871,11 +905,15 @@ private GobblinTrackingEvent createJobOrchestratedEvent(int currentAttempt, int
871905
return createGTE(TimingEvent.LauncherTimings.JOB_ORCHESTRATED, metadata);
872906
}
873907

908+
private GobblinTrackingEvent createJobSkippedEvent() {
909+
return createGTE(TimingEvent.LauncherTimings.JOB_SKIPPED, Maps.newHashMap());
910+
}
911+
874912
private GobblinTrackingEvent createJobStartEvent() {
875913
return createGTE(TimingEvent.LauncherTimings.JOB_START, Maps.newHashMap());
876914
}
877915

878-
private GobblinTrackingEvent createJobSkippedEvent() {
916+
private GobblinTrackingEvent createJobSkippedTimeEvent() {
879917
return createGTE(TimingEvent.JOB_SKIPPED_TIME, Maps.newHashMap());
880918
}
881919

gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/pegasus/org/apache/gobblin/service/ExecutionStatus.pdl

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -49,4 +49,9 @@ enum ExecutionStatus {
4949
* Flow cancelled.
5050
*/
5151
CANCELLED
52+
53+
/**
54+
* Flow or job is skipped
55+
*/
56+
SKIPPED
5257
}

gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowexecutions.snapshot.json

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@
1313
"name" : "ExecutionStatus",
1414
"namespace" : "org.apache.gobblin.service",
1515
"doc" : "Execution status for a flow or job",
16-
"symbols" : [ "COMPILED", "PENDING", "PENDING_RETRY", "PENDING_RESUME", "ORCHESTRATED", "RUNNING", "COMPLETE", "FAILED", "CANCELLED" ],
16+
"symbols" : [ "COMPILED", "PENDING", "PENDING_RETRY", "PENDING_RESUME", "ORCHESTRATED", "RUNNING", "COMPLETE", "FAILED", "CANCELLED", "SKIPPED" ],
1717
"symbolDocs" : {
1818
"CANCELLED" : "Flow cancelled.",
1919
"COMPILED" : "Flow compiled to jobs.",
@@ -23,7 +23,8 @@
2323
"PENDING" : "Flow or job is in pending state.",
2424
"PENDING_RESUME" : "Flow or job is currently resuming.",
2525
"PENDING_RETRY" : "Flow or job is pending retry.",
26-
"RUNNING" : "Flow or job is currently executing"
26+
"RUNNING" : "Flow or job is currently executing.",
27+
"SKIPPED" : "Flow or job is skipped."
2728
}
2829
}, {
2930
"type" : "record",

gobblin-restli/gobblin-flow-config-service/gobblin-flow-config-service-api/src/main/snapshot/org.apache.gobblin.service.flowstatuses.snapshot.json

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,7 @@
1313
"name" : "ExecutionStatus",
1414
"namespace" : "org.apache.gobblin.service",
1515
"doc" : "Execution status for a flow or job",
16-
"symbols" : [ "COMPILED", "PENDING", "PENDING_RETRY", "PENDING_RESUME", "ORCHESTRATED", "RUNNING", "COMPLETE", "FAILED", "CANCELLED" ],
16+
"symbols" : [ "COMPILED", "PENDING", "PENDING_RETRY", "PENDING_RESUME", "ORCHESTRATED", "RUNNING", "COMPLETE", "FAILED", "CANCELLED", "SKIPPED" ],
1717
"symbolDocs" : {
1818
"CANCELLED" : "Flow cancelled.",
1919
"COMPILED" : "Flow compiled to jobs.",
@@ -23,7 +23,8 @@
2323
"PENDING" : "Flow or job is in pending state.",
2424
"PENDING_RESUME" : "Flow or job is currently resuming.",
2525
"PENDING_RETRY" : "Flow or job is pending retry.",
26-
"RUNNING" : "Flow or job is currently executing"
26+
"RUNNING" : "Flow or job is currently executing.",
27+
"SKIPPED" : "Flow or job is skipped."
2728
}
2829
}, {
2930
"type" : "record",

gobblin-runtime/src/main/java/org/apache/gobblin/service/monitoring/FlowStatusGenerator.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -41,7 +41,7 @@
4141
@Slf4j
4242
public class FlowStatusGenerator {
4343
public static final List<String> FINISHED_STATUSES = Lists.newArrayList(ExecutionStatus.FAILED.name(),
44-
ExecutionStatus.COMPLETE.name(), ExecutionStatus.CANCELLED.name());
44+
ExecutionStatus.COMPLETE.name(), ExecutionStatus.CANCELLED.name(), ExecutionStatus.SKIPPED.name());
4545
public static final int MAX_LOOKBACK = 100;
4646

4747
private final JobStatusRetriever jobStatusRetriever;

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -99,7 +99,6 @@ public interface DagManagementStateStore {
9999
* {@link DagManagementStateStore#addDag}. This call is just an additional identifier which may be used
100100
* for DagNode level operations. In the future, it may be merged with checkpointDag.
101101
* @param dagNode dag node to be added
102-
* @param dagId dag id of the dag this dag node belongs to
103102
*/
104103
void updateDagNode(Dag.DagNode<JobExecutionPlan> dagNode) throws IOException;
105104

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagUtils.java

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -164,7 +164,8 @@ public static Set<DagNode<JobExecutionPlan>> getNext(Dag<JobExecutionPlan> dag)
164164
DagNode<JobExecutionPlan> node = nodesToExpand.poll();
165165
ExecutionStatus executionStatus = getExecutionStatus(node);
166166
boolean addFlag = true;
167-
if (executionStatus == PENDING || executionStatus == PENDING_RETRY || executionStatus == PENDING_RESUME) {
167+
if (executionStatus == PENDING || executionStatus == PENDING_RETRY || executionStatus == PENDING_RESUME ||
168+
executionStatus == SKIPPED) {
168169
//Add a node to be executed next, only if all of its parent nodes are COMPLETE.
169170
List<DagNode<JobExecutionPlan>> parentNodes = dag.getParents(node);
170171
for (DagNode<JobExecutionPlan> parentNode : parentNodes) {

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MySqlDagManagementStateStore.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@
4343
import org.apache.gobblin.service.modules.flowgraph.Dag;
4444
import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
4545
import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
46+
import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
4647
import org.apache.gobblin.service.monitoring.JobStatus;
4748
import org.apache.gobblin.service.monitoring.JobStatusRetriever;
4849
import org.apache.gobblin.util.ConfigUtils;

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProcUtils.java

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,6 @@
3737

3838
import org.apache.gobblin.configuration.ConfigurationKeys;
3939
import org.apache.gobblin.metrics.GobblinTrackingEvent;
40-
import org.apache.gobblin.metrics.event.EventSubmitter;
4140
import org.apache.gobblin.metrics.event.TimingEvent;
4241
import org.apache.gobblin.runtime.api.JobSpec;
4342
import org.apache.gobblin.runtime.api.Spec;
@@ -183,6 +182,19 @@ public static void cancelDagNode(Dag.DagNode<JobExecutionPlan> dagNodeToCancel,
183182
}
184183
}
185184

185+
/**
186+
* Emits JOB_SKIPPED GTE for each of the dependent job.
187+
*/
188+
public static void sendSkippedEventForDependentJobs(Dag<JobExecutionPlan> dag, Dag.DagNode<JobExecutionPlan> node,
189+
DagManagementStateStore dagManagementStateStore) throws IOException {
190+
for (Dag.DagNode<JobExecutionPlan> child : dag.getChildren(node)) {
191+
child.getValue().setExecutionStatus(SKIPPED);
192+
dagManagementStateStore.updateDagNode(child);
193+
Map<String, String> jobMetadata = TimingEventUtils.getJobMetadata(Maps.newHashMap(), child.getValue());
194+
DagProc.eventSubmitter.getTimingEvent(TimingEvent.LauncherTimings.JOB_SKIPPED).stop(jobMetadata);
195+
}
196+
}
197+
186198
public static void cancelDag(Dag<JobExecutionPlan> dag, DagManagementStateStore dagManagementStateStore) throws IOException {
187199
List<Dag.DagNode<JobExecutionPlan>> dagNodesToCancel = dag.getNodes();
188200
log.info("Found {} DagNodes to cancel (DagId {}).", dagNodesToCancel.size(), DagUtils.generateDagId(dag));
@@ -202,7 +214,7 @@ private static void sendJobCancellationEvent(Dag.DagNode<JobExecutionPlan> dagNo
202214
* Sets {@link Dag#flowEvent} and emits a {@link GobblinTrackingEvent} of the provided
203215
* flow event type.
204216
*/
205-
public static void setAndEmitFlowEvent(EventSubmitter eventSubmitter, Dag<JobExecutionPlan> dag, String flowEvent) {
217+
public static void setAndEmitFlowEvent(Dag<JobExecutionPlan> dag, String flowEvent) {
206218
if (!dag.isEmpty()) {
207219
// Every dag node will contain the same flow metadata
208220
Config config = DagUtils.getDagJobConfig(dag);
@@ -213,7 +225,7 @@ public static void setAndEmitFlowEvent(EventSubmitter eventSubmitter, Dag<JobExe
213225
flowMetadata.put(TimingEvent.METADATA_MESSAGE, dag.getMessage());
214226
}
215227

216-
eventSubmitter.getTimingEvent(flowEvent).stop(flowMetadata);
228+
DagProc.eventSubmitter.getTimingEvent(flowEvent).stop(flowMetadata);
217229
}
218230
}
219231

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProc.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -64,7 +64,7 @@ protected void act(DagManagementStateStore dagManagementStateStore, Optional<Dag
6464
}
6565

6666
dag.get().setMessage("Flow killed by request");
67-
DagProcUtils.setAndEmitFlowEvent(eventSubmitter, dag.get(), TimingEvent.FlowTimings.FLOW_CANCELLED);
67+
DagProcUtils.setAndEmitFlowEvent(dag.get(), TimingEvent.FlowTimings.FLOW_CANCELLED);
6868

6969
if (this.shouldKillSpecificJob) {
7070
Optional<Dag.DagNode<JobExecutionPlan>> dagNodeToCancel = dagManagementStateStore.getDagNodeWithJobStatus(this.dagNodeId).getLeft();

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -84,7 +84,7 @@ protected void act(DagManagementStateStore dagManagementStateStore, Optional<Dag
8484
dagProcEngineMetrics.markDagActionsAct(getDagActionType(), false);
8585
} else {
8686
DagProcUtils.submitNextNodes(dagManagementStateStore, dag.get(), getDagId());
87-
DagProcUtils.setAndEmitFlowEvent(eventSubmitter, dag.get(), TimingEvent.FlowTimings.FLOW_RUNNING);
87+
DagProcUtils.setAndEmitFlowEvent(dag.get(), TimingEvent.FlowTimings.FLOW_RUNNING);
8888
dagManagementStateStore.getDagManagerMetrics().conditionallyMarkFlowAsState(DagUtils.getFlowId(dag.get()),
8989
Dag.FlowState.RUNNING);
9090
DagProcUtils.sendEnforceFlowFinishDeadlineDagAction(dagManagementStateStore, getDagTask().getDagAction());

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -117,7 +117,7 @@ protected void act(DagManagementStateStore dagManagementStateStore, Pair<Optiona
117117
} else if (DagProcUtils.isDagFinished(dag)) {
118118
String flowEvent = DagProcUtils.calcFlowStatus(dag);
119119
dag.setFlowEvent(flowEvent);
120-
DagProcUtils.setAndEmitFlowEvent(eventSubmitter, dag, flowEvent);
120+
DagProcUtils.setAndEmitFlowEvent(dag, flowEvent);
121121
if (flowEvent.equals(TimingEvent.FlowTimings.FLOW_SUCCEEDED)) {
122122
// todo - verify if work from PR#3641 is required
123123
dagManagementStateStore.deleteDag(getDagId());
@@ -159,9 +159,12 @@ private void onJobFinish(DagManagementStateStore dagManagementStateStore, Dag.Da
159159
dag.setMessage("Flow failed because job " + jobName + " failed");
160160
dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_FAILED);
161161
dagManagementStateStore.getDagManagerMetrics().incrementExecutorFailed(dagNode);
162+
DagProcUtils.sendSkippedEventForDependentJobs(dag, dagNode, dagManagementStateStore);
162163
break;
163164
case CANCELLED:
165+
case SKIPPED:
164166
dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_CANCELLED);
167+
DagProcUtils.sendSkippedEventForDependentJobs(dag, dagNode, dagManagementStateStore);
165168
break;
166169
case COMPLETE:
167170
dagManagementStateStore.getDagManagerMetrics().incrementExecutorSuccess(dagNode);

gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ResumeDagProc.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -71,7 +71,7 @@ protected void act(DagManagementStateStore dagManagementStateStore, Optional<Dag
7171
long flowResumeTime = System.currentTimeMillis();
7272

7373
// Set the flow and its failed or cancelled nodes to PENDING_RESUME so that the flow will be resumed from the point before it failed
74-
DagProcUtils.setAndEmitFlowEvent(eventSubmitter, failedDag.get(), TimingEvent.FlowTimings.FLOW_PENDING_RESUME);
74+
DagProcUtils.setAndEmitFlowEvent(failedDag.get(), TimingEvent.FlowTimings.FLOW_PENDING_RESUME);
7575

7676
for (Dag.DagNode<JobExecutionPlan> node : failedDag.get().getNodes()) {
7777
ExecutionStatus executionStatus = node.getValue().getExecutionStatus();

gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaAvroJobStatusMonitor.java

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -141,6 +141,9 @@ public org.apache.gobblin.configuration.State parseJobStatus(GobblinTrackingEven
141141
case TimingEvent.LauncherTimings.JOB_PENDING:
142142
properties.put(JobStatusRetriever.EVENT_NAME_FIELD, ExecutionStatus.PENDING.name());
143143
break;
144+
case TimingEvent.LauncherTimings.JOB_SKIPPED:
145+
properties.put(JobStatusRetriever.EVENT_NAME_FIELD, ExecutionStatus.SKIPPED.name());
146+
break;
144147
case TimingEvent.FlowTimings.FLOW_PENDING_RESUME:
145148
case TimingEvent.LauncherTimings.JOB_PENDING_RESUME:
146149
properties.put(JobStatusRetriever.EVENT_NAME_FIELD, ExecutionStatus.PENDING_RESUME.name());

gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -115,8 +115,8 @@ public abstract class KafkaJobStatusMonitor extends HighLevelConsumer<byte[], by
115115
KAFKA_AUTO_OFFSET_RESET_KEY, KAFKA_AUTO_OFFSET_RESET_SMALLEST));
116116

117117
private static final List<ExecutionStatus> ORDERED_EXECUTION_STATUSES = ImmutableList
118-
.of(ExecutionStatus.COMPILED, ExecutionStatus.PENDING, ExecutionStatus.PENDING_RESUME, ExecutionStatus.PENDING_RETRY,
119-
ExecutionStatus.ORCHESTRATED, ExecutionStatus.RUNNING, ExecutionStatus.COMPLETE,
118+
.of(ExecutionStatus.COMPILED, ExecutionStatus.PENDING, ExecutionStatus.SKIPPED, ExecutionStatus.PENDING_RESUME,
119+
ExecutionStatus.PENDING_RETRY, ExecutionStatus.ORCHESTRATED, ExecutionStatus.RUNNING, ExecutionStatus.COMPLETE,
120120
ExecutionStatus.FAILED, ExecutionStatus.CANCELLED);
121121

122122
private final JobIssueEventHandler jobIssueEventHandler;

gobblin-service/src/test/java/org/apache/gobblin/service/modules/orchestration/proc/KillDagProcTest.java

Lines changed: 18 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,9 +19,9 @@
1919

2020
import java.io.IOException;
2121
import java.net.URISyntaxException;
22+
import java.util.HashSet;
2223
import java.util.List;
2324
import java.util.Optional;
24-
import java.util.Properties;
2525
import java.util.concurrent.ExecutionException;
2626
import java.util.stream.Collectors;
2727

@@ -186,7 +186,8 @@ public void killDagNode() throws IOException, URISyntaxException, InterruptedExc
186186
message("Test message").eventName(ExecutionStatus.COMPLETE.name()).startTime(flowExecutionId).shouldRetry(false).orchestratedTime(flowExecutionId).build();
187187

188188
doReturn(Optional.of(dag)).when(dagManagementStateStore).getDag(any());
189-
doReturn(new ImmutablePair<>(Optional.of(dag.getStartNodes().get(0)), Optional.of(jobStatus))).when(dagManagementStateStore).getDagNodeWithJobStatus(any());
189+
// third node (job2) will be queried for by the KillDagProc because we are killing that node
190+
doReturn(new ImmutablePair<>(Optional.of(dag.getNodes().get(2)), Optional.of(jobStatus))).when(dagManagementStateStore).getDagNodeWithJobStatus(any());
190191
doReturn(com.google.common.base.Optional.of(dag)).when(flowCompilationValidationHelper).createExecutionPlanIfValid(any());
191192

192193
LaunchDagProc launchDagProc = new LaunchDagProc(new LaunchDagTask(new DagActionStore.DagAction("fg", "flow2",
@@ -215,8 +216,6 @@ public void killDagNode() throws IOException, URISyntaxException, InterruptedExc
215216
.getInvocations()
216217
.stream()
217218
.filter(a -> a.getMethod().getName().equals("cancelJob"))
218-
.filter(a -> ((Properties) a.getArgument(1))
219-
.getProperty(ConfigurationKeys.SPEC_PRODUCER_SERIALIZED_FUTURE).equals(MockedSpecExecutor.dummySerializedFuture))
220219
.count())
221220
.sum();
222221
// kill dag proc tries to cancel only the exact dag node that was provided
@@ -226,5 +225,20 @@ public void killDagNode() throws IOException, URISyntaxException, InterruptedExc
226225
.submit(eq(TimingEvent.LauncherTimings.JOB_CANCEL), anyMap());
227226
Mockito.verify(this.mockedEventSubmitter, Mockito.times(numOfCancelledFlows))
228227
.submit(eq(TimingEvent.FlowTimings.FLOW_CANCELLED), anyMap());
228+
229+
Assert.assertEquals(dag.getNodes().get(0).getValue().getExecutionStatus(), ExecutionStatus.ORCHESTRATED); // because this was a mocked dag and we launched this job
230+
Assert.assertEquals(dag.getNodes().get(1).getValue().getExecutionStatus(), ExecutionStatus.PENDING); // because this was a mocked dag and we did not launch the job
231+
Assert.assertEquals(dag.getNodes().get(2).getValue().getExecutionStatus(), ExecutionStatus.CANCELLED); // because we cancelled this job
232+
Assert.assertEquals(dag.getNodes().get(3).getValue().getExecutionStatus(), ExecutionStatus.PENDING); // because this was a mocked dag and we did not launch the job
233+
Assert.assertEquals(dag.getNodes().get(4).getValue().getExecutionStatus(), ExecutionStatus.SKIPPED); // because its parent job was cancelled
234+
235+
Assert.assertTrue(dagManagementStateStore.hasRunningJobs(dagId));
236+
237+
dag.getNodes().get(0).getValue().setExecutionStatus(ExecutionStatus.COMPLETE);
238+
dag.getNodes().get(1).getValue().setExecutionStatus(ExecutionStatus.COMPLETE);
239+
dag.getNodes().get(3).getValue().setExecutionStatus(ExecutionStatus.COMPLETE);
240+
241+
doReturn(new HashSet<>(dag.getNodes())).when(dagManagementStateStore).getDagNodes(any());
242+
Assert.assertFalse(dagManagementStateStore.hasRunningJobs(dagId));
229243
}
230244
}

0 commit comments

Comments
 (0)