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 2019/12/19 21:54:00 UTC

[GitHub] [spark] tinhto-000 opened a new pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

tinhto-000 opened a new pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955
 
 
   ### What changes were proposed in this pull request?
   1) Added missing match case to SparkUncaughtExceptionHandler, so that it would not halt the process when the exception doesn't match any of the match case statements.
   2) Added log message before halting process.  During debugging it wasn't obvious why the Worker process would DEAD (until we set SPARK_NO_DAEMONIZE=1) due to the shell-scripts puts the process into background and essentially absorbs the exit code.
   3) Added SparkUncaughtExceptionHandlerSuite.  Basically we create a Spark exception-throwing application with SparkUncaughtExceptionHandler and then check its exit code.
   
   ### Why are the changes needed?
   SPARK-30310, because the process would halt unexpectedly.
   
   ### How was this patch tested?
   All unit tests (mvn test) were ran and OK.

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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572833970
 
 
   It's know failure. I pinged @HyukjinKwon to kindly ask about taking a look (as I don't have a context on pyspark) but it seems to be better to disable the test first as multiple PRs are affected.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572834168
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21226/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363966736
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   I agree if it were the logError in the try block that triggers another exception then doing another logError won't do any good.  But for other cases where logError still works, it would provide better debugging messages.

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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572833970
 
 
   It's known failure failing consistently from yesterday. I pinged @HyukjinKwon to kindly ask about taking a look (as I don't have a context on pyspark) but it seems to be better to disable the test first as multiple PRs are affected.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r360627566
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError("Uncaught OutOfMemoryError in thread " + thread + ", process halted.", oom)
 
 Review comment:
   thanks for the suggestion!  modded accordingly.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572832264
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364059159
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,24 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        try { logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom) }
 
 Review comment:
   I actually agree!  Will modify.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572832270
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116422/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572833832
 
 
   **[Test build #116438 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116438/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569186598
 
 
   cc. @gatorsmile @zsxwing 

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361576474
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import scala.util.Try
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        exitOnUncaughtException.toString),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+}
+
+// a thread that uses SparkUncaughtExceptionHandler, then throws the throwable
+class ThrowableThrowerThread(t: Throwable,
+    exitOnUncaughtException: Boolean) extends Thread {
+  override def run() {
+    Thread.setDefaultUncaughtExceptionHandler(
+      new SparkUncaughtExceptionHandler(exitOnUncaughtException))
+    throw t
+  }
+}
+
+// Objects to be invoked by spark-class for different Throwable types
+// that SparkUncaughtExceptionHandler handles.  spark-class will exit with
+// exit code dictated by either:
+// - SparkUncaughtExceptionHandler (SparkExitCode)
+// - main() (0, or -1 when args is empty)
+
+object RuntimeExceptionThrower {
 
 Review comment:
   All objects have same implementation except which exception to create and pass. Why not consolidating into one object, and additional argument to differentiate the exception? Define a new Enumeration for this and pass it as String for additional argument.

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572832633
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796551
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572857441
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573777352
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572896254
 
 
   **[Test build #116454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116454/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569108243
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364004133
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   OK, I get it, though it is a corner case on a corner case - fatal error, then logging throws a fatal error. It might be overkill to further wrap it, but, I am not against it for complete defensiveness.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363534336
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Please take a look at the content of try statement - if you don't have any log message, then logError in try statement would have been failed already. I don't think we can succeed logging here and below in Throwable.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573689500
 
 
   **[Test build #116645 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116645/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572832264
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573687924
 
 
   Jenkins, 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573777366
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116645/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-568752678
 
 
   **[Test build #4975 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4975/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572795652
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572832270
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116422/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572896254
 
 
   **[Test build #116454 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116454/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573690109
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21424/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081508
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569108247
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115817/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572919917
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116454/
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361578489
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   According to the content of try statement as of now, I guess the case we catch some exception here might probably be the case which Spark even fails to log. We may be better to do nothing but just halt.

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-575679461
 
 
   Merged to master

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363993038
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Thanks for the comment!  How about something like this?  Consider if we get there we don't really care why logError fails.
   
   ```
           try { logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom) }
           catch { case _: Throwable => }
           Runtime.getRuntime.halt(SparkExitCode.OOM)
   ```
   
   

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081508
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572919648
 
 
   **[Test build #116454 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116454/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).
    * This patch **fails due to an unknown error code, -9**.
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-568769027
 
 
   **[Test build #4975 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4975/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).
    * This patch **fails Spark unit 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572857449
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116438/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572919904
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572923064
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572794484
 
 
   @srowen Would you mind if I ask about revisiting this? Maybe with "ok to test". Thanks in advance!

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r360627627
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        if (exitOnUncaughtException) "true" else "false"),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+}
+
+// a thread that uses SparkUncaughtExceptionHandler, then throws the throwable
+class ThrowableThrowerThread(t: Throwable,
+    exitOnUncaughtException: Boolean) extends Thread {
+  override def run() {
+    Thread.setDefaultUncaughtExceptionHandler(
+      new SparkUncaughtExceptionHandler(exitOnUncaughtException))
+    throw t
+  }
+}
+
+// Objects to be invoked by spark-class for different Throwable types
+// that SparkUncaughtExceptionHandler handles.  spark-class will exit with
+// exit code dictated by either SparkUncaughtExceptionHandler (SparkExitCode)
+// or main() (0)
+
+object RuntimeExceptionThrower {
+  def main(args: Array[String]): Unit = {
+    val t = new ThrowableThrowerThread(new RuntimeException, if (args(0) == "true") true else false)
+    t.start()
+    t.join()
+    System.exit(0)
+  }
+}
+
+object OutOfMemoryErrorThrower {
+  def main(args: Array[String]): Unit = {
+    val t = new ThrowableThrowerThread(new OutOfMemoryError, if (args(0) == "true") true else false)
 
 Review comment:
   thanks for the suggestion!  modded the throwers to check args.length and exit with -1 if it's empty now.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572834168
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21226/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572833832
 
 
   **[Test build #116438 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116438/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572838933
 
 
   Oh OK. What about disabling the test and file an issue then? I've submitted #27158 to disable the 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572857441
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361578489
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   According to the content of try statement as of now (it tries to log the exception first), I guess the case we catch some exception here might probably be the case which Spark even fails to log (except the case try statement has a bug like SPARK-30310 discovered...), and trying to log may throw another exception which we don't catch here. We may be better to do nothing but just halt.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364056277
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,24 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        try { logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom) }
 
 Review comment:
   Thanks for the update! It's a nit, but maybe better to be consistent on style:
   
   ```
   try {
   ...
   } catch {
   ...
   }
   ```
   
   as one-liner of try statement and multi-lines or catch statement look inconsistent.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-568752678
 
 
   **[Test build #4975 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4975/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573777352
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572834161
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569946709
 
 
   Ping @tinhto-000 

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363769350
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Why would logging not succeed here?
   This still halts. I think it avoids masking the OOM with a MatchError.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573689500
 
 
   **[Test build #116645 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116645/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364773098
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,28 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        try {
+          logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
+        }
+        catch {
 
 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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572838933
 
 
   Oh OK. Never mind about missing ping. What about disabling the test and file an issue then? I've submitted #27158 to disable the 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361578489
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   According to the content of try statement as of now (it tries to log the exception first), I guess the case we catch some exception here might probably be the case which Spark even fails to log, and trying to log may throw another exception which we don't catch here. We may be better to do nothing but just halt.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363790772
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Thanks for the comment.
   
   But the `ShutdownHookManager.inShutdown()` call in the try block could also trigger another exception though, at least in theory?  In that case I believe the logError in Throwable would log it OK.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569108247
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/115817/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081514
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20609/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572857268
 
 
   **[Test build #116438 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116438/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).
    * This patch **fails PySpark unit 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572894712
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21241/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r360627600
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        if (exitOnUncaughtException) "true" else "false"),
 
 Review comment:
   thanks for the suggestion!  modded accordingly.
   also changed to use .toBoolean for converting string to boolean.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573690097
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361577374
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import scala.util.Try
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        exitOnUncaughtException.toString),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
 
 Review comment:
   And then getThrowerProcess can be inlined as it's only used in single place.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081308
 
 
   **[Test build #115817 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115817/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569080925
 
 
   Jenkins 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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572834161
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572894705
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796563
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21211/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081514
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/20609/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363534852
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   So in that case, well, I guess exit code would be only the hint.

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


With regards,
Apache Git Services

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


[GitHub] [spark] srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364764285
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,28 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        try {
+          logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
+        }
+        catch {
 
 Review comment:
   Nit: pull the catch up onto the previous line, here and below

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796117
 
 
   **[Test build #116422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116422/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572894705
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572894712
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21241/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
srowen closed pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955
 
 
   

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573690097
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364045750
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Thanks @srowen and @HeartSaVioR, I'll make that into the next 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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572894310
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569081308
 
 
   **[Test build #115817 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115817/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363790772
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Thanks for the comment.
   
   But the `ShutdownHookManager.inShutdown()` call in the try block could also trigger another exception though, at least in theory?
   
   But yes I agree, if the logError triggers it in the try then doing another logError in the catch won't do any good.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569107803
 
 
   **[Test build #115817 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/115817/testReport)** for PR 26955 at commit [`724e254`](https://github.com/apache/spark/commit/724e2540a5be3c8f8067f3d3db6a21167ad97fdf).
    * 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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363963250
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import scala.util.Try
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        exitOnUncaughtException.toString),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
 
 Review comment:
   Thanks for the suggestion!   Modified as such.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796551
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-569108243
 
 
   Merged build finished. Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572831975
 
 
   **[Test build #116422 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116422/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).
    * This patch **fails PySpark unit 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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796117
 
 
   **[Test build #116422 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116422/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-567701465
 
 
   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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572857449
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116438/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572796563
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21211/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361578489
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   According to the content of try statement as of now, I guess the case we catch some exception here might probably be the case which Spark even fails to log, and trying to log may throw another exception which we don't catch here. We may be better to do nothing but just halt.

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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363963188
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import scala.util.Try
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        exitOnUncaughtException.toString),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(RuntimeExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught OutOfMemoryError, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(OutOfMemoryErrorThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.UNCAUGHT_EXCEPTION)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException, exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionThrower, exitOnUncaughtException = false)
+    assert(process.waitFor == 0)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = true") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = true)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+  test("SPARK-30310: Test uncaught SparkFatalException (OOM), exitOnUncaughtException = false") {
+    val process = getThrowerProcess(SparkFatalExceptionWithOOMThrower,
+      exitOnUncaughtException = false)
+    assert(process.waitFor == SparkExitCode.OOM)
+  }
+
+}
+
+// a thread that uses SparkUncaughtExceptionHandler, then throws the throwable
+class ThrowableThrowerThread(t: Throwable,
+    exitOnUncaughtException: Boolean) extends Thread {
+  override def run() {
+    Thread.setDefaultUncaughtExceptionHandler(
+      new SparkUncaughtExceptionHandler(exitOnUncaughtException))
+    throw t
+  }
+}
+
+// Objects to be invoked by spark-class for different Throwable types
+// that SparkUncaughtExceptionHandler handles.  spark-class will exit with
+// exit code dictated by either:
+// - SparkUncaughtExceptionHandler (SparkExitCode)
+// - main() (0, or -1 when args is empty)
+
+object RuntimeExceptionThrower {
 
 Review comment:
   Thanks for the suggestion!   Modified as such.

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


With regards,
Apache Git Services

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


[GitHub] [spark] HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR edited a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572833970
 
 
   It's known failure. I pinged @HyukjinKwon to kindly ask about taking a look (as I don't have a context on pyspark) but it seems to be better to disable the test first as multiple PRs are affected.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572919904
 
 
   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


With regards,
Apache Git Services

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


[GitHub] [spark] SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
SparkQA commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573776541
 
 
   **[Test build #116645 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/116645/testReport)** for PR 26955 at commit [`3f9e96f`](https://github.com/apache/spark/commit/3f9e96f6e95c9375920a6b9e8e85c03e88b5cb7b).
    * 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361577326
 
 

 ##########
 File path: core/src/test/scala/org/apache/spark/util/SparkUncaughtExceptionHandlerSuite.scala
 ##########
 @@ -0,0 +1,156 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util
+
+import java.io.File
+
+import scala.util.Try
+
+import org.apache.spark.SparkFunSuite
+
+class SparkUncaughtExceptionHandlerSuite extends SparkFunSuite {
+
+  private val sparkHome =
+    sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!"))
+
+  // creates a spark-class process that invokes the exception thrower
+  // the testcases will detect the process's exit code
+  def getThrowerProcess(exceptionThrower: Any, exitOnUncaughtException: Boolean): Process = {
+    Utils.executeCommand(
+      Seq(s"$sparkHome/bin/spark-class",
+        exceptionThrower.getClass.getCanonicalName.dropRight(1), // drops the "$" at the end
+        exitOnUncaughtException.toString),
+      new File(sparkHome),
+      Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome))
+  }
+
+  test("SPARK-30310: Test uncaught RuntimeException, exitOnUncaughtException = true") {
 
 Review comment:
   I guess Spark codebase has been dealing with this kind of case via having Seq/Map to construct test matrix, and leverage `foreach` to construct each test. Like:
   
   ```
   Seq(
     ("RuntimeException", RuntimeExceptionThrower, true, SparkExitCode.UNCAUGHT_EXCEPTION),
     ...
   ).foreach { case (excKind, thrower, exitOnUncaughtException, expectedExitCode) =>
     test(s"SPARK-30310: Test uncaught $excKind, exitOnUncaughtException = $exitOnUncaughtException") {
       val process = getThrowerProcess(thrower, exitOnUncaughtException = exitOnUncaughtException)
       assert(process.waitFor == expectedExitCode)
     }
   }
   ```
   
   Having Enumeration makes this even simpler, as `excKind` and `thrower` would be consolidated into one.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572919917
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116454/
   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


With regards,
Apache Git Services

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


[GitHub] [spark] tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
tinhto-000 commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363475923
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Thanks for the comment.  
   
   Well the reason why for the logError is because it wasn't obvious to users or devs why the worker would just disappeared as DEAD on the UI, and there was nothing in the worker log file to tell what happened.  We couldn't find out why until we set SPARK_NO_DAEMONIZE=1 and examined the exit code.
   
   Is there any alternative to indicate the process halted unexpectedly?

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r361578489
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   According to the content of try statement as of now (it try to log the exception first), I guess the case we catch some exception here might probably be the case which Spark even fails to log, and trying to log may throw another exception which we don't catch here. We may be better to do nothing but just halt.

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


With regards,
Apache Git Services

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


[GitHub] [spark] HyukjinKwon commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HyukjinKwon commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-572838511
 
 
   Sorry I missed pings @HeartSaVioR - I am currently stuck at some works :(.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins removed a comment on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573690109
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder-K8s/21424/
   Test PASSed.

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


With regards,
Apache Git Services

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


[GitHub] [spark] AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on issue #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#issuecomment-573777366
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/116645/
   Test PASSed.

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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r364006784
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   I'd support wrapping it, as here we control the value of return code from 50 to 52 and doesn't sound ideal to have some case breaking 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


With regards,
Apache Git Services

---------------------------------------------------------------------
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 #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests

Posted by GitBox <gi...@apache.org>.
HeartSaVioR commented on a change in pull request #26955: [SPARK-30310] [Core] Resolve missing match case in SparkUncaughtExceptionHandler and added tests
URL: https://github.com/apache/spark/pull/26955#discussion_r363987915
 
 

 ##########
 File path: core/src/main/scala/org/apache/spark/util/SparkUncaughtExceptionHandler.scala
 ##########
 @@ -48,11 +48,17 @@ private[spark] class SparkUncaughtExceptionHandler(val exitOnUncaughtException:
             System.exit(SparkExitCode.OOM)
           case _ if exitOnUncaughtException =>
             System.exit(SparkExitCode.UNCAUGHT_EXCEPTION)
+          case _ =>
+            // SPARK-30310: Don't System.exit() when exitOnUncaughtException is false
         }
       }
     } catch {
-      case oom: OutOfMemoryError => Runtime.getRuntime.halt(SparkExitCode.OOM)
-      case t: Throwable => Runtime.getRuntime.halt(SparkExitCode.UNCAUGHT_EXCEPTION_TWICE)
+      case oom: OutOfMemoryError =>
+        logError(s"Uncaught OutOfMemoryError in thread $thread, process halted.", oom)
 
 Review comment:
   Oh OK. I agree ShutdownHookManager.inShutdown() could be...
   
   My point is that logging here could defeat the effort of catch statement, as if logError throws another exception, the return code becomes out of control. If we really want to log it here, we may want to wrap with try-catch statement again.

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


With regards,
Apache Git Services

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