Skip to content

[GOBBLIN-2193] Fail Azkaban job on Temporal Job Failure #4096

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

Open
wants to merge 10 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all 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
@@ -0,0 +1,36 @@
/*
* 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.cluster.event;

import lombok.AllArgsConstructor;
import lombok.Getter;
import org.apache.gobblin.runtime.JobState;


/**
* The `JobSummaryEvent` class represents an event that is triggered when a job completes.
* It contains information about the job state and a summary of the issues that caused the failure.
*/
@AllArgsConstructor
public class JobSummaryEvent {
@Getter
private final JobState jobState;
@Getter
private final String issuesSummary;
}
Original file line number Diff line number Diff line change
Expand Up @@ -36,15 +36,19 @@
import io.temporal.serviceclient.WorkflowServiceStubs;
import io.temporal.workflow.Workflow;

import org.apache.commons.text.TextStringBuilder;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.slf4j.Logger;

import org.apache.gobblin.annotation.Alpha;
import org.apache.gobblin.cluster.GobblinClusterConfigurationKeys;
import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest;
import org.apache.gobblin.cluster.event.JobSummaryEvent;
import org.apache.gobblin.metrics.Tag;
import org.apache.gobblin.runtime.JobLauncher;
import org.apache.gobblin.runtime.JobState;
import org.apache.gobblin.runtime.troubleshooter.Issue;
import org.apache.gobblin.source.workunit.WorkUnit;
import org.apache.gobblin.temporal.cluster.GobblinTemporalTaskRunner;
import org.apache.gobblin.temporal.GobblinTemporalConfigurationKeys;
Expand Down Expand Up @@ -107,13 +111,40 @@ protected Config applyJobLauncherOverrides(Config config) {
return configOverrides.withFallback(config);
}

private String getIssuesSummary() {
TextStringBuilder sb = new TextStringBuilder();
try {
List<Issue> issues = this.getIssueRepository().getAll();
if (issues.size() == 0) {
Copy link
Contributor

Choose a reason for hiding this comment

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

please use issues.isEmpty()

return "";
}
sb.appendln("");
sb.appendln("vvvvv============= Issues (summary) =============vvvvv");

for (int i = 0; i < issues.size(); i++) {
Issue issue = issues.get(i);

sb.appendln("%s) %s %s %s | source: %s", i + 1, issue.getSeverity().toString(), issue.getCode(),
issue.getSummary(), issue.getSourceClass());
}
sb.append("^^^^^=============================================^^^^^");
}
catch(Exception e) {
log.warn("Failed to get issue summary", e);
}
return sb.toString();
}

@Override
protected void handleLaunchFinalization() {
// NOTE: This code only makes sense when there is 1 source / workflow being launched per application for Temporal. This is a stop-gap
// for achieving batch job behavior. Given the current constraints of yarn applications requiring a static proxy user
// during application creation, it is not possible to have multiple workflows running in the same application.
// and so it makes sense to just kill the job after this is completed
log.info("Requesting the AM to shutdown after the job {} completed", this.jobContext.getJobId());
JobState jobState = this.jobContext.getJobState();
String issuesSummary = this.getIssuesSummary();
eventBus.post(new JobSummaryEvent(jobState, issuesSummary));
eventBus.post(new ClusterManagerShutdownRequest());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@
import org.apache.gobblin.cluster.GobblinClusterMetricTagNames;
import org.apache.gobblin.cluster.GobblinClusterUtils;
import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest;
import org.apache.gobblin.cluster.event.JobSummaryEvent;
import org.apache.gobblin.metrics.GobblinMetrics;
import org.apache.gobblin.metrics.MetricReporterException;
import org.apache.gobblin.metrics.MultiReporterException;
Expand Down Expand Up @@ -198,6 +199,9 @@ class YarnService extends AbstractIdleService {
private final AtomicLong allocationRequestIdGenerator = new AtomicLong(DEFAULT_ALLOCATION_REQUEST_ID);
private final ConcurrentMap<Long, WorkerProfile> workerProfileByAllocationRequestId = new ConcurrentHashMap<>();

@Getter
protected JobSummaryEvent jobSummaryEvent;

public YarnService(Config config, String applicationName, String applicationId, YarnConfiguration yarnConfiguration,
FileSystem fs, EventBus eventBus) throws Exception {
this.applicationName = applicationName;
Expand Down Expand Up @@ -304,6 +308,12 @@ public void handleContainerReleaseRequest(ContainerReleaseRequest containerRelea
}
}

@SuppressWarnings("unused")
@Subscribe
public void handleJobFailure(JobSummaryEvent jobSummaryEvent) {
this.jobSummaryEvent = jobSummaryEvent;
}

@Override
protected synchronized void startUp() throws Exception {
LOGGER.info("Starting the TemporalYarnService");
Expand Down Expand Up @@ -353,7 +363,11 @@ protected void shutDown() throws IOException {
}
}

this.amrmClientAsync.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, null, null);
if (this.jobSummaryEvent.getJobState() != null && !this.jobSummaryEvent.getJobState().getState().isSuccess()) {
this.amrmClientAsync.unregisterApplicationMaster(FinalApplicationStatus.FAILED, this.jobSummaryEvent.getIssuesSummary(), null);
} else {
this.amrmClientAsync.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, StringUtils.defaultString(this.jobSummaryEvent.getIssuesSummary()), null);
Copy link
Contributor

Choose a reason for hiding this comment

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

this.jobSummaryEvent.getIssuesSummary() wouldn't be null, right? since getIssuesSummary() returns an empty string as default. if yes, it is fine to use StringUtils.defaultString but we should use for both statuses or not use it at all

}
} catch (IOException | YarnException e) {
LOGGER.error("Failed to unregister the ApplicationMaster", e);
} finally {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,6 +41,8 @@
import com.typesafe.config.ConfigValueFactory;
import com.google.common.eventbus.EventBus;

import org.apache.gobblin.cluster.event.JobSummaryEvent;
import org.apache.gobblin.runtime.JobState;
import org.apache.gobblin.yarn.GobblinYarnConfigurationKeys;

import static org.mockito.Mockito.*;
Expand Down Expand Up @@ -123,4 +125,26 @@ public void testBuildContainerCommand() throws Exception {
String command = yarnService.buildContainerCommand(mockContainer, "testHelixParticipantId", "testHelixInstanceTag");
Assert.assertTrue(command.contains("-Xmx" + expectedJvmMemory + "M"));
}

@Test
public void testHandleJobFailureEvent() throws Exception {
YarnService yarnService = new YarnService(
this.defaultConfigs,
"testApplicationName",
"testApplicationId",
yarnConfiguration,
mockFileSystem,
eventBus
);

yarnService.startUp();

eventBus.post(new JobSummaryEvent(new JobState("name","id"), "summary"));

// Waiting for the event to be handled
Thread.sleep(100);
Assert.assertEquals(yarnService.jobSummaryEvent.getJobState().getJobName(),"name");
Assert.assertEquals(yarnService.jobSummaryEvent.getJobState().getJobId(),"id");
Assert.assertEquals(yarnService.jobSummaryEvent.getIssuesSummary(),"summary");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,8 @@ public class GobblinYarnAppLauncher {
// This flag tells if the Yarn application has already completed. This is used to
// tell if it is necessary to send a shutdown message to the ApplicationMaster.
private volatile boolean applicationCompleted = false;
private final Object applicationDone = new Object();
private volatile boolean applicationFailed = false;

private volatile boolean stopped = false;

Expand Down Expand Up @@ -380,6 +382,22 @@ public void launch() throws IOException, YarnException, InterruptedException {
}, 0, this.appReportIntervalMinutes, TimeUnit.MINUTES);

addServices();

// The YarnClient and all the services are started asynchronously.
// This will block until the application is completed and throws an exception to fail the Azkaban Job in case the
// underlying Yarn Application reports a job failure.
synchronized (this.applicationDone) {
while (!this.applicationCompleted) {
try {
this.applicationDone.wait();
Comment on lines +389 to +392
Copy link
Contributor

Choose a reason for hiding this comment

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

it might be simpler and cleaner to use CountDownLatch instead of explicit synchronization with synchronized, wait(), and notify()

if (this.applicationFailed) {
throw new RuntimeException("Gobblin Yarn application failed");
}
} catch (InterruptedException ie) {
LOGGER.error("Interrupted while waiting for the Gobblin Yarn application to finish", ie);
Copy link
Contributor

Choose a reason for hiding this comment

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

throw exception?

}
}
}
}

public boolean isApplicationRunning() {
Expand Down Expand Up @@ -453,7 +471,6 @@ public synchronized void stop() throws IOException, TimeoutException {
this.closer.close();
}
}

this.stopped = true;
}

Expand Down Expand Up @@ -482,9 +499,17 @@ public void handleApplicationReportArrivalEvent(ApplicationReportArrivalEvent ap
LOGGER.info("Gobblin Yarn application finished with final status: " +
applicationReport.getFinalApplicationStatus().toString());
if (applicationReport.getFinalApplicationStatus() == FinalApplicationStatus.FAILED) {
LOGGER.error("Gobblin Yarn application failed for the following reason: " + applicationReport.getDiagnostics());
applicationFailed = true;
LOGGER.error("Gobblin Yarn application failed because of the following issues: " + applicationReport.getDiagnostics());
} else if (StringUtils.isNotBlank(applicationReport.getDiagnostics())) {
Copy link
Contributor

Choose a reason for hiding this comment

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

I think this should be removed as it's not useful to have diagnostics for success cases, these are mostly task failures which have already been retried

Copy link
Member

Choose a reason for hiding this comment

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

It will be useful in case where no work units were generated as in that cases job always succeeds so it will be easier to know directly.

LOGGER.error("Gobblin Yarn application succeeded but has some warning issues: " + applicationReport.getDiagnostics());
}

synchronized (this.applicationDone) {
this.applicationDone.notify();
}


try {
GobblinYarnAppLauncher.this.stop();
} catch (IOException ioe) {
Expand Down
Loading