Skip to content

Commit

Permalink
[SPARK-13172][CORE][SQL] Stop using RichException.getStackTrace it is…
Browse files Browse the repository at this point in the history
… deprecated

Replace `getStackTraceString` with `Utils.exceptionString`

Author: Sean Owen <sowen@cloudera.com>

Closes #11182 from srowen/SPARK-13172.
  • Loading branch information
srowen authored and rxin committed Feb 14, 2016
1 parent 610196f commit 388cd9e
Show file tree
Hide file tree
Showing 7 changed files with 14 additions and 12 deletions.
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

0 comments on commit 388cd9e

Please sign in to comment.