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 69a84a3604a5267b61279522dc2b51edd12ae626..c6eff9e4552d10e8c23020f9b561ebc056c17e56 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
@@ -869,6 +871,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
@@ -1479,6 +1482,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 a73e1ef0288a5f7b94baf8e7cc45f43577b857b0..4b1d280624c57d4a7df65f1028984126586a06a3 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 7e1ef80c8456145df3b8a8148431682b1718b479..22b369a829418d79ab9316a08aefab6fed54958c 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")