[
https://issues.apache.org/jira/browse/GOBBLIN-2022?focusedWorklogId=911480&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-911480
]
ASF GitHub Bot logged work on GOBBLIN-2022:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 26/Mar/24 06:14
Start Date: 26/Mar/24 06:14
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #3896:
URL: https://github.com/apache/gobblin/pull/3896#discussion_r1538513094
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagProcFactory.java:
##########
@@ -39,16 +42,24 @@
public class DagProcFactory implements DagTaskVisitor<DagProc> {
private final FlowCompilationValidationHelper
flowCompilationValidationHelper;
+ private final JobStatusRetriever jobStatusRetriever;
Review Comment:
after your latest revision, perhaps this is no longer used?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProcUtils.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Maps;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManager;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.TimingEventUtils;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+@Slf4j
+public class DagProcUtils {
Review Comment:
needs javadoc
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java:
##########
@@ -46,18 +50,21 @@ public final void process(DagManagementStateStore
dagManagementStateStore) throw
S state = initialize(dagManagementStateStore); // todo - retry
T result = act(dagManagementStateStore, state); // todo - retry
commit(dagManagementStateStore, result); // todo - retry
- sendNotification(result, eventSubmitter); // todo - retry
log.info("{} successfully concluded actions for dagId : {}",
getClass().getSimpleName(), getDagId());
}
- protected abstract DagManager.DagId getDagId();
+ protected DagManager.DagId getDagId() {
+ return this.dagTask.getDagId();
+ }
+
+ protected DagNodeId getDagNodeId() {
+ return this.dagTask.getDagNodeId();
+ }
Review Comment:
this is fine, but alternatively you could provide only the most specific
(`getDagNodeId`), and then expect callers to create their own `DagId` via:
```
myDagProc.getDagNodeId().getDagId()
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (this.jobStatus.get().isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode.get()),
+ jobStatus.get().getCurrentAttempts(),
jobStatus.get().getMaxAttempts());
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode.get(),
getDagId());
+ }
+
+ if (!DagProcUtils.hasRunningJobs(getDagId(), dagManagementStateStore)) {
+ if (dag.getFlowEvent() == null) {
+ // If the dag flow event is not set, then it is successful
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
+ // send an event before cleaning up dag
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ // todo - verify if work from PR#3641 is required
+ dagManagementStateStore.deleteDag(getDagId());
+ } else {
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ dagManagementStateStore.markDagFailed(dag);
+ }
Review Comment:
several concerns seem mixed together. what do you think of:
```
if (dag.getFlowEvent() == null) {
// If the dag flow event is not set, then it is successful
dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
}
FlowEvent flowEvent = dag.getFlowEvent();
DagManagerUtils.emitFlowEvent(eventSubmitter, dag, flowEvent);
if (flowEvent.equals(TimingEvent.FlowTimings.FLOW_SUCCEEDED)) {
// todo - verify if work from PR#3641 is required
dagManagementStateStore.deleteDag(getDagId());
} else {
dagManagementStateStore.markDagFailed(dag);
}
```
?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final JobStatusRetriever jobStatusRetriever;
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private JobStatus jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask,
JobStatusRetriever jobStatusRetriever) {
+ super(reEvaluateDagTask);
+ this.jobStatusRetriever = jobStatusRetriever;
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
Review Comment:
ok. so when could it arise?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java:
##########
@@ -42,11 +43,14 @@ public abstract class DagTask {
@Getter public final DagActionStore.DagAction dagAction;
private final MultiActiveLeaseArbiter.LeaseObtainedStatus
leaseObtainedStatus;
@Getter protected final DagManager.DagId dagId;
+ @Getter protected final DagNodeId dagNodeId;
Review Comment:
why store these in a field, rather than just having the impl in:
```
public DagMgr.DagId getDagId() { ... }
```
(which computes it on demand)
?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java:
##########
@@ -42,11 +43,14 @@ public abstract class DagTask {
@Getter public final DagActionStore.DagAction dagAction;
private final MultiActiveLeaseArbiter.LeaseObtainedStatus
leaseObtainedStatus;
@Getter protected final DagManager.DagId dagId;
+ @Getter protected final DagNodeId dagNodeId;
public DagTask(DagActionStore.DagAction dagAction,
MultiActiveLeaseArbiter.LeaseObtainedStatus leaseObtainedStatus) {
this.dagAction = dagAction;
this.leaseObtainedStatus = leaseObtainedStatus;
this.dagId = DagManagerUtils.generateDagId(dagAction.getFlowGroup(),
dagAction.getFlowName(), dagAction.getFlowExecutionId());
+ this.dagNodeId = new DagNodeId(dagAction.getFlowGroup(),
dagAction.getFlowName(),
+ Long.parseLong(dagAction.getFlowExecutionId()),
dagAction.getFlowGroup(), dagAction.getJobName());
Review Comment:
the reason I encouraged you to add a method `DagAction::getDagNodeId` is
that every one of these params came from the same `dagAction`. hence, the
"conversion" belongs either in that class or as a `static` in `DagNodeId`:
```
public static DagNodeId fromDagAction(DagAction dagAction) { ... }
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
Review Comment:
feels like this too belongs in `initialize`. could it go there?
##########
gobblin-metrics-libs/gobblin-metrics-base/src/main/avro/DagActionStoreChangeEvent.avsc:
##########
@@ -31,12 +31,14 @@
"symbols": [
"KILL",
"RESUME",
- "LAUNCH"
+ "LAUNCH",
+ "REEVALUATE"
],
"symbolDocs": {
"KILL": "Kill the flow corresponding to this dag",
"RESUME": "Resume or start a new flow corresponding to this dag",
- "LAUNCH": "Launch a new execution of the flow corresponding to this
dag"
+ "LAUNCH": "Launch a new execution of the flow corresponding to this
dag",
+ "REEVALUATE": "Re-evaluate what needs to be done when a job status is
received"
Review Comment:
nit: "...upon receipt of a *final* job status"
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MostlyMySqlDagManagementStateStore.java:
##########
@@ -235,4 +242,16 @@ public void
tryAcquireQuota(Collection<Dag.DagNode<JobExecutionPlan>> dagNodes)
public boolean releaseQuota(Dag.DagNode<JobExecutionPlan> dagNode) throws
IOException {
return this.quotaManager.releaseQuota(dagNode);
}
+
+ @Override
+ public Optional<JobStatus> getJobStatus(String flowGroup, String flowName,
long flowExecutionId, String jobGroup, String jobName) {
+ Iterator<JobStatus> jobStatusIterator =
+ jobStatusRetriever.getJobStatusesForFlowExecution(flowName, flowGroup,
flowExecutionId, jobName, jobGroup);
+
+ if (jobStatusIterator.hasNext()) {
+ return Optional.of(jobStatusIterator.next());
Review Comment:
suggest a comment to describe why you only return the first job status.
what's the meaning/importance of the others you might skip?
##########
gobblin-service/src/test/java/org/apache/gobblin/service/monitoring/FsJobStatusRetrieverTest.java:
##########
@@ -47,6 +50,8 @@ public void setUp() throws Exception {
.withValue(FsJobStatusRetriever.CONF_PREFIX + "." +
ConfigurationKeys.STATE_STORE_ROOT_DIR_KEY,
ConfigValueFactory.fromAnyRef(stateStoreDir));
this.jobStatusRetriever = new FsJobStatusRetriever(config,
mock(MultiContextIssueRepository.class));
+ this.mysqlDagActionStore = mock(MysqlDagActionStore.class);
+ doNothing().when(this.mysqlDagActionStore).addDagAction(any(), any(),
any(), any());
Review Comment:
from looking at this PR's diff, I can't see why this is necessary. (I also
wouldn't expect to see a mysql store being necessary for a FS retriever test.)
these are strong indications that it deserves a source comment
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProcUtils.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Future;
+
+import com.google.common.collect.Maps;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.JobSpec;
+import org.apache.gobblin.runtime.api.Spec;
+import org.apache.gobblin.runtime.api.SpecExecutor;
+import org.apache.gobblin.runtime.api.SpecProducer;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManager;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.TimingEventUtils;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+
+
+@Slf4j
+public class DagProcUtils {
+
+ /**
+ * - submits a {@link JobSpec} to a {@link SpecExecutor}
+ * - updates metrics
+ * - add updated dag node state to dagManagementStateStore
+ */
+ static void submitJobToExecutor(DagManagementStateStore
dagManagementStateStore, Dag.DagNode<JobExecutionPlan> dagNode,
+ DagManager.DagId dagId) {
+ DagManagerUtils.incrementJobAttempt(dagNode);
+ JobExecutionPlan jobExecutionPlan =
DagManagerUtils.getJobExecutionPlan(dagNode);
+ JobSpec jobSpec = DagManagerUtils.getJobSpec(dagNode);
+ Map<String, String> jobMetadata =
TimingEventUtils.getJobMetadata(Maps.newHashMap(), jobExecutionPlan);
+
+ String specExecutorUri = DagManagerUtils.getSpecExecutorUri(dagNode);
+
+ // Run this spec on selected executor
+ SpecProducer<Spec> producer;
+ try {
+ producer = DagManagerUtils.getSpecProducer(dagNode);
+ TimingEvent jobOrchestrationTimer =
DagProc.eventSubmitter.getTimingEvent(TimingEvent.LauncherTimings.JOB_ORCHESTRATED);
Review Comment:
I somehow missed earlier that `eventSubmitter` was `static`, but seeing it
used here does look off.
we can wait and see how things develop before reworking too much. but,
still, I'd personally hedge my bets, by adding this method to `DagProc`:
```
protected TimingEvent createTimingEvent(String name) {
return eventSubmitter.getTimingEvent(name);
}
```
and then having this method take an additional param of `Function<String,
TimingEvent> createTimingEvent`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
Review Comment:
my instinct is to make this one combined, atomic call into the DMSS, e.g.:
```
Optional<X> getDagNodeWithJobStatus(DagNodeId dagNodeId);
```
where X is some paring of `(Dag.DagNode<JEP>, JobStatus)`
then that type `Optional<X>` would be the return type of `initialize` and
the param type of `act`, so there would be no need to separately maintain an
`Optional<JobState> jobState` member
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (this.jobStatus.get().isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode.get()),
+ jobStatus.get().getCurrentAttempts(),
jobStatus.get().getMaxAttempts());
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode.get(),
getDagId());
+ }
+
+ if (!DagProcUtils.hasRunningJobs(getDagId(), dagManagementStateStore)) {
+ if (dag.getFlowEvent() == null) {
+ // If the dag flow event is not set, then it is successful
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
+ // send an event before cleaning up dag
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ // todo - verify if work from PR#3641 is required
+ dagManagementStateStore.deleteDag(getDagId());
+ } else {
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ dagManagementStateStore.markDagFailed(dag);
+ }
+ }
+
+ return null;
+ }
+
+ /**
+ * Sets status of a dag node inside the given Dag.
+ * todo - DMSS should support this functionality like an atomic get-and-set
operation.
Review Comment:
... and speaking of atomicity, should it be compare-and-set?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagManagementStateStore.java:
##########
@@ -186,4 +187,9 @@ default void deleteFailedDag(Dag<JobExecutionPlan> dag)
throws IOException {
* Returns a {@link DagManagerMetrics} that monitors dags execution.
*/
DagManagerMetrics getDagManagerMetrics();
+
+ /**
+ * Returns job status or empty optional if job status is not present in the
JobStatusStore.
+ */
+ Optional<JobStatus> getJobStatus(String flowGroup, String flowName, long
flowExecutionId, String jobGroup, String jobName);
Review Comment:
`getDagNode` already takes a `DagNodeId` param. having this also take
`DagNodeId` would improve abstraction
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (this.jobStatus.get().isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode.get()),
+ jobStatus.get().getCurrentAttempts(),
jobStatus.get().getMaxAttempts());
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode.get(),
getDagId());
+ }
+
+ if (!DagProcUtils.hasRunningJobs(getDagId(), dagManagementStateStore)) {
Review Comment:
should this be `else if`?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -275,6 +279,7 @@ static void
addJobStatusToStateStore(org.apache.gobblin.configuration.State jobS
modifyStateIfRetryRequired(jobStatus);
stateStore.put(storeName, tableName, jobStatus);
if (isNewStateTransitionToFinal(jobStatus, states)) {
+ dagActionStore.addDagAction(flowGroup, flowName, flowExecutionId,
jobName, DagActionStore.DagActionType.REEVALUATE);
eventProducer.emitObservabilityEvent(jobStatus);
Review Comment:
the fact that you must add mocking for the `DagActionStore` just to keep
existing tests working that call this `static` method, suggests the method is
doing too much. instead, how about returning `Optional<JobStatus>` that is
present only in cases where `isNewStateTransitionToFinal`?
that way `dagActionStore` and `eventProducer` are only called directly
within `processMessage`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (this.jobStatus.get().isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode.get()),
+ jobStatus.get().getCurrentAttempts(),
jobStatus.get().getMaxAttempts());
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode.get(),
getDagId());
+ }
+
+ if (!DagProcUtils.hasRunningJobs(getDagId(), dagManagementStateStore)) {
+ if (dag.getFlowEvent() == null) {
+ // If the dag flow event is not set, then it is successful
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
+ // send an event before cleaning up dag
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ // todo - verify if work from PR#3641 is required
+ dagManagementStateStore.deleteDag(getDagId());
+ } else {
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ dagManagementStateStore.markDagFailed(dag);
+ }
+ }
+
+ return null;
+ }
+
+ /**
+ * Sets status of a dag node inside the given Dag.
+ * todo - DMSS should support this functionality like an atomic get-and-set
operation.
+ */
+ private void setStatus(DagManagementStateStore dagManagementStateStore,
+ Dag.DagNode<JobExecutionPlan> dagNode, ExecutionStatus executionStatus)
throws IOException {
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+ DagNodeId dagNodeId = dagNode.getValue().getId();
+ for (Dag.DagNode<JobExecutionPlan> node : dag.getNodes()) {
+ if (node.getValue().getId().equals(dagNodeId)) {
+ node.getValue().setExecutionStatus(executionStatus);
+ dagManagementStateStore.checkpointDag(dag);
+ return;
+ }
+ }
+ log.error("DagNode with id {} not found in Dag {}", dagNodeId, getDagId());
+ }
+
+ /**
+ * Method that defines the actions to be performed when a job finishes
either successfully or with failure.
+ * This method updates the state of the dag and performs clean up actions as
necessary.
+ */
+ private void onJobFinish(DagManagementStateStore dagManagementStateStore,
+ Dag.DagNode<JobExecutionPlan> dagNode, ExecutionStatus executionStatus)
+ throws IOException {
+ String jobName = DagManagerUtils.getFullyQualifiedJobName(dagNode);
+ log.info("Job {} of Dag {} has finished with status {}", jobName,
getDagId(), executionStatus.name());
+ // Only decrement counters and quota for jobs that actually ran on the
executor, not from a GaaS side failure/skip event
+ if (dagManagementStateStore.releaseQuota(dagNode)) {
+
dagManagementStateStore.getDagManagerMetrics().decrementRunningJobMetrics(dagNode);
+ }
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ switch (executionStatus) {
+ case FAILED:
+ dag.setMessage("Flow failed because job " + jobName + " failed");
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_FAILED);
+
dagManagementStateStore.getDagManagerMetrics().incrementExecutorFailed(dagNode);
+ break;
+ case CANCELLED:
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_CANCELLED);
+ break;
+ case COMPLETE:
+
dagManagementStateStore.getDagManagerMetrics().incrementExecutorSuccess(dagNode);
+ submitNext(dagManagementStateStore);
+ break;
+ default:
+ log.warn("It should not reach here. Job status {} is unexpected.",
executionStatus);
+ }
+
+ //Checkpoint the dag state, it should have an updated value of dag fields
+ dagManagementStateStore.checkpointDag(dag);
+ }
+
+ /**
+ * Submit next set of Dag nodes in the Dag identified by the provided dagId
+ */
+ void submitNext(DagManagementStateStore dagManagementStateStore) throws
IOException {
+ // get the most up-to-date dag from the store before finding the next dag
nodes to run
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+ Set<Dag.DagNode<JobExecutionPlan>> nextNodes =
DagManagerUtils.getNext(dag);
+
+ if (nextNodes.size() > 1) {
+ handleMultipleJobs(nextNodes);
+ }
+
+ if (!nextNodes.isEmpty()) {
+ Dag.DagNode<JobExecutionPlan> nextNode =
nextNodes.stream().findFirst().get();
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, nextNode,
getDagId());
+ log.info("Submitted job {} for dagId {}",
DagManagerUtils.getJobName(nextNode), getDagId());
+ }
+ }
+
+ private DagNodeId findDagNodeForDagAction(DagActionStore.DagAction
dagAction) {
+ return new DagNodeId(dagAction.getFlowGroup(), dagAction.getFlowName(),
Long.parseLong(dagAction.getFlowExecutionId()),
+ dagAction.getFlowGroup(), dagAction.getJobName());
+ }
Review Comment:
may not be used anywhere... is it?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java:
##########
@@ -46,18 +50,21 @@ public final void process(DagManagementStateStore
dagManagementStateStore) throw
S state = initialize(dagManagementStateStore); // todo - retry
T result = act(dagManagementStateStore, state); // todo - retry
commit(dagManagementStateStore, result); // todo - retry
- sendNotification(result, eventSubmitter); // todo - retry
log.info("{} successfully concluded actions for dagId : {}",
getClass().getSimpleName(), getDagId());
}
- protected abstract DagManager.DagId getDagId();
+ protected DagManager.DagId getDagId() {
+ return this.dagTask.getDagId();
+ }
+
+ protected DagNodeId getDagNodeId() {
+ return this.dagTask.getDagNodeId();
+ }
protected abstract S initialize(DagManagementStateStore
dagManagementStateStore) throws IOException;
protected abstract T act(DagManagementStateStore dagManagementStateStore, S
state) throws IOException;
- protected abstract void sendNotification(T result, EventSubmitter
eventSubmitter) throws IOException;
Review Comment:
if this really is imperative, let's document in code via a comment.
in addition, if we really find little value in the common pattern of `init
-> act -> sendNotification`, then there's likely no need for the generic param
`T` (thus: `void act(DMSS dmss, S state) throws IOE`)
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -275,6 +279,7 @@ static void
addJobStatusToStateStore(org.apache.gobblin.configuration.State jobS
modifyStateIfRetryRequired(jobStatus);
Review Comment:
post-`ReevaluateDagProc`, can't we keep the reading of and deciding about
retries minimal in or even absent from this class and situate it over there?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+import org.apache.gobblin.service.monitoring.JobStatusRetriever;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final JobStatusRetriever jobStatusRetriever;
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private JobStatus jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask,
JobStatusRetriever jobStatusRetriever) {
+ super(reEvaluateDagTask);
+ this.jobStatusRetriever = jobStatusRetriever;
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+ this.jobStatus = DagManagerUtils.pollJobStatus(dagNode.get(),
this.jobStatusRetriever, this.jobStatusPolledTimer).get();
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Expected Statuses are {}",
dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
Review Comment:
seem error-prone, even in the near-term, not to have a SSoT. please explain
more to justify why it actually would be safe, or else describe how we could
merge the two into a SSoT
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,207 @@
+/*
+ * 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.gobblin.service.modules.orchestration.proc;
+
+import java.io.IOException;
+import java.util.Optional;
+import java.util.Set;
+
+import com.codahale.metrics.Timer;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.ServiceMetricNames;
+import org.apache.gobblin.metrics.event.TimingEvent;
+import org.apache.gobblin.runtime.api.DagActionStore;
+import org.apache.gobblin.service.ExecutionStatus;
+import org.apache.gobblin.service.modules.flowgraph.Dag;
+import org.apache.gobblin.service.modules.flowgraph.DagNodeId;
+import
org.apache.gobblin.service.modules.orchestration.DagManagementStateStore;
+import org.apache.gobblin.service.modules.orchestration.DagManagerUtils;
+import org.apache.gobblin.service.modules.orchestration.task.ReevaluateDagTask;
+import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
+import org.apache.gobblin.service.monitoring.FlowStatusGenerator;
+import org.apache.gobblin.service.monitoring.JobStatus;
+
+
+/**
+ * An implementation for {@link DagProc} that launches a new job if there
exists a job whose pre-requisite jobs are
+ * completed successfully. If there are no more jobs to run and no job is
running for the Dag, it cleans up the Dag.
+ * (In future), if there are multiple new jobs to be launched, separate launch
dag actions are created for each of them.
+ */
+@Slf4j
+public class ReevaluateDagProc extends
DagProc<Optional<Dag.DagNode<JobExecutionPlan>>, Void> {
+ private final Timer jobStatusPolledTimer;
+ private final DagNodeId dagNodeId;
+ private Optional<JobStatus> jobStatus;
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ this.jobStatusPolledTimer =
metricContext.timer(ServiceMetricNames.JOB_STATUS_POLLED_TIMER);
+ this.dagNodeId = getDagNodeId();
+ }
+
+ @Override
+ protected Optional<Dag.DagNode<JobExecutionPlan>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Optional<Dag.DagNode<JobExecutionPlan>> dagNode =
dagManagementStateStore.getDagNode(this.dagNodeId);
+ if (!dagNode.isPresent()) {
+ log.error("DagNode not found for a Reevaluate DagAction with dag node id
{}", this.dagNodeId);
+ return Optional.empty();
+ }
+
+ this.jobStatus =
dagManagementStateStore.getJobStatus(getDagNodeId().getFlowGroup(),
getDagNodeId().getFlowName(),
+ getDagNodeId().getFlowExecutionId(), getDagNodeId().getJobGroup(),
getDagNodeId().getJobName());
+ if (!this.jobStatus.isPresent()) {
+ log.error("JobStatus not found for the dag node {}", getDagNodeId());
+ return Optional.empty();
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(jobStatus.get().getEventName());
+ if
(!FlowStatusGenerator.FINISHED_STATUSES.contains(executionStatus.name())) {
+ log.warn("Job status for dagNode {} is {}. Re-evaluate dag action should
have been created only for finished status - {}",
+ dagNodeId, executionStatus, FlowStatusGenerator.FINISHED_STATUSES);
+ return Optional.empty();
+ }
+
+ setStatus(dagManagementStateStore, dagNode.get(), executionStatus);
+ return dagNode;
+ }
+
+ @Override
+ protected Void act(DagManagementStateStore dagManagementStateStore,
Optional<Dag.DagNode<JobExecutionPlan>> dagNode)
+ throws IOException {
+ if (!dagNode.isPresent()) {
+ // for several reasons it may receive an empty dag node, refer
initialize method
+ return null;
+ }
+
+ ExecutionStatus executionStatus =
dagNode.get().getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode.get(), executionStatus);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode.get());
+
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (this.jobStatus.get().isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode.get()),
+ jobStatus.get().getCurrentAttempts(),
jobStatus.get().getMaxAttempts());
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode.get(),
getDagId());
+ }
+
+ if (!DagProcUtils.hasRunningJobs(getDagId(), dagManagementStateStore)) {
+ if (dag.getFlowEvent() == null) {
+ // If the dag flow event is not set, then it is successful
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
+ // send an event before cleaning up dag
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ // todo - verify if work from PR#3641 is required
+ dagManagementStateStore.deleteDag(getDagId());
+ } else {
+ DagManagerUtils.emitFlowEvent(eventSubmitter, dag, dag.getFlowEvent());
+ dagManagementStateStore.markDagFailed(dag);
+ }
Review Comment:
once we separate it like this, it again raises the question of whether the
first two lines in my comment above belong better within `onJobFinish` (as
[discussed](https://github.com/apache/gobblin/pull/3896/files#r1538378681))
Issue Time Tracking
-------------------
Worklog Id: (was: 911480)
Time Spent: 3h (was: 2h 50m)
> create dag proc for taking actions on job completion
> ----------------------------------------------------
>
> Key: GOBBLIN-2022
> URL: https://issues.apache.org/jira/browse/GOBBLIN-2022
> Project: Apache Gobblin
> Issue Type: Task
> Reporter: Arjun Singh Bora
> Priority: Major
> Time Spent: 3h
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)