Skip to content

[SPARK-53126][SQL][CONNECT] Refactor SparkErrorUtils#stackTraceToString to handle null input and use it to replace ExceptionUtils#getStackTrace #51844

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

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The implementation of ExceptionUtils#getStackTrace is as follows:

/**
     * Gets the stack trace from a Throwable as a String, including suppressed and cause exceptions.
     *
     * <p>
     * The result of this method vary by JDK version as this method
     * uses {@link Throwable#printStackTrace(java.io.PrintWriter)}.
     * </p>
     *
     * @param throwable  the {@link Throwable} to be examined, may be null
     * @return the stack trace as generated by the exception's
     * {@code printStackTrace(PrintWriter)} method, or an empty String if {@code null} input
     */
    public static String getStackTrace(final Throwable throwable) {
        if (throwable == null) {
            return StringUtils.EMPTY;
        }
        final StringWriter sw = new StringWriter();
        throwable.printStackTrace(new PrintWriter(sw, true));
        return sw.toString();
    }

@dongjoon-hyun Do you agree to replace them with the refactored stackTraceToString?

The difference, as I see it, lies in the encoding—stackTraceToString always uses UTF-8, while getStackTrace uses the default encoding.

If this is an issue, I can add a new getStackTrace method without modifying stackTraceToString.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for this PR, @LuciferYang . Thank you.

}
new String(out.toByteArray, UTF_8)
}

/**
Expand Down
5 changes: 5 additions & 0 deletions scalastyle-config.xml
Original file line number Diff line number Diff line change
Expand Up @@ -419,6 +419,11 @@ This file is divided into 3 sections:
<customMessage>Use getRootCause of SparkErrorUtils or Utils instead</customMessage>
</check>

<check customId="commonslang3getstacktrace" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">\bExceptionUtils\.getStackTrace\b</parameter></parameters>
<customMessage>Use stackTraceToString of SparkErrorUtils or Utils instead</customMessage>
</check>

<check customId="commonslang3strings" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
<parameters><parameter name="regex">org\.apache\.commons\.lang3\.Strings\b</parameter></parameters>
<customMessage>Use Java String methods instead</customMessage>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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}
Expand Down Expand Up @@ -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)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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 =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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)
Expand Down Expand Up @@ -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))
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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)
}
Expand Down