[ 
https://issues.apache.org/jira/browse/GOBBLIN-2022?focusedWorklogId=912903&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-912903
 ]

ASF GitHub Bot logged work on GOBBLIN-2022:
-------------------------------------------

                Author: ASF GitHub Bot
            Created on: 03/Apr/24 19:55
            Start Date: 03/Apr/24 19:55
    Worklog Time Spent: 10m 
      Work Description: phet commented on code in PR #3896:
URL: https://github.com/apache/gobblin/pull/3896#discussion_r1550353159


##########
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:
   I really don't think those should be `static` (in `DagProcUtils`) but rather 
might belong to a common base class for both to derive from





Issue Time Tracking
-------------------

    Worklog Id:     (was: 912903)
    Time Spent: 13h 50m  (was: 13h 40m)

> 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: 13h 50m
>  Remaining Estimate: 0h
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to