From a18092bc1a5863994436dd88f9fd04c59706ac76 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 12 Feb 2016 10:49:35 +0000 Subject: [PATCH] Replace getStackTraceString with Utils.exceptionString --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 6 +++--- .../scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 4 ++-- .../sql/catalyst/expressions/ExpressionEvalHelper.scala | 3 ++- .../hive/thriftserver/SparkExecuteStatementOperation.scala | 5 +++-- .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 2 +- .../spark/streaming/receiver/ReceiverSupervisor.scala | 4 ++-- 7 files changed, 14 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ee0b8a1c95fd8..379dc14ad7cd1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -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 } @@ -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 } @@ -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 } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 2626f5a16dfb8..fe2c8299a0d91 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -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 diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 62972a0738211..d8849d59482e6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -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) { @@ -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")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index e028d22a54ba0..cf26d4843d84f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -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. @@ -82,7 +83,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { s""" |Code generation of $expression failed: |$e - |${e.getStackTraceString} + |${Utils.exceptionString(e)} """.stripMargin) } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index cd2167c4ecb18..8fef22cf777f6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -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, @@ -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 @@ -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) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 8932ce9503a3d..f141a9bd0ff81 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -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 diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index d0195fb14f0a3..9cde5ae080c0a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -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} @@ -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)}") } }