-
Notifications
You must be signed in to change notification settings - Fork 751
[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
base: master
Are you sure you want to change the base?
Changes from 6 commits
b17f3dd
3bac719
ba88388
4fd615d
a449483
d898960
17d12ae
849991f
bb1a8f3
d23ef07
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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 |
---|---|---|
|
@@ -36,6 +36,9 @@ | |
import io.temporal.serviceclient.WorkflowServiceStubs; | ||
import io.temporal.workflow.Workflow; | ||
|
||
import org.apache.commons.text.TextStringBuilder; | ||
import org.apache.gobblin.cluster.event.JobSummaryEvent; | ||
import org.apache.gobblin.runtime.JobState; | ||
import org.apache.hadoop.fs.FileSystem; | ||
import org.apache.hadoop.fs.Path; | ||
import org.slf4j.Logger; | ||
|
@@ -45,6 +48,7 @@ | |
import org.apache.gobblin.cluster.event.ClusterManagerShutdownRequest; | ||
import org.apache.gobblin.metrics.Tag; | ||
import org.apache.gobblin.runtime.JobLauncher; | ||
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; | ||
|
@@ -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) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. please use |
||
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()); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -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; | ||
|
||
|
@@ -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
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. it might be simpler and cleaner to use |
||
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); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. throw exception? |
||
} | ||
} | ||
} | ||
iPalash marked this conversation as resolved.
Show resolved
Hide resolved
|
||
} | ||
|
||
public boolean isApplicationRunning() { | ||
|
@@ -453,7 +471,6 @@ public synchronized void stop() throws IOException, TimeoutException { | |
this.closer.close(); | ||
} | ||
} | ||
|
||
this.stopped = true; | ||
} | ||
|
||
|
@@ -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())) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe 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 There was a problem hiding this comment. Choose a reason for hiding this commentThe 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) { | ||
|
Uh oh!
There was an error while loading. Please reload this page.