-
Notifications
You must be signed in to change notification settings - Fork 28.2k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-21170] [CORE] Utils.tryWithSafeFinallyAndFailureCallbacks throws IllegalArgumentException: Self-suppression not permitted #18384
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -1349,7 +1349,7 @@ private[spark] object Utils extends Logging { | |
finallyBlock | ||
} catch { | ||
case t: Throwable => | ||
if (originalThrowable != null) { | ||
if (originalThrowable != null && originalThrowable != t) { | ||
originalThrowable.addSuppressed(t) | ||
logWarning(s"Suppressing exception in finally: " + t.getMessage, t) | ||
throw originalThrowable | ||
|
@@ -1387,16 +1387,18 @@ private[spark] object Utils extends Logging { | |
catchBlock | ||
} catch { | ||
case t: Throwable => | ||
originalThrowable.addSuppressed(t) | ||
logWarning(s"Suppressing exception in catch: " + t.getMessage, t) | ||
if (originalThrowable != t) { | ||
originalThrowable.addSuppressed(t) | ||
logWarning(s"Suppressing exception in catch: " + t.getMessage, t) | ||
} | ||
} | ||
throw originalThrowable | ||
} finally { | ||
try { | ||
finallyBlock | ||
} catch { | ||
case t: Throwable => | ||
if (originalThrowable != null) { | ||
if (originalThrowable != null && originalThrowable != t) { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Really in this case and the similar one at the top you could write:
because otherwise it's just re-throwing the exception anyway rather than handling it |
||
originalThrowable.addSuppressed(t) | ||
logWarning(s"Suppressing exception in finally: " + t.getMessage, t) | ||
throw originalThrowable | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -38,7 +38,7 @@ import org.apache.commons.math3.stat.inference.ChiSquareTest | |
import org.apache.hadoop.conf.Configuration | ||
import org.apache.hadoop.fs.Path | ||
|
||
import org.apache.spark.{SparkConf, SparkFunSuite} | ||
import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext} | ||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.network.util.ByteUnit | ||
|
||
|
@@ -1024,4 +1024,89 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { | |
assert(redactedConf("spark.sensitive.property") === Utils.REDACTION_REPLACEMENT_TEXT) | ||
|
||
} | ||
|
||
test("tryWithSafeFinally") { | ||
var e = new Error("Block0") | ||
val finallyBlockError = new Error("Finally Block") | ||
var isErrorOccurred = false | ||
// if the try and finally blocks throw different exception instances | ||
try { | ||
Utils.tryWithSafeFinally { throw e }(finallyBlock = { throw finallyBlockError }) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.head == finallyBlockError) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try and finally blocks throw the same exception instance then it should not | ||
// try to add to suppressed and get IllegalArgumentException | ||
e = new Error("Block1") | ||
isErrorOccurred = false | ||
try { | ||
Utils.tryWithSafeFinally { throw e }(finallyBlock = { throw e }) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.length == 0) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try throws the exception and finally doesn't throw exception | ||
e = new Error("Block2") | ||
isErrorOccurred = false | ||
try { | ||
Utils.tryWithSafeFinally { throw e }(finallyBlock = {}) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.length == 0) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try and finally block don't throw exception | ||
Utils.tryWithSafeFinally {}(finallyBlock = {}) | ||
} | ||
|
||
test("tryWithSafeFinallyAndFailureCallbacks") { | ||
var e = new Error("Block0") | ||
val catchBlockError = new Error("Catch Block") | ||
val finallyBlockError = new Error("Finally Block") | ||
var isErrorOccurred = false | ||
TaskContext.setTaskContext(TaskContext.empty()) | ||
// if the try, catch and finally blocks throw different exception instances | ||
try { | ||
Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }(catchBlock = | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If you wouldn't mind, try to rewrap these so that you don't split this single named argument:
|
||
{ throw catchBlockError }, finallyBlock = { throw finallyBlockError }) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.head == catchBlockError) | ||
assert(t.getSuppressed.last == finallyBlockError) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try, catch and finally blocks throw the same exception instance then it should not | ||
// try to add to suppressed and get IllegalArgumentException | ||
e = new Error("Block1") | ||
isErrorOccurred = false | ||
try { | ||
Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }(catchBlock = | ||
{ throw e }, finallyBlock = { throw e }) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.length == 0) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try throws the exception, catch and finally don't throw exceptions | ||
e = new Error("Block2") | ||
isErrorOccurred = false | ||
try { | ||
Utils.tryWithSafeFinallyAndFailureCallbacks { throw e }(catchBlock = {}, finallyBlock = {}) | ||
} catch { | ||
case t: Error => | ||
assert(t.getSuppressed.length == 0) | ||
isErrorOccurred = true | ||
} | ||
assert(isErrorOccurred) | ||
// if the try, catch and finally blocks don't throw exceptions | ||
Utils.tryWithSafeFinallyAndFailureCallbacks {}(catchBlock = {}, finallyBlock = {}) | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
@devaraj-kavali this is fine to merge but I noticed an existing minor problem in the message here: should be
s"Suppressing ... ${t.getMessage}"