You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2020/10/19 23:40:57 UTC

[GitHub] [spark] xkrogen opened a new pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

xkrogen opened a new pull request #30096:
URL: https://github.com/apache/spark/pull/30096


   ### What changes were proposed in this pull request?
   Currently when run in `cluster` mode on YARN, the Spark `yarn.Client` will print out the application report into the logs, to be easily viewed by users. For example:
   ```
   INFO yarn.Client: 
    	 client token: Token { kind: YARN_CLIENT_TOKEN, service:  }
    	 diagnostics: N/A
    	 ApplicationMaster host: X.X.X.X
    	 ApplicationMaster RPC port: 0
    	 queue: default
    	 start time: 1602782566027
    	 final status: UNDEFINED
    	 tracking URL: http://hostname:8888/proxy/application_<id>/
    	 user: xkrogen
   ```
   
   I propose adding, alongside the application report, some additional lines like:
   ```
            Driver Logs (stdout): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stdout?start=-4096
            Driver Logs (stderr): http://hostname:8042/node/containerlogs/container_<id>/xkrogen/stderr?start=-4096
   ```
   
   This information isn't contained in the `ApplicationReport`, so it's necessary to query the ResourceManager REST API. For now I have added this as an always-on feature, but if there is any concern about adding this REST dependency, I think hiding this feature behind an off-by-default flag is reasonable.
   
   ### Why are the changes needed?
   Typically, the tracking URL can be used to find the logs of the ApplicationMaster/driver while the application is running. Later, the Spark History Server can be used to track this information down, using the stdout/stderr links on the Executors page.
   
   However, in the situation when the driver crashed _before_ writing out a history file, the SHS may not be aware of this application, and thus does not contain links to the driver logs. When this situation arises, it can be difficult for users to debug further, since they can't easily find their driver logs.
   
   It is possible to reach the logs by using the `yarn logs` commands, but the average Spark user isn't aware of this and shouldn't have to be.
   
   With this information readily available in the logs, users can quickly jump to their driver logs, even if it crashed before the SHS became aware of the application. This has the additional benefit of providing a quick way to access driver logs, which often contain useful information, in a single click (instead of navigating through the Spark UI).
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, some additional print statements will be created in the application report when using YARN in cluster mode.
   
   ### How was this patch tested?
   Manually tested on a cluster for now. I would appreciate any guidance on where would be an appropriate place to add a unit test, if any.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722224644


   **[Test build #130643 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130643/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721995236


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35219/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722563994


   Thanks for the reviews @otterc, @tgravescs and @HeartSaVioR !
   Merging to master.
   
   Thanks for the contribution @xkrogen :-)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722485945


   **[Test build #130656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130656/testReport)** for PR 30096 at commit [`42212a8`](https://github.com/apache/spark/commit/42212a8c566776d92d9910e86aff2343055e6680).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516923948



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())
+            .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))
+      case _ =>
+        logInfo(s"Unable to fetch app attempts info from $baseRmUrl, got "

Review comment:
       Makes sense, I will upgrade to WARN.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721921508


   Jenkins, add to whitelist


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-712502179


   Can one of the admins verify this patch?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722021455






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r518180280



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1188,30 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns an empty map if the links could not be fetched. If this feature is
+   * disabled via [[CLIENT_INCLUDE_DRIVER_LOGS_LINK]], an empty map is returned immediately.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): IMap[String, String] = {
+    if (!sparkConf.get(CLIENT_INCLUDE_DRIVER_LOGS_LINK)) {
+      return IMap()
+    }
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)

Review comment:
       Sure, makes sense to me. Updated.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517670959



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
##########
@@ -189,6 +189,15 @@ package object config extends Logging {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString("1s")
 
+  private[spark] val CLIENT_REPORT_INCLUDE_DRIVER_LOGS_LINK =
+    ConfigBuilder("spark.yarn.report.includeDriverLogsLink")
+      .doc("In cluster mode, whether print include links to the driver container's logs alongside "

Review comment:
       Thanks, looks like I wrote this too fast.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516858215



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())
+            .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))
+      case _ =>
+        logInfo(s"Unable to fetch app attempts info from $baseRmUrl, got "

Review comment:
       Should this be at Info level?

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")

Review comment:
       Nit: indentation should be 2 spaces




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722031168


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35226/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-712500177


   Can one of the admins verify this patch?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721933850


   **[Test build #130615 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130615/testReport)** for PR 30096 at commit [`b6fd413`](https://github.com/apache/spark/commit/b6fd413e2ca5729df1c6490b9f0103a4ccd7dc97).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722497633


   **[Test build #130656 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130656/testReport)** for PR 30096 at commit [`42212a8`](https://github.com/apache/spark/commit/42212a8c566776d92d9910e86aff2343055e6680).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722001282


   **[Test build #130622 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130622/testReport)** for PR 30096 at commit [`dfc7a23`](https://github.com/apache/spark/commit/dfc7a23fd7ab8ae9e9e10cc30fbeb0e2d3010bf9).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722224017


   jenkins, test this please


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517607144



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1189,26 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns `None` if the links could not be fetched.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)

Review comment:
       oh never mind, we actually already use it.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721981675


   **[Test build #130618 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130618/testReport)** for PR 30096 at commit [`745c8cf`](https://github.com/apache/spark/commit/745c8cf33d12e2cd096db562085db6785e8f7b86).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517591670



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1189,26 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns `None` if the links could not be fetched.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId.toString).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {

Review comment:
       what happens when you query and the driver container hasn't started yet?

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1189,26 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns `None` if the links could not be fetched.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)

Review comment:
       even though WebAppUtils is public it has an @Private annotation on it so I don't think we should use it.

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1080,9 +1085,9 @@ private[spark] class Client(
         // If DEBUG is enabled, log report details every iteration
         // Otherwise, log them every time the application changes state
         if (log.isDebugEnabled) {
-          logDebug(formatReportDetails(report))
+          logDebug(formatReportDetails(report, getDriverLogsLink(report.getApplicationId)))
         } else if (lastState != state) {
-          logInfo(formatReportDetails(report))
+          logInfo(formatReportDetails(report, getDriverLogsLink(report.getApplicationId)))

Review comment:
       I would prefer to see this configurable as this could add quite a bit of load to the RM UI.  Also do we really need to print this every time we do the report details when the state changes?  It seems like you only need this when the driver starts, which would help with the load on RM.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722529270


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35267/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722006450






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722019455


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517624169



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1189,26 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns `None` if the links could not be fetched.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId.toString).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {

Review comment:
       This was a great question. You actually get a valid JSON back with `logsLink` empty, which results in bad behavior. Updating in the new push.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722245511


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35253/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722006438


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35219/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721958443






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721916626


   @mridulm FYI this should be ready for final review now. I added unit tests for the parsing logic. I will do another test of the updated logic on a real cluster to be sure.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721924617


   **[Test build #130615 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130615/testReport)** for PR 30096 at commit [`b6fd413`](https://github.com/apache/spark/commit/b6fd413e2ca5729df1c6490b9f0103a4ccd7dc97).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721992894


   **[Test build #130622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130622/testReport)** for PR 30096 at commit [`dfc7a23`](https://github.com/apache/spark/commit/dfc7a23fd7ab8ae9e9e10cc30fbeb0e2d3010bf9).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721973407


   New push fixes a bug when the driver container hasn't yet started (thanks Thomas) and hides the feature behind a disabled-by-default config. I would be curious if anyone has thoughts on whether enabled-by-default or disabled-by-default is more appropriate.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-712502179


   Can one of the admins verify this patch?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721958422


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35216/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721933991






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721924617


   **[Test build #130615 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130615/testReport)** for PR 30096 at commit [`b6fd413`](https://github.com/apache/spark/commit/b6fd413e2ca5729df1c6490b9f0103a4ccd7dc97).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722256808






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
mridulm commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721921611


   ok to test


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722001413






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722021455






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517647854



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1577,6 +1618,19 @@ private object Client extends Logging {
     writer.flush()
     out.closeEntry()
   }
+
+  private[yarn] def parseAppAttemptsJsonResponse(jsonString: String): Option[(String, String)] = {
+    val objectMapper = new ObjectMapper()
+    // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+    // which allows for safe continuation of chaining. The `elements()` call will be empty,
+    // and None will get returned.
+    objectMapper.readTree(jsonString)
+      .path("appAttempts").path("appAttempt")
+      .elements().asScala.toList.takeRight(1).headOption
+      .map(_.path("logsLink").asText(""))
+      .filterNot(_ == "")
+      .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))

Review comment:
       Nice, this is great! Thanks a lot for the pointer.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722256808






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722006458


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35219/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721992894


   **[Test build #130622 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130622/testReport)** for PR 30096 at commit [`dfc7a23`](https://github.com/apache/spark/commit/dfc7a23fd7ab8ae9e9e10cc30fbeb0e2d3010bf9).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] mridulm commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
mridulm commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516895583



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")

Review comment:
       Looks like there is no way to reference these from yarn project directly ?
   It does not look like `RMWSConsts` is a public api ?

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())

Review comment:
       Do we want to filter out in case there is no `logsLink` in response ?
   We will end up with invalid url if missing (and so empty).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722039594


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35226/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722021323


   **[Test build #130624 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130624/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722039609


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722497823






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722019455






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721346209


   > Is it still WIP @xkrogen ?
   
   Yes, just because I haven't looked at how to test it in an automated fashion (if that makes sense at all). Wanted to gather some feedback on the approach first. It looks like there are no fundamental issues, so I'll work on that soon.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] asfgit closed pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #30096:
URL: https://github.com/apache/spark/pull/30096


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722529295






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722235498






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722009878


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35223/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722013868


   Pushed up a new commit addressing the last config-related comments from @tgravescs, and also updated the PR description to indicate some manual live-cluster testing that was done.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r518041164



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1188,30 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns an empty map if the links could not be fetched. If this feature is
+   * disabled via [[CLIENT_INCLUDE_DRIVER_LOGS_LINK]], an empty map is returned immediately.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): IMap[String, String] = {
+    if (!sparkConf.get(CLIENT_INCLUDE_DRIVER_LOGS_LINK)) {
+      return IMap()
+    }
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)

Review comment:
       I'd safely guard with try-catch and swallow the exception with logging - IMHO this is to add additional information, which is not major functionality to let the process fail in the middle.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516925449



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")

Review comment:
       Yeah, I do not think so. Even in Hadoop itself (`LogsCLI`), this same approach is taken: https://github.com/apache/hadoop/blob/master/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/cli/LogsCLI.java#L368




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721973715


   **[Test build #130618 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130618/testReport)** for PR 30096 at commit [`745c8cf`](https://github.com/apache/spark/commit/745c8cf33d12e2cd096db562085db6785e8f7b86).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722039612


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35226/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722497823






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722013776


   **[Test build #130624 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130624/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722256778


   Kubernetes integration test status success
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35253/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517636953



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1577,6 +1618,19 @@ private object Client extends Logging {
     writer.flush()
     out.closeEntry()
   }
+
+  private[yarn] def parseAppAttemptsJsonResponse(jsonString: String): Option[(String, String)] = {
+    val objectMapper = new ObjectMapper()
+    // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+    // which allows for safe continuation of chaining. The `elements()` call will be empty,
+    // and None will get returned.
+    objectMapper.readTree(jsonString)
+      .path("appAttempts").path("appAttempt")
+      .elements().asScala.toList.takeRight(1).headOption
+      .map(_.path("logsLink").asText(""))
+      .filterNot(_ == "")
+      .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))

Review comment:
       I just remembered there is some dealing with container logs in YarnContainerInfoHelper.  while its different since not going thru rest api it might be nice to at least have constants for the "/stdout?start=-4096" and "stderr?start=-4096" so not hardcoded in multiple places




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] tgravescs commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
tgravescs commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517659582



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
##########
@@ -189,6 +189,15 @@ package object config extends Logging {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString("1s")
 
+  private[spark] val CLIENT_REPORT_INCLUDE_DRIVER_LOGS_LINK =
+    ConfigBuilder("spark.yarn.report.includeDriverLogsLink")
+      .doc("In cluster mode, whether print include links to the driver container's logs alongside "

Review comment:
       whether the client application report includes links to the driver container's logs.

##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala
##########
@@ -189,6 +189,15 @@ package object config extends Logging {
       .timeConf(TimeUnit.MILLISECONDS)
       .createWithDefaultString("1s")
 
+  private[spark] val CLIENT_REPORT_INCLUDE_DRIVER_LOGS_LINK =
+    ConfigBuilder("spark.yarn.report.includeDriverLogsLink")

Review comment:
       we are trying to not add a bunch of unneeded components to configs. how about spark.yarn.includeDriverLogsLinkInReport




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516863291



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")

Review comment:
       Thanks, will make sure to fix this and double-check any style issues when I prepare a final PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722006450


   Merged build finished. Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722001413






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722019424


   Kubernetes integration test status failure
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35223/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-712500177


   Can one of the admins verify this patch?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516863924



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())
+            .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))
+      case _ =>
+        logInfo(s"Unable to fetch app attempts info from $baseRmUrl, got "

Review comment:
       Do you think it should be WARN? I waffled on this given that it is a very non-critical code path.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721947736


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35216/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722039609






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722485945


   **[Test build #130656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130656/testReport)** for PR 30096 at commit [`42212a8`](https://github.com/apache/spark/commit/42212a8c566776d92d9910e86aff2343055e6680).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721973715


   **[Test build #130618 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130618/testReport)** for PR 30096 at commit [`745c8cf`](https://github.com/apache/spark/commit/745c8cf33d12e2cd096db562085db6785e8f7b86).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722529295






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722019464


   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/35223/
   Test FAILed.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722235498






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722013776


   **[Test build #130624 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130624/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722511846


   Kubernetes integration test starting
   URL: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder-K8s/35267/
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722224644


   **[Test build #130643 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130643/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] HeartSaVioR commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r518041164



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1188,30 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  /**
+   * Fetch links to the logs of the driver for the given application ID. This requires hitting the
+   * RM REST API. Returns an empty map if the links could not be fetched. If this feature is
+   * disabled via [[CLIENT_INCLUDE_DRIVER_LOGS_LINK]], an empty map is returned immediately.
+   */
+  private def getDriverLogsLink(appId: ApplicationId): IMap[String, String] = {
+    if (!sparkConf.get(CLIENT_INCLUDE_DRIVER_LOGS_LINK)) {
+      return IMap()
+    }
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)

Review comment:
       I'd safely guard with try-catch and swallow the exception with logging for whole this method (except if statement above) - IMHO this is to add additional information, which is not major functionality to let the process fail in the middle.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721933991






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721958443






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] otterc commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
otterc commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516891488



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())
+            .map(baseUrl => (s"$baseUrl/stdout?start=-4096", s"$baseUrl/stderr?start=-4096"))
+      case _ =>
+        logInfo(s"Unable to fetch app attempts info from $baseRmUrl, got "

Review comment:
       Yes because when I look at any errors during fetch of application report, they are logged at error level. Ideally if that succeeds this should as well. If it doesn't, I think it should be logged at warn level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721981819






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r517627254



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1080,9 +1085,9 @@ private[spark] class Client(
         // If DEBUG is enabled, log report details every iteration
         // Otherwise, log them every time the application changes state
         if (log.isDebugEnabled) {
-          logDebug(formatReportDetails(report))
+          logDebug(formatReportDetails(report, getDriverLogsLink(report.getApplicationId)))
         } else if (lastState != state) {
-          logInfo(formatReportDetails(report))
+          logInfo(formatReportDetails(report, getDriverLogsLink(report.getApplicationId)))

Review comment:
       Sure, added a configuration for this.
   
   You can't necessarily just do it a single time, since with different application attempts you can get a different driver link. Given that there is already logic in place to only print the report when the status changes, this seems like a pretty low cost. We already poll the RM API to get an `ApplicationReport` once per second (by default). By comparison, this new logic will cause the `appattempts` API to get hit just a few times over the entire lifetime of the application (`ACCEPTED`, `RUNNING`, `FINISHED`). So the relative cost is very low vs. existing logic (up to hundreds or thousands of calls to get application report vs. 3-ish calls to get driver log links).
   
   This assumes that you don't have DEBUG logging enabled. One thing we can consider is only updating the driver logs link if `lastState != state`, regardless of the log level.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] xkrogen commented on a change in pull request #30096: [SPARK-33185][YARN][WIP] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
xkrogen commented on a change in pull request #30096:
URL: https://github.com/apache/spark/pull/30096#discussion_r516925641



##########
File path: resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
##########
@@ -1172,6 +1181,31 @@ private[spark] class Client(
     }.mkString("")
   }
 
+  private def getDriverLogsLink(appId: String): Option[(String, String)] = {
+    val baseRmUrl = WebAppUtils.getRMWebAppURLWithScheme(hadoopConf)
+    val response = ClientBuilder.newClient()
+      .target(baseRmUrl)
+      .path("ws").path("v1").path("cluster").path("apps").path(appId).path("appattempts")
+      .request(MediaType.APPLICATION_JSON)
+      .get()
+    response.getStatusInfo.getFamily match {
+      case Family.SUCCESSFUL =>
+        val objectMapper = new ObjectMapper()
+        // If JSON response is malformed somewhere along the way, MissingNode will be returned,
+        // which allows for safe continuation of chaining. The `elements()` call will be empty,
+        // and None will get returned.
+        objectMapper.readTree(response.readEntity(classOf[String]))
+            .path("appAttempts").path("appAttempt")
+            .elements().asScala.toList.takeRight(1).headOption
+            .map(_.path("logsLink").asText())

Review comment:
       Yeah, good call.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] AmplabJenkins removed a comment on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-721981819






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] [spark] SparkQA commented on pull request #30096: [SPARK-33185][YARN] Set up yarn.Client to print direct links to driver stdout/stderr

Posted by GitBox <gi...@apache.org>.
SparkQA commented on pull request #30096:
URL: https://github.com/apache/spark/pull/30096#issuecomment-722235334


   **[Test build #130643 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/130643/testReport)** for PR 30096 at commit [`fd23aa0`](https://github.com/apache/spark/commit/fd23aa08f14cdd10e66c3c873041a0dfa422aee5).
    * This patch passes all tests.
    * This patch merges cleanly.
    * This patch adds no public classes.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org