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

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

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


##########
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. again calling `DMSS::getDag` here
   
   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.)
   ?





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

    Worklog Id:     (was: 912875)
    Time Spent: 12.5h  (was: 12h 20m)

> 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: 12.5h
>  Remaining Estimate: 0h
>




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

Reply via email to