Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[GOBBLIN-2119] ignore adding deadline dag actions if they are already present #4008

Merged
merged 4 commits into from
Jul 23, 2024
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@
package org.apache.gobblin.service.modules.orchestration.proc;

import java.io.IOException;
import java.sql.SQLIntegrityConstraintViolationException;
import java.util.Collections;
import java.util.List;
import java.util.Map;
Expand Down Expand Up @@ -193,15 +194,33 @@ public static void sendCancellationEvent(JobExecutionPlan jobExecutionPlan) {

private static void sendEnforceJobStartDeadlineDagAction(DagManagementStateStore dagManagementStateStore, Dag.DagNode<JobExecutionPlan> dagNode)
throws IOException {
dagManagementStateStore.addJobDagAction(dagNode.getValue().getFlowGroup(), dagNode.getValue().getFlowName(),
dagNode.getValue().getFlowExecutionId(), dagNode.getValue().getJobName(),
DagActionStore.DagAction dagAction = new DagActionStore.DagAction(dagNode.getValue().getFlowGroup(),
dagNode.getValue().getFlowName(), dagNode.getValue().getFlowExecutionId(), dagNode.getValue().getJobName(),
DagActionStore.DagActionType.ENFORCE_JOB_START_DEADLINE);
try {
dagManagementStateStore.addDagAction(dagAction);
} catch (IOException e) {
if (e.getCause() != null && e.getCause() instanceof SQLIntegrityConstraintViolationException) {
dagManagementStateStore.deleteDagAction(dagAction);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why delete then re-add? is this to trigger CDC again?

Copy link
Contributor Author

@arjun4084346 arjun4084346 Jul 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The intention is to have a deadline dag proc with the new deadline time.

Copy link
Contributor

@umustafi umustafi Jul 23, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see so we try launching same job but want to ensure the deadline is updated to reflect newer launch or at least that there's a deadline in memory for it

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

for the metrics on startup it will show up as several deletions and additions but I think this is ok. no negative values

log.warn("Duplicate ENFORCE_JOB_START_DEADLINE Dag Action is being created. Ignoring... " + e.getMessage());
dagManagementStateStore.addDagAction(dagAction);
}
}
}

public static void sendEnforceFlowFinishDeadlineDagAction(DagManagementStateStore dagManagementStateStore, DagActionStore.DagAction launchDagAction)
throws IOException {
dagManagementStateStore.addFlowDagAction(launchDagAction.getFlowGroup(), launchDagAction.getFlowName(),
DagActionStore.DagAction dagAction = DagActionStore.DagAction.forFlow(launchDagAction.getFlowGroup(), launchDagAction.getFlowName(),
launchDagAction.getFlowExecutionId(), DagActionStore.DagActionType.ENFORCE_FLOW_FINISH_DEADLINE);
try {
dagManagementStateStore.addDagAction(dagAction);
} catch (IOException e) {
if (e.getCause() != null && e.getCause() instanceof SQLIntegrityConstraintViolationException) {
dagManagementStateStore.deleteDagAction(dagAction);
log.warn("Duplicate ENFORCE_FLOW_FINISH_DEADLINE Dag Action is being created. Ignoring... " + e.getMessage());
dagManagementStateStore.addDagAction(dagAction);
}
}
}

public static long getDefaultJobStartDeadline(Config config) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -235,8 +235,8 @@ protected void processMessage(DecodeableKafkaRecord<byte[],byte[]> message) {
if (updatedJobStatus.getRight() == NewState.FINISHED) {
try {
this.dagManagementStateStore.addJobDagAction(flowGroup, flowName, flowExecutionId, jobName, DagActionStore.DagActionType.REEVALUATE);
} catch (Exception e) {
if (isExceptionInstanceOf(e, nonRetryableExceptions)) {
} catch (IOException e) {
if (e.getCause() != null && isThrowableInstanceOf(e.getCause(), nonRetryableExceptions)) {
// todo - add metrics
log.warn("Duplicate REEVALUATE Dag Action is being created. Ignoring... " + e.getMessage());
} else {
Expand Down Expand Up @@ -426,7 +426,7 @@ public static long getExecutionIdFromTableName(String tableName) {

protected abstract org.apache.gobblin.configuration.State parseJobStatus(GobblinTrackingEvent event);

public static boolean isExceptionInstanceOf(Exception exception, List<Class<? extends Exception>> typesList) {
public static boolean isThrowableInstanceOf(Throwable exception, List<Class<? extends Exception>> typesList) {
return typesList.stream().anyMatch(e -> e.isInstance(exception));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import java.util.ArrayList;
import java.util.List;

import org.apache.gobblin.service.modules.orchestration.task.DagProcessingEngineMetrics;
import org.apache.hadoop.fs.Path;
import org.mockito.Mockito;
import org.testng.annotations.AfterClass;
Expand Down Expand Up @@ -55,13 +54,17 @@
import org.apache.gobblin.service.modules.orchestration.DagManagerTest;
import org.apache.gobblin.service.modules.orchestration.MySqlDagManagementStateStore;
import org.apache.gobblin.service.modules.orchestration.MySqlDagManagementStateStoreTest;
import org.apache.gobblin.service.modules.orchestration.task.DagProcessingEngineMetrics;
import org.apache.gobblin.service.modules.orchestration.task.LaunchDagTask;
import org.apache.gobblin.service.modules.spec.JobExecutionPlan;
import org.apache.gobblin.service.modules.spec.JobExecutionPlanDagFactory;
import org.apache.gobblin.service.modules.utils.FlowCompilationValidationHelper;
import org.apache.gobblin.util.ConfigUtils;

import static org.mockito.ArgumentMatchers.*;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyBoolean;
import static org.mockito.ArgumentMatchers.anyLong;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doReturn;
import static org.mockito.Mockito.mock;
Expand Down Expand Up @@ -123,7 +126,7 @@ public void launchDag() throws IOException, InterruptedException, URISyntaxExcep
.forEach(sp -> Mockito.verify(sp, Mockito.never()).addSpec(any()));

Mockito.verify(this.dagManagementStateStore, Mockito.times(numOfLaunchedJobs))
.addFlowDagAction(any(), any(), anyLong(), eq(DagActionStore.DagActionType.ENFORCE_FLOW_FINISH_DEADLINE));
.addJobDagAction(any(), any(), anyLong(), eq(DagActionStore.NO_JOB_NAME_DEFAULT), eq(DagActionStore.DagActionType.ENFORCE_FLOW_FINISH_DEADLINE));
}

@Test
Expand Down
Loading