Skip to content
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
Original file line number Diff line number Diff line change
Expand Up @@ -16,30 +16,33 @@
*/
package org.apache.spark.util

import org.apache.logging.log4j.Level
import org.scalatest.BeforeAndAfterAll

import org.apache.spark.internal.Logging

class PatternLoggingSuite extends LoggingSuiteBase with BeforeAndAfterAll {

override protected def logFilePath: String = "target/pattern.log"
override def className: String = classOf[PatternLoggingSuite].getSimpleName
override def logFilePath: String = "target/pattern.log"

override def beforeAll(): Unit = Logging.disableStructuredLogging()

override def afterAll(): Unit = Logging.enableStructuredLogging()

override def expectedPatternForBasicMsg(level: String): String =
s""".*$level PatternLoggingSuite: This is a log message\n"""
Copy link
Contributor Author

@panbingkun panbingkun Mar 31, 2024

Choose a reason for hiding this comment

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

Let's make full use of the variable className to eliminate hard coding text PatternLoggingSuite

override def expectedPatternForBasicMsg(level: Level): String = {
s""".*$level $className: This is a log message\n"""
}

override def expectedPatternForMsgWithMDC(level: String): String =
s""".*$level PatternLoggingSuite: Lost executor 1.\n"""
override def expectedPatternForMsgWithMDC(level: Level): String =
s""".*$level $className: Lost executor 1.\n"""

override def expectedPatternForMsgWithMDCAndException(level: String): String =
s""".*$level PatternLoggingSuite: Error in executor 1.\njava.lang.RuntimeException: OOM\n.*"""
override def expectedPatternForMsgWithMDCAndException(level: Level): String =
s""".*$level $className: Error in executor 1.\njava.lang.RuntimeException: OOM\n.*"""

override def verifyMsgWithConcat(level: String, logOutput: String): Unit = {
override def verifyMsgWithConcat(level: Level, logOutput: String): Unit = {
val pattern =
s""".*$level PatternLoggingSuite: Min Size: 2, Max Size: 4. Please double check.\n"""
s""".*$level $className: Min Size: 2, Max Size: 4. Please double check.\n"""
assert(pattern.r.matches(logOutput))
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,23 +19,28 @@ package org.apache.spark.util
import java.io.File
import java.nio.file.Files

import com.fasterxml.jackson.databind.ObjectMapper
import com.fasterxml.jackson.module.scala.DefaultScalaModule
import org.apache.logging.log4j.Level
import org.scalatest.funsuite.AnyFunSuite // scalastyle:ignore funsuite

import org.apache.spark.internal.{LogEntry, Logging, MDC}
import org.apache.spark.internal.LogKey.{EXECUTOR_ID, MAX_SIZE, MIN_SIZE}

abstract class LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuite
with Logging {
trait LoggingSuiteBase
extends AnyFunSuite // scalastyle:ignore funsuite
with Logging {

protected def logFilePath: String
def className: String
def logFilePath: String

protected lazy val logFile: File = {
private lazy val logFile: File = {
val pwd = new File(".").getCanonicalPath
new File(pwd + "/" + logFilePath)
}

// Returns the first line in the log file that contains the given substring.
protected def captureLogOutput(f: () => Unit): String = {
// Return the newly added log contents in the log file after executing the function `f`
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Obviously, the using of the function is:
Return the newly added log contents in the log file after executing the function f

private def captureLogOutput(f: () => Unit): String = {
val content = if (logFile.exists()) {
Files.readString(logFile.toPath)
} else {
Expand All @@ -52,35 +57,33 @@ abstract class LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuit

def msgWithMDCAndException: LogEntry = log"Error in executor ${MDC(EXECUTOR_ID, "1")}."

def expectedPatternForBasicMsg(level: Level): String
Copy link
Contributor Author

Choose a reason for hiding this comment

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

Using 'Level' instead of 'String' may be more reasonable


def msgWithConcat: LogEntry = log"Min Size: ${MDC(MIN_SIZE, "2")}, " +
log"Max Size: ${MDC(MAX_SIZE, "4")}. " +
log"Please double check."

def expectedPatternForMsgWithMDC(level: Level): String

def expectedPatternForBasicMsg(level: String): String

def expectedPatternForMsgWithMDC(level: String): String

def expectedPatternForMsgWithMDCAndException(level: String): String
def expectedPatternForMsgWithMDCAndException(level: Level): String

def verifyMsgWithConcat(level: String, logOutput: String): Unit
def verifyMsgWithConcat(level: Level, logOutput: String): Unit

test("Basic logging") {
val msg = "This is a log message"
Copy link
Contributor Author

@panbingkun panbingkun Mar 31, 2024

Choose a reason for hiding this comment

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

msg and basicMsg are duplicated, we can make full use of basicMsg.

def basicMsg: String = "This is a log message"

Seq(
("ERROR", () => logError(msg)),
("WARN", () => logWarning(msg)),
("INFO", () => logInfo(msg))).foreach { case (level, logFunc) =>
(Level.ERROR, () => logError(basicMsg)),
(Level.WARN, () => logWarning(basicMsg)),
(Level.INFO, () => logInfo(basicMsg))).foreach { case (level, logFunc) =>
val logOutput = captureLogOutput(logFunc)
assert(expectedPatternForBasicMsg(level).r.matches(logOutput))
}
}

test("Logging with MDC") {
Seq(
("ERROR", () => logError(msgWithMDC)),
("WARN", () => logWarning(msgWithMDC)),
("INFO", () => logInfo(msgWithMDC))).foreach {
(Level.ERROR, () => logError(msgWithMDC)),
(Level.WARN, () => logWarning(msgWithMDC)),
(Level.INFO, () => logInfo(msgWithMDC))).foreach {
case (level, logFunc) =>
val logOutput = captureLogOutput(logFunc)
assert(expectedPatternForMsgWithMDC(level).r.matches(logOutput))
Expand All @@ -90,9 +93,9 @@ abstract class LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuit
test("Logging with MDC and Exception") {
val exception = new RuntimeException("OOM")
Seq(
("ERROR", () => logError(msgWithMDCAndException, exception)),
("WARN", () => logWarning(msgWithMDCAndException, exception)),
("INFO", () => logInfo(msgWithMDCAndException, exception))).foreach {
(Level.ERROR, () => logError(msgWithMDCAndException, exception)),
(Level.WARN, () => logWarning(msgWithMDCAndException, exception)),
(Level.INFO, () => logInfo(msgWithMDCAndException, exception))).foreach {
case (level, logFunc) =>
val logOutput = captureLogOutput(logFunc)
assert(expectedPatternForMsgWithMDCAndException(level).r.findFirstIn(logOutput).isDefined)
Expand All @@ -101,9 +104,9 @@ abstract class LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuit

test("Logging with concat") {
Seq(
("ERROR", () => logError(msgWithConcat)),
("WARN", () => logWarning(msgWithConcat)),
("INFO", () => logInfo(msgWithConcat))).foreach {
(Level.ERROR, () => logError(msgWithConcat)),
(Level.WARN, () => logWarning(msgWithConcat)),
(Level.INFO, () => logInfo(msgWithConcat))).foreach {
case (level, logFunc) =>
val logOutput = captureLogOutput(logFunc)
verifyMsgWithConcat(level, logOutput)
Expand All @@ -112,32 +115,87 @@ abstract class LoggingSuiteBase extends AnyFunSuite // scalastyle:ignore funsuit
}

class StructuredLoggingSuite extends LoggingSuiteBase {
private val className = this.getClass.getName.stripSuffix("$")
override def className: String = classOf[StructuredLoggingSuite].getName
override def logFilePath: String = "target/structured.log"

override def expectedPatternForBasicMsg(level: String): String =
s"""\\{"ts":"[^"]+","level":"$level","msg":"This is a log message","logger":"$className"}\n"""

override def expectedPatternForMsgWithMDC(level: String): String =
// scalastyle:off line.size.limit
s"""\\{"ts":"[^"]+","level":"$level","msg":"Lost executor 1.","context":\\{"executor_id":"1"},"logger":"$className"}\n"""
// scalastyle:on
private val jsonMapper = new ObjectMapper().registerModule(DefaultScalaModule)
private def compactAndToRegexPattern(json: String): String = {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

In testing, this method compactAndToRegexPattern will ultimately be used to compact the output JSON and convert it into a regex pattern for matching and validation.

jsonMapper.readTree(json).toString.
replace("<timestamp>", """[^"]+""").
replace(""""<stacktrace>"""", """.*""").
replace("{", """\{""") + "\n"
}

override def expectedPatternForMsgWithMDCAndException(level: String): String =
// scalastyle:off line.size.limit
s"""\\{"ts":"[^"]+","level":"$level","msg":"Error in executor 1.","context":\\{"executor_id":"1"},"exception":\\{"class":"java.lang.RuntimeException","msg":"OOM","stacktrace":.*},"logger":"$className"}\n"""
// scalastyle:on
override def expectedPatternForBasicMsg(level: Level): String = {
compactAndToRegexPattern(
s"""
{
"ts": "<timestamp>",
"level": "$level",
"msg": "This is a log message",
"logger": "$className"
}""")
}

override def verifyMsgWithConcat(level: String, logOutput: String): Unit = {
// scalastyle:off line.size.limit
val pattern1 =
s"""\\{"ts":"[^"]+","level":"$level","msg":"Min Size: 2, Max Size: 4. Please double check.","context":\\{"min_size":"2","max_size": "4"},"logger":"$className"}\n"""
override def expectedPatternForMsgWithMDC(level: Level): String = {
compactAndToRegexPattern(
s"""
{
"ts": "<timestamp>",
"level": "$level",
"msg": "Lost executor 1.",
"context": {
"executor_id": "1"
},
"logger": "$className"
}""")
}

val pattern2 =
s"""\\{"ts":"[^"]+","level":"$level","msg":"Min Size: 2, Max Size: 4. Please double check.","context":\\{"max_size":"4","min_size":"2"},"logger":"$className"}\n"""
override def expectedPatternForMsgWithMDCAndException(level: Level): String = {
compactAndToRegexPattern(
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This line is too long,
Show it in pretty json to developers,
This can also eliminate the comment scalastyle:off line.size.limit

s"""
{
"ts": "<timestamp>",
"level": "$level",
"msg": "Error in executor 1.",
"context": {
"executor_id": "1"
},
"exception": {
"class": "java.lang.RuntimeException",
"msg": "OOM",
"stacktrace": "<stacktrace>"
},
"logger": "$className"
}""")
}

override def verifyMsgWithConcat(level: Level, logOutput: String): Unit = {
val pattern1 = compactAndToRegexPattern(
s"""
{
"ts": "<timestamp>",
"level": "$level",
"msg": "Min Size: 2, Max Size: 4. Please double check.",
"context": {
"min_size": "2",
"max_size": "4"
},
"logger": "$className"
}""")

val pattern2 = compactAndToRegexPattern(
s"""
{
"ts": "<timestamp>",
"level": "$level",
"msg": "Min Size: 2, Max Size: 4. Please double check.",
"context": {
"max_size": "4",
"min_size": "2"
},
"logger": "$className"
}""")
assert(pattern1.r.matches(logOutput) || pattern2.r.matches(logOutput))
// scalastyle:on
}

}