Skip to content
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-13172] [CORE] [SQL] Stop using RichException.getStackTrace it is deprecated #11182

Closed
wants to merge 1 commit into from
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 @@ -981,7 +981,7 @@ class DAGScheduler(
case NonFatal(e) =>
stage.makeNewStageAttempt(partitionsToCompute.size)
listenerBus.post(SparkListenerStageSubmitted(stage.latestInfo, properties))
abortStage(stage, s"Task creation failed: $e\n${e.getStackTraceString}", Some(e))
abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e))
runningStages -= stage
return
}
Expand Down Expand Up @@ -1017,7 +1017,7 @@ class DAGScheduler(
// Abort execution
return
case NonFatal(e) =>
abortStage(stage, s"Task serialization failed: $e\n${e.getStackTraceString}", Some(e))
abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e))
runningStages -= stage
return
}
Expand All @@ -1044,7 +1044,7 @@ class DAGScheduler(
}
} catch {
case NonFatal(e) =>
abortStage(stage, s"Task creation failed: $e\n${e.getStackTraceString}", Some(e))
abortStage(stage, s"Task creation failed: $e\n${Utils.exceptionString(e)}", Some(e))
runningStages -= stage
return
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -617,7 +617,7 @@ object JarCreationTest extends Logging {
Utils.classForName(args(1))
} catch {
case t: Throwable =>
exception = t + "\n" + t.getStackTraceString
exception = t + "\n" + Utils.exceptionString(t)
exception = exception.replaceAll("\n", "\n\t")
}
Option(exception).toSeq.iterator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import org.apache.spark.executor.TaskMetrics
import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.SchedulingMode.SchedulingMode
import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster}
import org.apache.spark.util.CallSite
import org.apache.spark.util.{CallSite, Utils}

class DAGSchedulerEventProcessLoopTester(dagScheduler: DAGScheduler)
extends DAGSchedulerEventProcessLoop(dagScheduler) {
Expand Down Expand Up @@ -1665,7 +1665,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with Timeou
}

// Does not include message, ONLY stack trace.
val stackTraceString = e.getStackTraceString
val stackTraceString = Utils.exceptionString(e)

// should actually include the RDD operation that invoked the method:
assert(stackTraceString.contains("org.apache.spark.rdd.RDD.count"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.optimizer.DefaultOptimizer
import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project}
import org.apache.spark.sql.types.DataType
import org.apache.spark.util.Utils

/**
* A few helper functions for expression evaluation testing. Mixin this trait to use them.
Expand Down Expand Up @@ -82,7 +83,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks {
s"""
|Code generation of $expression failed:
|$e
|${e.getStackTraceString}
|${Utils.exceptionString(e)}
""".stripMargin)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLConf}
import org.apache.spark.sql.execution.SetCommand
import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes}
import org.apache.spark.sql.types._
import org.apache.spark.util.{Utils => SparkUtils}

private[hive] class SparkExecuteStatementOperation(
parentSession: HiveSession,
Expand Down Expand Up @@ -231,7 +232,7 @@ private[hive] class SparkExecuteStatementOperation(
if (getStatus().getState() == OperationState.CANCELED) {
return
} else {
setState(OperationState.ERROR);
setState(OperationState.ERROR)
throw e
}
// Actually do need to catch Throwable as some failures don't inherit from Exception and
Expand All @@ -241,7 +242,7 @@ private[hive] class SparkExecuteStatementOperation(
logError(s"Error executing query, currentState $currentState, ", e)
setState(OperationState.ERROR)
HiveThriftServer2.listener.onStatementError(
statementId, e.getMessage, e.getStackTraceString)
statementId, e.getMessage, SparkUtils.exceptionString(e))
throw new HiveSQLException(e.toString)
}
setState(OperationState.FINISHED)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,7 +217,7 @@ object SparkSubmitClassLoaderTest extends Logging {
Utils.classForName(args(1))
} catch {
case t: Throwable =>
exception = t + "\n" + t.getStackTraceString
exception = t + "\n" + Utils.exceptionString(t)
exception = exception.replaceAll("\n", "\n\t")
}
Option(exception).toSeq.iterator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer
import scala.concurrent._
import scala.util.control.NonFatal

import org.apache.spark.{Logging, SparkConf, SparkEnv}
import org.apache.spark.{Logging, SparkConf}
import org.apache.spark.storage.StreamBlockId
import org.apache.spark.util.{ThreadUtils, Utils}

Expand Down Expand Up @@ -174,7 +174,7 @@ private[streaming] abstract class ReceiverSupervisor(
}
} catch {
case NonFatal(t) =>
logError("Error stopping receiver " + streamId + t.getStackTraceString)
logError(s"Error stopping receiver $streamId ${Utils.exceptionString(t)}")
}
}

Expand Down