[ https://issues.apache.org/jira/browse/GOBBLIN-2193?focusedWorklogId=957255&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-957255 ]
ASF GitHub Bot logged work on GOBBLIN-2193: ------------------------------------------- Author: ASF GitHub Bot Created on: 16/Feb/25 13:30 Start Date: 16/Feb/25 13:30 Worklog Time Spent: 10m Work Description: khandelwal-prateek commented on code in PR #4096: URL: https://github.com/apache/gobblin/pull/4096#discussion_r1957304647 ########## gobblin-temporal/src/main/java/org/apache/gobblin/temporal/yarn/YarnService.java: ########## @@ -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); Review Comment: `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 ########## gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java: ########## @@ -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())) { Review Comment: 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 ########## gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java: ########## @@ -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(); Review Comment: it might be simpler and cleaner to use `CountDownLatch` instead of explicit synchronization with `synchronized, wait(), and notify()` ########## gobblin-yarn/src/main/java/org/apache/gobblin/yarn/GobblinYarnAppLauncher.java: ########## @@ -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(); + 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); Review Comment: throw exception? ########## gobblin-temporal/src/main/java/org/apache/gobblin/temporal/joblauncher/GobblinTemporalJobLauncher.java: ########## @@ -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) { Review Comment: please use `issues.isEmpty()` Issue Time Tracking ------------------- Worklog Id: (was: 957255) Time Spent: 2h 10m (was: 2h) > Fail Azkaban job on when temporal job fails > ------------------------------------------- > > Key: GOBBLIN-2193 > URL: https://issues.apache.org/jira/browse/GOBBLIN-2193 > Project: Apache Gobblin > Issue Type: Improvement > Reporter: Swapnil Palash > Assignee: Hung Tran > Priority: Major > Time Spent: 2h 10m > Remaining Estimate: 0h > > Currently when the temporal job running on Yarn fails, we don't propagate the > error back to Azkaban job which launches the Yarn Application. > The change here bubbles the issues encountered when the job fails upto the > GobblinYarnAppLaucher run by the Azkaban job and fails with a > RuntimeException after logging the issues summary. -- This message was sent by Atlassian Jira (v8.20.10#820010)