Skip to content

Commit 563fde1

Browse files
committed
output errer info when Process exitcode not zero
1 parent 8861cdf commit 563fde1

File tree

3 files changed

+23
-4
lines changed

3 files changed

+23
-4
lines changed

core/src/main/scala/org/apache/spark/util/Utils.scala

Lines changed: 18 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,9 +20,11 @@ package org.apache.spark.util
2020
import java.io._
2121
import java.net._
2222
import java.nio.ByteBuffer
23-
import java.util.{Locale, Random, UUID}
23+
import java.util.{Properties, Locale, Random, UUID}
2424
import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor}
2525

26+
import org.apache.log4j.PropertyConfigurator
27+
2628
import scala.collection.JavaConversions._
2729
import scala.collection.Map
2830
import scala.collection.mutable.ArrayBuffer
@@ -869,6 +871,7 @@ private[spark] object Utils extends Logging {
869871
val exitCode = process.waitFor()
870872
stdoutThread.join() // Wait for it to finish reading output
871873
if (exitCode != 0) {
874+
logError(s"Process $command exited with code $exitCode: ${output.toString}")
872875
throw new SparkException("Process " + command + " exited with code " + exitCode)
873876
}
874877
output.toString
@@ -1479,6 +1482,20 @@ private[spark] object Utils extends Logging {
14791482
}
14801483
}
14811484

1485+
/**
1486+
* config a log4j properties used for testsuite
1487+
*/
1488+
def configTestLog4j(level: String): Unit = {
1489+
val pro = new Properties()
1490+
pro.put("log4j.rootLogger", s"$level, console")
1491+
pro.put("log4j.appender.console", "org.apache.log4j.ConsoleAppender")
1492+
pro.put("log4j.appender.console.target", "System.err")
1493+
pro.put("log4j.appender.console.layout", "org.apache.log4j.PatternLayout")
1494+
pro.put("log4j.appender.console.layout.ConversionPattern",
1495+
"%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n")
1496+
PropertyConfigurator.configure(pro)
1497+
}
1498+
14821499
}
14831500

14841501
/**

core/src/test/scala/org/apache/spark/DriverSuite.scala

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -18,9 +18,9 @@
1818
package org.apache.spark
1919

2020
import java.io.File
21+
import java.util.Properties
2122

22-
import org.apache.log4j.Logger
23-
import org.apache.log4j.Level
23+
import org.apache.log4j.{PropertyConfigurator, Logger, Level}
2424

2525
import org.scalatest.FunSuite
2626
import org.scalatest.concurrent.Timeouts
@@ -54,7 +54,7 @@ class DriverSuite extends FunSuite with Timeouts {
5454
*/
5555
object DriverWithoutCleanup {
5656
def main(args: Array[String]) {
57-
Logger.getRootLogger().setLevel(Level.WARN)
57+
Utils.configTestLog4j("INFO")
5858
val sc = new SparkContext(args(0), "DriverWithoutCleanup")
5959
sc.parallelize(1 to 100, 4).count()
6060
}

core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -317,6 +317,7 @@ class SparkSubmitSuite extends FunSuite with Matchers {
317317

318318
object JarCreationTest {
319319
def main(args: Array[String]) {
320+
Utils.configTestLog4j("INFO")
320321
val conf = new SparkConf()
321322
val sc = new SparkContext(conf)
322323
val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
@@ -338,6 +339,7 @@ object JarCreationTest {
338339

339340
object SimpleApplicationTest {
340341
def main(args: Array[String]) {
342+
Utils.configTestLog4j("INFO")
341343
val conf = new SparkConf()
342344
val sc = new SparkContext(conf)
343345
val configs = Seq("spark.master", "spark.app.name")

0 commit comments

Comments
 (0)