diff --git a/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala
index c2b56fe85b7fa..3e7feaf1b1dfb 100644
--- a/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala
+++ b/common/utils/src/main/scala/org/apache/spark/util/SparkErrorUtils.scala
@@ -100,12 +100,16 @@ private[spark] trait SparkErrorUtils extends Logging {
}
def stackTraceToString(t: Throwable): String = {
- val out = new java.io.ByteArrayOutputStream
- SparkErrorUtils.tryWithResource(new PrintWriter(out)) { writer =>
- t.printStackTrace(writer)
- writer.flush()
+ Option(t) match {
+ case None => ""
+ case Some(throwable) =>
+ val out = new java.io.ByteArrayOutputStream
+ SparkErrorUtils.tryWithResource(new PrintWriter(out)) { writer =>
+ throwable.printStackTrace(writer)
+ writer.flush()
+ }
+ new String(out.toByteArray, UTF_8)
}
- new String(out.toByteArray, UTF_8)
}
/**
diff --git a/scalastyle-config.xml b/scalastyle-config.xml
index 2dec5468c51b5..a12c79062993d 100644
--- a/scalastyle-config.xml
+++ b/scalastyle-config.xml
@@ -419,6 +419,11 @@ This file is divided into 3 sections:
Use getRootCause of SparkErrorUtils or Utils instead
+
+ \bExceptionUtils\.getStackTrace\b
+ Use stackTraceToString of SparkErrorUtils or Utils instead
+
+
org\.apache\.commons\.lang3\.Strings\b
Use Java String methods instead
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
index 7320c6e3918c8..f94c657bf08b7 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala
@@ -30,7 +30,6 @@ import com.google.common.collect.{Lists, Maps}
import com.google.protobuf.{Any => ProtoAny, ByteString}
import io.grpc.{Context, Status, StatusRuntimeException}
import io.grpc.stub.StreamObserver
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.spark.{SparkClassNotFoundException, SparkEnv, SparkException, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Since}
@@ -3496,7 +3495,7 @@ class SparkConnectPlanner(
.setExceptionMessage(e.toString())
.setErrorClass(e.getCondition)
- val stackTrace = Option(ExceptionUtils.getStackTrace(e))
+ val stackTrace = Option(Utils.stackTraceToString(e))
stackTrace.foreach { s =>
exception_builder.setStackTrace(s)
}
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
index 292d2eee0f159..10ab12fe2272f 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
@@ -30,7 +30,6 @@ import com.google.rpc.{Code => RPCCode, ErrorInfo, Status => RPCStatus}
import io.grpc.Status
import io.grpc.protobuf.StatusProto
import io.grpc.stub.StreamObserver
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.json4s.JsonDSL._
import org.json4s.jackson.JsonMethods
@@ -217,7 +216,7 @@ private[connect] object ErrorUtils extends Logging {
.put(errorId, st)
}
- lazy val stackTrace = Option(ExceptionUtils.getStackTrace(st))
+ lazy val stackTrace = Option(Utils.stackTraceToString(st))
val stackTraceEnabled = sessionHolderOpt.exists(
_.session.sessionState.conf.getConf(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED))
val withStackTrace =
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
index 7cc181ea6945a..347ed6ef8b4e3 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala
@@ -21,7 +21,6 @@ import java.util.{ArrayList => JArrayList, Arrays, List => JList}
import scala.jdk.CollectionConverters._
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema}
import org.apache.hadoop.hive.ql.Driver
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
@@ -35,6 +34,7 @@ import org.apache.spark.sql.classic.ClassicConversions._
import org.apache.spark.sql.execution.{QueryExecution, QueryExecutionException, SQLExecution}
import org.apache.spark.sql.execution.HiveResult.hiveResultString
import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.util.Utils
private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.sparkSession)
@@ -86,7 +86,7 @@ private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.
throw st
case cause: Throwable =>
logError(log"Failed in [${MDC(COMMAND, command)}]", cause)
- throw new QueryExecutionException(ExceptionUtils.getStackTrace(cause))
+ throw new QueryExecutionException(Utils.stackTraceToString(cause))
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
index 9389bb425492e..fbc1099307e0c 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala
@@ -29,7 +29,6 @@ import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import scala.jdk.CollectionConverters._
-import org.apache.commons.lang3.exception.ExceptionUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.hive.common.StatsSetupConst
@@ -921,7 +920,7 @@ private[hive] class HiveClientImpl(
// Wrap the original hive error with QueryExecutionException and throw it
// if there is an error in query processing.
// This works for hive 4.x and later versions.
- throw new QueryExecutionException(ExceptionUtils.getStackTrace(e))
+ throw new QueryExecutionException(Utils.stackTraceToString(e))
} finally {
closeDriver(driver)
}