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/09/04 03:04:27 UTC

[GitHub] [spark] KevinSmile opened a new pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

KevinSmile opened a new pull request #29644:
URL: https://github.com/apache/spark/pull/29644


   ### What changes were proposed in this pull request?
   fix  [SPARK-32598] (missing driver logs in UI Executors tab in standalone mode) by solving 2 more general problems:
   1.  Currently, a driver doesn't know its own driverId.
   2. Currently, a driver doesn't know its worker info.
   
   ### Why are the changes needed?
   1. fix bug  [SPARK-32598].
   2. Solving the two more general problems mentioned above may help future changes.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. User will see driver logs in UI-ApplicationDetails-Executors tab.
   
   ### How was this patch tested?
   Re-check [SPARK-32598] and found this user-facing bug fixed.


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

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


   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 commented on pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

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


   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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   > Can you do some manual tests and paste screenshots of UI to show the fix result in the PR description?
   
   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] AmplabJenkins removed a comment on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


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


----------------------------------------------------------------
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] srowen commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   Merged to master/3.1/3.0. It conflicted in 2.4; if we need it there, it needs another 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 commented on pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

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


   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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   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] Ngone51 commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
##########
@@ -1400,19 +1400,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
         ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics,
         null))
 
-      val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info)
-      esummary.foreach { execSummary =>
-        assert(execSummary.failedTasks === 1)
-        assert(execSummary.succeededTasks === 1)
-        assert(execSummary.killedTasks === 0)
+      val allExecutorStageSummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_
+        .info)
+      allExecutorStageSummary.foreach { executorStageSummary =>
+        assert(executorStageSummary.failedTasks === 1)
+        assert(executorStageSummary.succeededTasks === 1)
+        assert(executorStageSummary.killedTasks === 0)
       }
 
       val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info)
       assert(allExecutorSummary.size === 2)
-      allExecutorSummary.foreach { allExecSummary =>
-        assert(allExecSummary.failedTasks === 1)
-        assert(allExecSummary.activeTasks === 0)
-        assert(allExecSummary.completedTasks === 1)
+      allExecutorSummary.foreach { executorSummary =>
+        assert(executorSummary.failedTasks === 1)
+        assert(executorSummary.activeTasks === 0)
+        assert(executorSummary.completedTasks === 1)

Review comment:
       Why the test changed?  Looks like it's only a renaming change.




----------------------------------------------------------------
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] Ngone51 commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   LGTM. I also verified locally. It looks good.
   
   @jiangxb1987 Could you also take a look?


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   @HyukjinKwon Can you help let Jenkins test this?


----------------------------------------------------------------
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] KevinSmile commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,20 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  private def extractDriverLogUrls(): Map[String, String] = {

Review comment:
       ok, done

##########
File path: core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
##########
@@ -1400,19 +1400,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
         ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics,
         null))
 
-      val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info)
-      esummary.foreach { execSummary =>
-        assert(execSummary.failedTasks === 1)
-        assert(execSummary.succeededTasks === 1)
-        assert(execSummary.killedTasks === 0)
+      val allExecutorStageSummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_
+        .info)
+      allExecutorStageSummary.foreach { executorStageSummary =>
+        assert(executorStageSummary.failedTasks === 1)
+        assert(executorStageSummary.succeededTasks === 1)
+        assert(executorStageSummary.killedTasks === 0)
       }
 
       val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info)
       assert(allExecutorSummary.size === 2)
-      allExecutorSummary.foreach { allExecSummary =>
-        assert(allExecSummary.failedTasks === 1)
-        assert(allExecSummary.activeTasks === 0)
-        assert(allExecSummary.completedTasks === 1)
+      allExecutorSummary.foreach { executorSummary =>
+        assert(executorSummary.failedTasks === 1)
+        assert(executorSummary.activeTasks === 0)
+        assert(executorSummary.completedTasks === 1)

Review comment:
       Oh It's the first commit which fix confusing names nit, and is unrelated to this issue. I have reverted this commit.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,17 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  override def getDriverLogUrls: Option[Map[String, String]] = {
+    sys.env.get("SPARK_DRIVER_LOG_URL_STDOUT") match {
+      case Some(_) =>
+        val prefix = "SPARK_DRIVER_LOG_URL_"
+        Some(sys.env.filterKeys(_.startsWith(prefix))
+          .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap)
+      case None => // client mode
+        None
+    }

Review comment:
       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] KevinSmile commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,17 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  override def getDriverLogUrls: Option[Map[String, String]] = {
+    sys.env.get("SPARK_DRIVER_LOG_URL_STDOUT") match {
+      case Some(_) =>
+        val prefix = "SPARK_DRIVER_LOG_URL_"
+        Some(sys.env.filterKeys(_.startsWith(prefix))
+          .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap)
+      case None => // client mode
+        None
+    }

Review comment:
       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] KevinSmile commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
##########
@@ -1400,19 +1400,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
         ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics,
         null))
 
-      val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info)
-      esummary.foreach { execSummary =>
-        assert(execSummary.failedTasks === 1)
-        assert(execSummary.succeededTasks === 1)
-        assert(execSummary.killedTasks === 0)
+      val allExecutorStageSummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_
+        .info)
+      allExecutorStageSummary.foreach { executorStageSummary =>
+        assert(executorStageSummary.failedTasks === 1)
+        assert(executorStageSummary.succeededTasks === 1)
+        assert(executorStageSummary.killedTasks === 0)
       }
 
       val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info)
       assert(allExecutorSummary.size === 2)
-      allExecutorSummary.foreach { allExecSummary =>
-        assert(allExecSummary.failedTasks === 1)
-        assert(allExecSummary.activeTasks === 0)
-        assert(allExecSummary.completedTasks === 1)
+      allExecutorSummary.foreach { executorSummary =>
+        assert(executorSummary.failedTasks === 1)
+        assert(executorSummary.activeTasks === 0)
+        assert(executorSummary.completedTasks === 1)

Review comment:
       Oh It's the first commit which fix confusing names nit, and is unrelated to this issue. I have reverted this commit.




----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #129656 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129656/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).
    * 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 commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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






----------------------------------------------------------------
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] Ngone51 commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,20 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  private def extractDriverLogUrls(): Map[String, String] = {

Review comment:
       Shall we inline this function to the `getDriverLogUrls`?

##########
File path: core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
##########
@@ -1400,19 +1400,20 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
         ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), new ExecutorMetrics,
         null))
 
-      val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info)
-      esummary.foreach { execSummary =>
-        assert(execSummary.failedTasks === 1)
-        assert(execSummary.succeededTasks === 1)
-        assert(execSummary.killedTasks === 0)
+      val allExecutorStageSummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_
+        .info)
+      allExecutorStageSummary.foreach { executorStageSummary =>
+        assert(executorStageSummary.failedTasks === 1)
+        assert(executorStageSummary.succeededTasks === 1)
+        assert(executorStageSummary.killedTasks === 0)
       }
 
       val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info)
       assert(allExecutorSummary.size === 2)
-      allExecutorSummary.foreach { allExecSummary =>
-        assert(allExecSummary.failedTasks === 1)
-        assert(allExecSummary.activeTasks === 0)
-        assert(allExecSummary.completedTasks === 1)
+      allExecutorSummary.foreach { executorSummary =>
+        assert(executorSummary.failedTasks === 1)
+        assert(executorSummary.activeTasks === 0)
+        assert(executorSummary.completedTasks === 1)

Review comment:
       Why the test changed?  Looks like it's only a renaming change.

##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,17 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  override def getDriverLogUrls: Option[Map[String, String]] = {
+    sys.env.get("SPARK_DRIVER_LOG_URL_STDOUT") match {
+      case Some(_) =>
+        val prefix = "SPARK_DRIVER_LOG_URL_"
+        Some(sys.env.filterKeys(_.startsWith(prefix))
+          .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap)
+      case None => // client mode
+        None
+    }

Review comment:
       How about:
   ```suggestion
      val prefix = "SPARK_DRIVER_LOG_URL_"
       val logUrls = sys.env.filterKeys(_.startsWith(prefix))
             .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap
       if (logUrls.nonEmpty) Some(logUrls) else None
   ```




----------------------------------------------------------------
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] srowen commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone mode

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


   In standalone mode aren't these just the logs from the application driver, the stdout output? I don't know this part well (and not enough to review)


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

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


   Direct bug reason: 
   the original author forgot to implement `getDriverLogUrls` in `StandaloneSchedulerBackend`
   
   https://github.com/apache/spark/blob/1de272f98d0ff22d0dd151797f22b8faf310963a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala#L71-L75


----------------------------------------------------------------
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] KevinSmile edited a comment on pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

Posted by GitBox <gi...@apache.org>.
KevinSmile edited a comment on pull request #29644:
URL: https://github.com/apache/spark/pull/29644#issuecomment-686875689


   Direct bug reason: 
   the original author forgot to implement `getDriverLogUrls` in `StandaloneSchedulerBackend`
   
   https://github.com/apache/spark/blob/1de272f98d0ff22d0dd151797f22b8faf310963a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala#L70-L75


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #134103 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134103/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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






----------------------------------------------------------------
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] srowen closed pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   


----------------------------------------------------------------
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] Ngone51 commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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






----------------------------------------------------------------
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] HyukjinKwon commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   retest 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] SparkQA commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #129656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129656/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone mode

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


   Yes, it's just stdout & stderr of driver, currently missing in web-ui. Thanks anyway! @srowen 


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #134103 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134103/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #129656 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/129656/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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






----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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






----------------------------------------------------------------
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] Ngone51 commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   Can you do some manual tests and paste screenshots of UI to show the fix result in the PR description?


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone mode

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


   @srowen  Can you also help find someone review this PR? Thanks!


----------------------------------------------------------------
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] Ngone51 commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,17 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  override def getDriverLogUrls: Option[Map[String, String]] = {
+    sys.env.get("SPARK_DRIVER_LOG_URL_STDOUT") match {
+      case Some(_) =>
+        val prefix = "SPARK_DRIVER_LOG_URL_"
+        Some(sys.env.filterKeys(_.startsWith(prefix))
+          .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap)
+      case None => // client mode
+        None
+    }

Review comment:
       How about:
   ```suggestion
      val prefix = "SPARK_DRIVER_LOG_URL_"
       val logUrls = sys.env.filterKeys(_.startsWith(prefix))
             .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)).toMap
       if (logUrls.nonEmpty) Some(logUrls) else None
   ```




----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


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


----------------------------------------------------------------
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] KevinSmile removed a comment on pull request #29644: [SPARK-32598][Scheduler] fix missing driver logs in UI Executors tab in standalone mode

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


   Direct bug reason: 
   the original author forgot to implement `getDriverLogUrls` in `StandaloneSchedulerBackend`
   
   https://github.com/apache/spark/blob/1de272f98d0ff22d0dd151797f22b8faf310963a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala#L70-L75


----------------------------------------------------------------
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] Ngone51 commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,20 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  private def extractDriverLogUrls(): Map[String, String] = {

Review comment:
       Shall we inline this function to the `getDriverLogUrls`?




----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   **[Test build #134103 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/134103/testReport)** for PR 29644 at commit [`a0bdce9`](https://github.com/apache/spark/commit/a0bdce946d8d5df9a9dcf480e4a7a00ff43a0335).
    * 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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone mode

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


   @zsxwing @Ngone51 Do you have time to review this? Thanks!


----------------------------------------------------------------
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] KevinSmile commented on a change in pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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



##########
File path: core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala
##########
@@ -231,6 +232,20 @@ private[spark] class StandaloneSchedulerBackend(
     }
   }
 
+  private def extractDriverLogUrls(): Map[String, String] = {

Review comment:
       ok, 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] KevinSmile edited a comment on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

Posted by GitBox <gi...@apache.org>.
KevinSmile edited a comment on pull request #29644:
URL: https://github.com/apache/spark/pull/29644#issuecomment-692623887


   @Ngone51 Thanks for your suggestion! Yes you're right, I have updated this patch. I was thinking of using RPC to get full worker info may help further changes, seems no need.
   Also I find it hard to have a standalone cluster mode in suites, and this patch just add some envs, so I didn't add ut here. 


----------------------------------------------------------------
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 #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


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


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   > Can you do some manual tests and paste screenshots of UI to show the fix result in the PR description?
   
   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] AmplabJenkins commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


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


----------------------------------------------------------------
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] KevinSmile commented on pull request #29644: [SPARK-32598][Scheduler] Fix missing driver logs under UI App-Executors tab in standalone cluster mode

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


   @Ngone51 Thanks for your suggestion! Yes you're right, I have updated this patch. I was thinking of using RPC to get full worker info may help further changes, seems no need.
   


----------------------------------------------------------------
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