Skip to content
Snippets Groups Projects
Commit 9afdfc94 authored by Davies Liu's avatar Davies Liu Committed by Davies Liu
Browse files

[SPARK-13286] [SQL] add the next expression of SQLException as cause

## What changes were proposed in this pull request?

Some JDBC driver (for example PostgreSQL) does not use the underlying exception as cause, but have another APIs (getNextException) to access that, so it it's included in the error logging, making us hard to find the root cause, especially in batch mode.

This PR will pull out the next exception and add it as cause (if it's different) or suppressed (if there is another different cause).

## How was this patch tested?

Can't reproduce this on the default JDBC driver, so did not add a regression test.

Author: Davies Liu <davies@databricks.com>

Closes #14722 from davies/keep_cause.
parent 97d461b7
No related branches found
No related tags found
No related merge requests found
......@@ -17,7 +17,7 @@
package org.apache.spark.sql.execution.datasources.jdbc
import java.sql.{Connection, Driver, DriverManager, PreparedStatement}
import java.sql.{Connection, Driver, DriverManager, PreparedStatement, SQLException}
import java.util.Properties
import scala.collection.JavaConverters._
......@@ -289,7 +289,7 @@ object JdbcUtils extends Logging {
}
val stmt = insertStatement(conn, table, rddSchema, dialect)
val setters: Array[JDBCValueSetter] = rddSchema.fields.map(_.dataType)
.map(makeSetter(conn, dialect, _)).toArray
.map(makeSetter(conn, dialect, _)).toArray
try {
var rowCount = 0
......@@ -322,6 +322,17 @@ object JdbcUtils extends Logging {
conn.commit()
}
committed = true
} catch {
case e: SQLException =>
val cause = e.getNextException
if (e.getCause != cause) {
if (e.getCause == null) {
e.initCause(cause)
} else {
e.addSuppressed(cause)
}
}
throw e
} finally {
if (!committed) {
// The stage must fail. We got here through an exception path, so
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment