Skip to content

[SPARK-37805][TESTS] Refactor TestUtils#configTestLog4j method to use log4j2 api #35095

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

Closed
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
30 changes: 16 additions & 14 deletions core/src/main/scala/org/apache/spark/TestUtils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ import java.nio.file.{Files => JavaFiles, Paths}
import java.nio.file.attribute.PosixFilePermission.{OWNER_EXECUTE, OWNER_READ, OWNER_WRITE}
import java.security.SecureRandom
import java.security.cert.X509Certificate
import java.util.{Arrays, EnumSet, Locale, Properties}
import java.util.{Arrays, EnumSet, Locale}
import java.util.concurrent.{TimeoutException, TimeUnit}
import java.util.jar.{JarEntry, JarOutputStream, Manifest}
import java.util.regex.Pattern
Expand All @@ -41,9 +41,10 @@ import scala.util.Try

import com.google.common.io.{ByteStreams, Files}
import org.apache.commons.lang3.StringUtils
// scalastyle:off
import org.apache.log4j.PropertyConfigurator
// scalastyle:on
import org.apache.logging.log4j.LogManager
import org.apache.logging.log4j.core.LoggerContext
import org.apache.logging.log4j.core.appender.ConsoleAppender
import org.apache.logging.log4j.core.config.builder.api.ConfigurationBuilderFactory
import org.eclipse.jetty.server.Handler
import org.eclipse.jetty.server.Server
import org.eclipse.jetty.server.handler.DefaultHandler
Expand Down Expand Up @@ -418,17 +419,18 @@ private[spark] object TestUtils {
}

/**
* config a log4j properties used for testsuite
* config a log4j2 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)
def configTestLog4j2(level: String): Unit = {
val builder = ConfigurationBuilderFactory.newConfigurationBuilder()
val appenderBuilder = builder.newAppender("console", "CONSOLE")
.addAttribute("target", ConsoleAppender.Target.SYSTEM_ERR)
appenderBuilder.add(builder.newLayout("PatternLayout")
.addAttribute("pattern", "%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n"))
builder.add(appenderBuilder)
builder.add(builder.newRootLogger(level).add(builder.newAppenderRef("console")))
val configuration = builder.build()
LogManager.getContext(false).asInstanceOf[LoggerContext].reconfigure(configuration)
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This rewriting looks okay. I just wonder the purpose of configTestLog4j and if it is necessary. These tests are not checking the log (as they are still passed with no-op configTestLog4j now). And seems it is not used to reduce verbose logging.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Maybe we can just remove configTestLog4j?

Copy link
Contributor Author

@LuciferYang LuciferYang Jan 5, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I traced back to the original pr (SPARK-3193), it seems that configTestLog4j is only added to help troubleshoot the failed UT on Jenkins about the Process exitcode != 0

I agree to remove this method directly,(EDIT) what do you think @dongjoon-hyun ?

Copy link
Contributor Author

@LuciferYang LuciferYang Jan 5, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm... It seems that the method of calling configTestLog4j in the current UTs is in the process of runSparkSubmit. If we directly remove this method, I can not find out where to see the relevant log of runSparkSubmit (not in target/unit-tests.log), which is really not conducive to troubleshooting.

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Then it is okay to keep it, I think.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ok ~

}

/**
Expand Down
2 changes: 1 addition & 1 deletion core/src/test/scala/org/apache/spark/DriverSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ class DriverSuite extends SparkFunSuite with TimeLimits {
*/
object DriverWithoutCleanup {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf
val sc = new SparkContext(args(0), "DriverWithoutCleanup", conf)
sc.parallelize(1 to 100, 4).count()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1520,7 +1520,7 @@ class SparkSubmitSuite

object JarCreationTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
val sc = new SparkContext(conf)
val result = sc.makeRDD(1 to 100, 10).mapPartitions { x =>
Expand All @@ -1544,7 +1544,7 @@ object JarCreationTest extends Logging {

object SimpleApplicationTest {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
val sc = new SparkContext(conf)
val configs = Seq("spark.master", "spark.app.name")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ object WholeStageCodegenSparkSubmitSuite extends Assertions with Logging {
var spark: SparkSession = _

def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

spark = SparkSession.builder().getOrCreate()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -371,7 +371,7 @@ class HiveSparkSubmitSuite

object SetMetastoreURLTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

val sparkConf = new SparkConf(loadDefaults = true)
val builder = SparkSession.builder()
Expand Down Expand Up @@ -409,7 +409,7 @@ object SetMetastoreURLTest extends Logging {

object SetWarehouseLocationTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

val sparkConf = new SparkConf(loadDefaults = true).set(UI_ENABLED, false)
val providedExpectedWarehouseLocation =
Expand Down Expand Up @@ -489,7 +489,7 @@ object SetWarehouseLocationTest extends Logging {
// can load the jar defined with the function.
object TemporaryHiveUDFTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
conf.set(UI_ENABLED, false)
val sc = new SparkContext(conf)
Expand Down Expand Up @@ -527,7 +527,7 @@ object TemporaryHiveUDFTest extends Logging {
// can load the jar defined with the function.
object PermanentHiveUDFTest1 extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
conf.set(UI_ENABLED, false)
val sc = new SparkContext(conf)
Expand Down Expand Up @@ -565,7 +565,7 @@ object PermanentHiveUDFTest1 extends Logging {
// can load the jar defined with the function.
object PermanentHiveUDFTest2 extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
conf.set(UI_ENABLED, false)
val sc = new SparkContext(conf)
Expand Down Expand Up @@ -600,7 +600,7 @@ object PermanentHiveUDFTest2 extends Logging {
// We test if we can load user jars in both driver and executors when HiveContext is used.
object SparkSubmitClassLoaderTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
val conf = new SparkConf()
val hiveWarehouseLocation = Utils.createTempDir()
conf.set(UI_ENABLED, false)
Expand Down Expand Up @@ -670,7 +670,7 @@ object SparkSubmitClassLoaderTest extends Logging {
// We test if we can correctly set spark sql configurations when HiveContext is used.
object SparkSQLConfTest extends Logging {
def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")
// We override the SparkConf to add spark.sql.hive.metastore.version and
// spark.sql.hive.metastore.jars to the beginning of the conf entry array.
// So, if metadataHive get initialized after we set spark.sql.hive.metastore.version but
Expand Down Expand Up @@ -711,7 +711,7 @@ object SPARK_9757 extends QueryTest {
protected var spark: SparkSession = _

def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

val hiveWarehouseLocation = Utils.createTempDir()
val sparkContext = new SparkContext(
Expand Down Expand Up @@ -760,7 +760,7 @@ object SPARK_11009 extends QueryTest {
protected var spark: SparkSession = _

def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

val sparkContext = new SparkContext(
new SparkConf()
Expand Down Expand Up @@ -791,7 +791,7 @@ object SPARK_14244 extends QueryTest {
protected var spark: SparkSession = _

def main(args: Array[String]): Unit = {
TestUtils.configTestLog4j("INFO")
TestUtils.configTestLog4j2("INFO")

val sparkContext = new SparkContext(
new SparkConf()
Expand Down