aboutsummaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorscwf <wangfei1@huawei.com>2014-09-09 11:57:01 -0700
committerAndrew Or <andrewor14@gmail.com>2014-09-09 11:57:01 -0700
commit26862337c97ce14794178d6378fb4155dd24acb9 (patch)
tree67e3f89b6a64f23116246fc55fc58418eed3ca53
parentf0f1ba09b195f23f0c89af6fa040c9e01dfa8951 (diff)
downloadspark-26862337c97ce14794178d6378fb4155dd24acb9.tar.gz
spark-26862337c97ce14794178d6378fb4155dd24acb9.tar.bz2
spark-26862337c97ce14794178d6378fb4155dd24acb9.zip
[SPARK-3193]output errer info when Process exit code is not zero in test suite
https://issues.apache.org/jira/browse/SPARK-3193 I noticed that sometimes pr tests failed due to the Process exitcode != 0,refer to https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18688/consoleFull https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/19118/consoleFull [info] SparkSubmitSuite: [info] - prints usage on empty input [info] - prints usage with only --help [info] - prints error with unrecognized options [info] - handle binary specified but not class [info] - handles arguments with --key=val [info] - handles arguments to user program [info] - handles arguments to user program with name collision [info] - handles YARN cluster mode [info] - handles YARN client mode [info] - handles standalone cluster mode [info] - handles standalone client mode [info] - handles mesos client mode [info] - handles confs with flag equivalents [info] - launch simple application with spark-submit *** FAILED *** [info] org.apache.spark.SparkException: Process List(./bin/spark-submit, --class, org.apache.spark.deploy.SimpleApplicationTest, --name, testApp, --master, local, file:/tmp/1408854098404-0/testJar-1408854098404.jar) exited with code 1 [info] at org.apache.spark.util.Utils$.executeAndGetOutput(Utils.scala:872) [info] at org.apache.spark.deploy.SparkSubmitSuite.runSparkSubmit(SparkSubmitSuite.scala:311) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply$mcV$sp(SparkSubmitSuite.scala:291) [info] at org.apache.spark.deploy.SparkSubmitSuite$$anonfun$14.apply(SparkSubmitSuite.scala:284) [info] at org.apacSpark assembly has been built with Hive, including Datanucleus jars on classpath this PR output the process error info when failed, it can be helpful for diagnosis. Author: scwf <wangfei1@huawei.com> Closes #2108 from scwf/output-test-error-info and squashes the following commits: 0c48082 [scwf] minor fix according to comments 563fde1 [scwf] output errer info when Process exitcode not zero
-rw-r--r--core/src/main/scala/org/apache/spark/util/Utils.scala19
-rw-r--r--core/src/test/scala/org/apache/spark/DriverSuite.scala5
-rw-r--r--core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala2
3 files changed, 21 insertions, 5 deletions
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index 0ae28f911e..79943766d0 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -20,9 +20,11 @@ package org.apache.spark.util
import java.io._
import java.net._
import java.nio.ByteBuffer
-import java.util.{Locale, Random, UUID}
+import java.util.{Properties, Locale, Random, UUID}
import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor}
+import org.apache.log4j.PropertyConfigurator
+
import scala.collection.JavaConversions._
import scala.collection.Map
import scala.collection.mutable.ArrayBuffer
@@ -834,6 +836,7 @@ private[spark] object Utils extends Logging {
val exitCode = process.waitFor()
stdoutThread.join() // Wait for it to finish reading output
if (exitCode != 0) {
+ logError(s"Process $command exited with code $exitCode: ${output}")
throw new SparkException("Process " + command + " exited with code " + exitCode)
}
output.toString
@@ -1444,6 +1447,20 @@ private[spark] object Utils extends Logging {
}
}
+ /**
+ * config a log4j properties used for testsuite
+ */
+ def configTestLog4j(level: String): Unit = {
+ val pro = new Properties()
+ pro.put("log4j.rootLogger", s"$level, console")
+ pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender")
+ pro.put("log4j.appender.console.target", "System.err")
+ pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout")
+ pro.put("log4j.appender.console.layout.ConversionPattern",
+ "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n")
+ PropertyConfigurator.configure(pro)
+ }
+
}
/**
diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala
index a73e1ef028..4b1d280624 100644
--- a/core/src/test/scala/org/apache/spark/DriverSuite.scala
+++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala
@@ -19,9 +19,6 @@ package org.apache.spark
import java.io.File
-import org.apache.log4j.Logger
-import org.apache.log4j.Level
-
import org.scalatest.FunSuite
import org.scalatest.concurrent.Timeouts
import org.scalatest.prop.TableDrivenPropertyChecks._
@@ -54,7 +51,7 @@ class DriverSuite extends FunSuite with Timeouts {
*/
object DriverWithoutCleanup {
def main(args: Array[String]) {
- Logger.getRootLogger().setLevel(Level.WARN)
+ Utils.configTestLog4j("INFO")
val sc = new SparkContext(args(0), "DriverWithoutCleanup")
sc.parallelize(1 to 100, 4).count()
}
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 7e1ef80c84..22b369a829 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -317,6 +317,7 @@ class SparkSubmitSuite extends FunSuite with Matchers {
object JarCreationTest {
def main(args: Array[String]) {
+ Utils.configTestLog4j("INFO")
val conf = new SparkConf()
val sc = new SparkContext(conf)
val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
@@ -338,6 +339,7 @@ object JarCreationTest {
object SimpleApplicationTest {
def main(args: Array[String]) {
+ Utils.configTestLog4j("INFO")
val conf = new SparkConf()
val sc = new SparkContext(conf)
val configs = Seq("spark.master", "spark.app.name")