Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-3193]output errer info when Process exit code is not zero in test suite #2108

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
19 changes: 18 additions & 1 deletion core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
}

}

/**
Expand Down
5 changes: 1 addition & 4 deletions core/src/test/scala/org/apache/spark/DriverSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -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._
Expand Down Expand Up @@ -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()
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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 =>
Expand All @@ -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")
Expand Down