[
https://issues.apache.org/jira/browse/GOBBLIN-2022?focusedWorklogId=912870&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-912870
]
ASF GitHub Bot logged work on GOBBLIN-2022:
-------------------------------------------
Author: ASF GitHub Bot
Created on: 03/Apr/24 16:50
Start Date: 03/Apr/24 16:50
Worklog Time Spent: 10m
Work Description: phet commented on code in PR #3896:
URL: https://github.com/apache/gobblin/pull/3896#discussion_r1550017946
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MostlyMySqlDagManagementStateStore.java:
##########
@@ -211,12 +211,13 @@ public boolean containsDag(DagManager.DagId dagId) throws
IOException {
return this.dagStateStore.existsDag(dagId);
}
- public Optional<Pair<Dag.DagNode<JobExecutionPlan>, JobStatus>>
getDagNodeWithJobStatus(DagNodeId dagNodeId) {
- Optional<JobStatus> jobStatus = getJobStatus(dagNodeId);
- if (this.dagNodes.containsKey(dagNodeId) && jobStatus.isPresent()) {
- return Optional.of(ImmutablePair.of(this.dagNodes.get(dagNodeId),
jobStatus.get()));
+ public Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
getDagNodeWithJobStatus(DagNodeId dagNodeId) {
+ if (this.dagNodes.containsKey(dagNodeId)) {
+ // no point of searching for status if the node itself is absent.
Review Comment:
good comment... but let's move to the `else` block
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/MostlyMySqlDagManagementStateStore.java:
##########
@@ -211,12 +211,13 @@ public boolean containsDag(DagManager.DagId dagId) throws
IOException {
return this.dagStateStore.existsDag(dagId);
}
- public Optional<Pair<Dag.DagNode<JobExecutionPlan>, JobStatus>>
getDagNodeWithJobStatus(DagNodeId dagNodeId) {
- Optional<JobStatus> jobStatus = getJobStatus(dagNodeId);
- if (this.dagNodes.containsKey(dagNodeId) && jobStatus.isPresent()) {
- return Optional.of(ImmutablePair.of(this.dagNodes.get(dagNodeId),
jobStatus.get()));
+ public Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
getDagNodeWithJobStatus(DagNodeId dagNodeId) {
+ if (this.dagNodes.containsKey(dagNodeId)) {
+ // no point of searching for status if the node itself is absent.
+ Optional<JobStatus> jobStatus = getJobStatus(dagNodeId);
Review Comment:
probably personal preference, but I don't see a need to introduce the name
`jobStatus` (since `getJobStatus(dagNodeId)` is already clear)
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -221,13 +236,13 @@ protected void
processMessage(DecodeableKafkaRecord<byte[],byte[]> message) {
* It fills missing fields in job status and also merge the fields with the
* existing job status in the state store. Merging is required because we
* do not want to lose the information sent by other GobblinTrackingEvents.
- * @param jobStatus
+ * Returns false if adding this state transitions the job status of the job
to final, otherwise returns false.
+ * It will also return false if the job status was already final before
calling this method.
Review Comment:
this is out-of-date for the `Optional`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/ReevaluateDagTask.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.task;
+
+import org.apache.gobblin.service.modules.orchestration.DagActionStore;
+import org.apache.gobblin.service.modules.orchestration.DagTaskVisitor;
+import org.apache.gobblin.service.modules.orchestration.LeaseAttemptStatus;
+
+
+/**
+ * A {@link DagTask} responsible to handle re-evaluate dag actions.
+ */
+
+public class ReevaluateDagTask extends DagTask {
+ public ReevaluateDagTask(DagActionStore.DagAction dagAction,
LeaseAttemptStatus.LeaseObtainedStatus leaseObtainedStatus,
+ DagActionStore dagActionStore) {
+ super(dagAction, leaseObtainedStatus, dagActionStore);
+ }
+
+ public <T> T host(DagTaskVisitor<T> visitor) {
+ return visitor.meet(this);
+ }
+
+ @Override
+ public boolean conclude() {
+ // todo - release lease
+ return true;
+ }
Review Comment:
since urmi's recent change is in, looks like a "merge oversight", as [we now
conclude
in](https://github.com/apache/gobblin/blob/fb85f07e544dae8cd51edff1387136a55df28aa5/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/task/DagTask.java#L60)
`DagTask`.
to guard against this possible error, let's make it
```
public *final* boolean DagTask::conclude
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+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<Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>> {
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ }
+
+ @Override
+ protected Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus =
+ dagManagementStateStore.getDagNodeWithJobStatus(this.dagNodeId);
+
+ if (!dagNodeWithJobStatus.getLeft().isPresent() ||
!dagNodeWithJobStatus.getRight().isPresent()) {
+ // this is possible when MALA malfunctions and a duplicated reevaluate
dag proc is launched for a dag node that is
+ // already "reevaluated" and cleaned up.
+ return ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(dagNodeWithJobStatus.getRight().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 ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ setStatus(dagManagementStateStore, dagNodeWithJobStatus.getLeft().get(),
executionStatus);
+ return dagNodeWithJobStatus;
+ }
+
+ @Override
+ protected void act(DagManagementStateStore dagManagementStateStore,
Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus)
+ throws IOException {
+ if (!dagNodeWithJobStatus.getLeft().isPresent()) {
+ log.error("DagNode or its job status not found for a Reevaluate
DagAction with dag node id {}", this.dagNodeId);
+ return;
+ }
+
+ Dag.DagNode<JobExecutionPlan> dagNode =
dagNodeWithJobStatus.getLeft().get();
+ JobStatus jobStatus = dagNodeWithJobStatus.getRight().get();
+ ExecutionStatus executionStatus = dagNode.getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode, executionStatus);
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
Review Comment:
rather than the ordering:
1. calling `onJobFinish`
2. inside that callstack accessing `DMSS::getDag`
3. returning from `onJobFinish`
4. here calling `DMSS::getDag`
could you instead have:
1. called `DMSS::getDag` here in `act`
2. passed that to `onJobFinish`, so no need to `DMSS::getDag` within
3. after `onJobFinish` returns, continue using *that same` result of (1.)
?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+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<Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>> {
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ }
+
+ @Override
+ protected Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus =
+ dagManagementStateStore.getDagNodeWithJobStatus(this.dagNodeId);
+
+ if (!dagNodeWithJobStatus.getLeft().isPresent() ||
!dagNodeWithJobStatus.getRight().isPresent()) {
+ // this is possible when MALA malfunctions and a duplicated reevaluate
dag proc is launched for a dag node that is
+ // already "reevaluated" and cleaned up.
+ return ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(dagNodeWithJobStatus.getRight().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 ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ setStatus(dagManagementStateStore, dagNodeWithJobStatus.getLeft().get(),
executionStatus);
+ return dagNodeWithJobStatus;
+ }
+
+ @Override
+ protected void act(DagManagementStateStore dagManagementStateStore,
Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus)
+ throws IOException {
+ if (!dagNodeWithJobStatus.getLeft().isPresent()) {
+ log.error("DagNode or its job status not found for a Reevaluate
DagAction with dag node id {}", this.dagNodeId);
+ return;
Review Comment:
I agree with logging at `ERROR`, also to still `return`, rather than
throwing. given the severity, I suggest further to increment a metric we might
be able to alert on.
in addition, let's please add a code comment to capture the understanding
for maintainers. did you say this arises when the MALA leasing doesn't work
cleanly and another `DagProc::process` has cleaned up the Dag, yet did not
complete the lease before this current one acquired its own?
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -277,9 +292,8 @@ static void
addJobStatusToStateStore(org.apache.gobblin.configuration.State jobS
modifyStateIfRetryRequired(jobStatus);
stateStore.put(storeName, tableName, jobStatus);
- if (isNewStateTransitionToFinal(jobStatus, states)) {
- eventProducer.emitObservabilityEvent(jobStatus);
- }
+
+ return isNewStateTransitionToFinal(jobStatus, states) ?
Optional.of(jobStatus) : Optional.empty();
Review Comment:
more canonical:
```
return Optional.of(jobStatus).filter(jobStatus ->
isNewStateTransitionToFinal(jobStatus, states)
);
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/DagActionStore.java:
##########
@@ -35,7 +35,7 @@ enum DagActionType {
LAUNCH, // Launch new flow execution invoked adhoc or through scheduled
trigger
RETRY, // Invoked through DagManager for flows configured to allow retries
CANCEL, // Invoked through DagManager if flow has been stuck in
Orchestrated state for a while
- ADVANCE // Launch next step in multi-hop dag
+ REEVALUATE // Launch next step in multi-hop dag
Review Comment:
nit: let's align this comment w/ the description you just added to
`DagActionStoreChangeEvent.avsc`
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/DagProc.java:
##########
@@ -40,6 +40,8 @@
* actions based on the type of {@link DagTask}. Submitting events in time is
found to be important in
* <a href="https://github.com/apache/gobblin/pull/3641">PR#3641</a>, hence
initialize and act methods submit events as
* they happen.
+ * Parameter T is the type of object that needs an initialisation before the
dag proc does the main work in `act` method.
Review Comment:
great description! the javadoc syntax should be:
```
@param <T> type of the initialization "state" on which to {@link DagProc#act}
```
(the key above is `<T>` syntax for a generic type name, rather than `T`/`t`,
for a regular param name)
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitorFactory.java:
##########
@@ -49,24 +52,29 @@ public class KafkaJobStatusMonitorFactory implements
Provider<KafkaJobStatusMoni
private final JobIssueEventHandler jobIssueEventHandler;
private final MultiContextIssueRepository issueRepository;
private final boolean instrumentationEnabled;
+ private final DagActionStore dagActionStore;
+ private final boolean dagProcEngineEnabled;
Review Comment:
I didn't see this being used in `KafkaJobStatusMonitor`, so you don't seem
to need to have it here.
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+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.
Review Comment:
suggest:
```
A {@link DagProc} to launch any subsequent (dependent) job(s) once all
pre-requisite job(s) in the Dag have succeeded. When there are no more jobs to
run and no more running, it cleans up the Dag.
```
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -193,7 +197,18 @@ protected void
processMessage(DecodeableKafkaRecord<byte[],byte[]> message) {
org.apache.gobblin.configuration.State jobStatus =
parseJobStatus(gobblinTrackingEvent);
if (jobStatus != null) {
try (Timer.Context context =
getMetricContext().timer(GET_AND_SET_JOB_STATUS).time()) {
- addJobStatusToStateStore(jobStatus, this.stateStore,
this.eventProducer);
+ Optional<org.apache.gobblin.configuration.State> updatedJobStatus
= addJobStatusToStateStore(jobStatus, this.stateStore);
+ boolean isJobStatusUpdated = updatedJobStatus.isPresent();
Review Comment:
I don't see a need for this intermediate name. javadoc on
`addJobStatusToStateStore` should be the place to convey semantics of
`Optional::isPresent()` vs. not
##########
gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/ReevaluateDagProc.java:
##########
@@ -0,0 +1,194 @@
+/*
+ * 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 org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.metrics.event.TimingEvent;
+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<Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>> {
+
+ public ReevaluateDagProc(ReevaluateDagTask reEvaluateDagTask) {
+ super(reEvaluateDagTask);
+ }
+
+ @Override
+ protected Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
initialize(DagManagementStateStore dagManagementStateStore)
+ throws IOException {
+ Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus =
+ dagManagementStateStore.getDagNodeWithJobStatus(this.dagNodeId);
+
+ if (!dagNodeWithJobStatus.getLeft().isPresent() ||
!dagNodeWithJobStatus.getRight().isPresent()) {
+ // this is possible when MALA malfunctions and a duplicated reevaluate
dag proc is launched for a dag node that is
+ // already "reevaluated" and cleaned up.
+ return ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ ExecutionStatus executionStatus =
ExecutionStatus.valueOf(dagNodeWithJobStatus.getRight().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 ImmutablePair.of(Optional.empty(), Optional.empty());
+ }
+
+ setStatus(dagManagementStateStore, dagNodeWithJobStatus.getLeft().get(),
executionStatus);
+ return dagNodeWithJobStatus;
+ }
+
+ @Override
+ protected void act(DagManagementStateStore dagManagementStateStore,
Pair<Optional<Dag.DagNode<JobExecutionPlan>>, Optional<JobStatus>>
dagNodeWithJobStatus)
+ throws IOException {
+ if (!dagNodeWithJobStatus.getLeft().isPresent()) {
+ log.error("DagNode or its job status not found for a Reevaluate
DagAction with dag node id {}", this.dagNodeId);
+ return;
+ }
+
+ Dag.DagNode<JobExecutionPlan> dagNode =
dagNodeWithJobStatus.getLeft().get();
+ JobStatus jobStatus = dagNodeWithJobStatus.getRight().get();
+ ExecutionStatus executionStatus = dagNode.getValue().getExecutionStatus();
+ onJobFinish(dagManagementStateStore, dagNode, executionStatus);
+ Dag<JobExecutionPlan> dag =
dagManagementStateStore.getDag(getDagId()).get();
+
+ if (jobStatus.isShouldRetry()) {
+ log.info("Retrying job: {}, current attempts: {}, max attempts: {}",
+ DagManagerUtils.getFullyQualifiedJobName(dagNode),
jobStatus.getCurrentAttempts(), jobStatus.getMaxAttempts());
+ // todo - be careful when unsetting this, it is possible that this is
set to FAILED because some other job in the
+ // dag failed and is also not retryable. in that case if this job's
retry passes, overall status of the dag can be
+ // set to PASS, which would be incorrect.
+ dag.setFlowEvent(null);
+ DagProcUtils.submitJobToExecutor(dagManagementStateStore, dagNode,
getDagId());
+ } else if (!dagManagementStateStore.hasRunningJobs(getDagId())) {
+ if (dag.getFlowEvent() == null) {
+ // If the dag flow event is not set and there are no more jobs
running, then it is successful
+ // also note that `onJobFinish` method does whatever is required to do
after job finish, determining a Dag's
+ // status is not possible on individual job's finish status
+ dag.setFlowEvent(TimingEvent.FlowTimings.FLOW_SUCCEEDED);
+ }
+ String 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);
+ }
+ }
+ }
+
+ /**
+ * 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);
+ dagManagementStateStore.deleteDagNodeState(getDagId(), dagNode);
+ }
+
+ /**
+ * Submit next set of Dag nodes in the Dag identified by the provided dagId
+ */
+ void submitNext(DagManagementStateStore dagManagementStateStore) throws
IOException {
Review Comment:
we can wait to refactor until we actually implement `handleMultipleJobs`,
but for now, please keep this method signature
[aligned](https://github.com/apache/gobblin/blob/fb85f07e544dae8cd51edff1387136a55df28aa5/gobblin-service/src/main/java/org/apache/gobblin/service/modules/orchestration/proc/LaunchDagProc.java#L87)
w/ `LaunchDagProc::submitNext`.
my expectation is to eventually have a common base class (e.g.
`AbstractExecutorSubmittingDagProc<T>`), which only those two among the
`DagProc`s would derive from
##########
gobblin-service/src/main/java/org/apache/gobblin/service/monitoring/KafkaJobStatusMonitor.java:
##########
@@ -193,7 +197,18 @@ protected void
processMessage(DecodeableKafkaRecord<byte[],byte[]> message) {
org.apache.gobblin.configuration.State jobStatus =
parseJobStatus(gobblinTrackingEvent);
if (jobStatus != null) {
try (Timer.Context context =
getMetricContext().timer(GET_AND_SET_JOB_STATUS).time()) {
- addJobStatusToStateStore(jobStatus, this.stateStore,
this.eventProducer);
+ Optional<org.apache.gobblin.configuration.State> updatedJobStatus
= addJobStatusToStateStore(jobStatus, this.stateStore);
+ boolean isJobStatusUpdated = updatedJobStatus.isPresent();
+ // todo - retried/resumed jobs *may* not be handled here, we may
want to create their dag action elsewhere
+ if (isJobStatusUpdated) {
+ jobStatus = updatedJobStatus.get();
+ this.eventProducer.emitObservabilityEvent(jobStatus);
+ String flowName =
jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_NAME_FIELD);
+ String flowGroup =
jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_GROUP_FIELD);
+ String flowExecutionId =
jobStatus.getProp(TimingEvent.FlowEventConstants.FLOW_EXECUTION_ID_FIELD);
+ String jobName =
jobStatus.getProp(TimingEvent.FlowEventConstants.JOB_NAME_FIELD);
+ this.dagActionStore.addJobDagAction(flowGroup, flowName,
flowExecutionId, jobName, DagActionStore.DagActionType.REEVALUATE);
Review Comment:
seems like a common pattern. where would you suggest to add a util factory
method like:
```
DagAction createFromJobState(State s, DagActionType actionType)
```
?
Issue Time Tracking
-------------------
Worklog Id: (was: 912870)
Time Spent: 11h 40m (was: 11.5h)
> 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: 11h 40m
> Remaining Estimate: 0h
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)