Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
74 commits
Select commit Hold shift + click to select a range
7dec5eb
[SPARK-47705][INFRA] Sort LogKey alphabetically and build a test to e…
dtenedor Apr 3, 2024
6a0555c
[SPARK-47700][SQL] Fix formatting of error messages with treeNode
jchen5 Apr 3, 2024
49eefc5
[SPARK-47722][SS] Wait until RocksDB background work finish before cl…
WweiL Apr 3, 2024
fbe6b1d
[SPARK-47721][DOC] Guidelines for the Structured Logging Framework
gengliangwang Apr 3, 2024
e3aab8c
[SPARK-47210][SQL] Addition of implicit casting without indeterminate…
mihailomilosevic2001 Apr 3, 2024
d87ac8e
[SPARK-47708][CONNECT] Do not log gRPC exception to stderr in PySpark
nemanja-boric-databricks Apr 4, 2024
447f8af
[SPARK-47720][CORE] Update `spark.speculation.multiplier` to 3 and `s…
dongjoon-hyun Apr 4, 2024
678aeb7
[SPARK-47683][PYTHON][BUILD] Decouple PySpark core API to pyspark.cor…
HyukjinKwon Apr 4, 2024
c25fd93
[SPARK-47705][INFRA][FOLLOWUP] Sort LogKey alphabetically and build a…
panbingkun Apr 4, 2024
d272a1b
[SPARK-47724][PYTHON][TESTS] Add an environment variable for testing …
HyukjinKwon Apr 4, 2024
d75c775
[SPARK-46812][PYTHON][TESTS][FOLLOWUP] Skip `pandas`-required tests i…
dongjoon-hyun Apr 4, 2024
3f6ac60
[SPARK-47577][CORE][PART1] Migrate logError with variables to structu…
gengliangwang Apr 4, 2024
f6999df
[SPARK-47081][CONNECT] Support Query Execution Progress
grundprinzip Apr 4, 2024
bffb02d
[SPARK-47565][PYTHON] PySpark worker pool crash resilience
Apr 4, 2024
3b8aea3
Revert "[SPARK-47708][CONNECT] Do not log gRPC exception to stderr in…
nemanja-boric-databricks Apr 4, 2024
5f9f5db
[SPARK-47689][SQL][FOLLOWUP] More accurate file path in TASK_WRITE_FA…
cloud-fan Apr 4, 2024
5ca3467
[SPARK-47729][PYTHON][TESTS] Get the proper default port for pyspark-…
HyukjinKwon Apr 4, 2024
25fc67f
[SPARK-47728][DOC] Document G1 Concurrent GC metrics
LucaCanali Apr 4, 2024
e3405c1
[SPARK-47610][CONNECT][FOLLOWUP] Add -Dio.netty.tryReflectionSetAcces…
pan3793 Apr 4, 2024
3fd0cd6
[SPARK-47598][CORE] MLLib: Migrate logError with variables to structu…
panbingkun Apr 4, 2024
240923c
[SPARK-46812][PYTHON][TESTS][FOLLOWUP] Check should_test_connect and …
dongjoon-hyun Apr 4, 2024
fb96b1a
[SPARK-47723][CORE][TESTS] Introduce a tool that can sort alphabetica…
panbingkun Apr 5, 2024
404d58c
[SPARK-47081][CONNECT][FOLLOW-UP] Add the `shell` module into PyPI pa…
HyukjinKwon Apr 5, 2024
b9ca91d
[SPARK-47712][CONNECT] Allow connect plugins to create and process Da…
tomvanbussel Apr 5, 2024
0107435
[SPARK-47734][PYTHON][TESTS] Fix flaky DataFrame.writeStream doctest …
JoshRosen Apr 5, 2024
d5620cb
[SPARK-47289][SQL] Allow extensions to log extended information in ex…
parthchandra Apr 5, 2024
aeb082e
[SPARK-47081][CONNECT][TESTS][FOLLOW-UP] Skip the flaky doctests for now
HyukjinKwon Apr 5, 2024
97e63ff
[SPARK-47735][PYTHON][TESTS] Make pyspark.testing.connectutils compat…
HyukjinKwon Apr 5, 2024
12d0367
[SPARK-47724][PYTHON][TESTS][FOLLOW-UP] Make testing script to inheri…
HyukjinKwon Apr 5, 2024
6bd0ccf
[SPARK-47511][SQL][FOLLOWUP] Rename the config REPLACE_NULLIF_USING_W…
cloud-fan Apr 5, 2024
c34baeb
[SPARK-47719][SQL] Change spark.sql.legacy.timeParserPolicy default t…
srielau Apr 5, 2024
18072b5
[SPARK-47577][CORE][PART2] Migrate logError with variables to structu…
gengliangwang Apr 5, 2024
1efbf43
[SPARK-47310][SS] Add micro-benchmark for merge operations for multip…
anishshri-db Apr 5, 2024
d1ace24
[SPARK-47582][SQL] Migrate Catalyst logInfo with variables to structu…
dtenedor Apr 5, 2024
11abc64
[SPARK-47094][SQL] SPJ : Dynamically rebalance number of buckets when…
szehon-ho Apr 6, 2024
42dc815
[SPARK-47743][CORE] Use milliseconds as the time unit in logging
gengliangwang Apr 6, 2024
7385f19
[SPARK-47592][CORE] Connector module: Migrate logError with variables…
panbingkun Apr 6, 2024
d69df59
[SPARK-47738][BUILD] Upgrade Kafka to 3.7.0
panbingkun Apr 6, 2024
60a3fbc
[SPARK-47727][PYTHON] Make SparkConf to root level to for both SparkS…
HyukjinKwon Apr 6, 2024
644687b
[SPARK-47709][BUILD] Upgrade tink to 1.13.0
LuciferYang Apr 6, 2024
4d9dbb3
[SPARK-46722][CONNECT][SS][TESTS][FOLLOW-UP] Drop the tables after te…
HyukjinKwon Apr 7, 2024
c11585a
[SPARK-47751][PYTHON][CONNECT] Make pyspark.worker_utils compatible w…
HyukjinKwon Apr 7, 2024
d743012
[SPARK-47753][PYTHON][CONNECT][TESTS] Make pyspark.testing compatible…
HyukjinKwon Apr 7, 2024
f7dff4a
[SPARK-47752][PS][CONNECT] Make pyspark.pandas compatible with pyspar…
HyukjinKwon Apr 7, 2024
e92e8f5
[SPARK-47744] Add support for negative-valued bytes in range encoder
neilramaswamy Apr 7, 2024
0c992b2
[SPARK-47755][CONNECT] Pivot should fail when the number of distinct …
zhengruifeng Apr 7, 2024
b299b2b
[SPARK-47299][PYTHON][DOCS] Use the same `versions.json` in the dropd…
panbingkun Apr 8, 2024
cc6c0eb
[MINOR][TESTS] Deduplicate test cases `test_parse_datatype_string`
HyukjinKwon Apr 8, 2024
ad2367c
[MINOR][PYTHON][SS][TESTS] Drop the tables after being used at `test_…
HyukjinKwon Apr 8, 2024
f576b85
[SPARK-47541][SQL] Collated strings in complex types supporting opera…
nikolamand-db Apr 8, 2024
d55bb61
[SPARK-47558][SS] State TTL support for ValueState
sahnib Apr 8, 2024
3a39ac2
[SPARK-47713][SQL][CONNECT] Fix a self-join failure
zhengruifeng Apr 8, 2024
eb8e997
[SPARK-47657][SQL] Implement collation filter push down support per f…
stefankandic Apr 8, 2024
f0d8f82
[SPARK-47750][DOCS][SQL] Postgres: Document Mapping Spark SQL Data Ty…
yaooqinn Apr 8, 2024
211afd4
[MINOR][PYTHON][CONNECT][TESTS] Enable `MapInPandasParityTests.test_d…
zhengruifeng Apr 8, 2024
f94d95d
[SPARK-47762][PYTHON][CONNECT] Add pyspark.sql.connect.protobuf into …
HyukjinKwon Apr 8, 2024
29d077f
[SPARK-47748][BUILD] Upgrade `zstd-jni` to 1.5.6-2
panbingkun Apr 8, 2024
60806c6
[SPARK-47746] Implement ordinal-based range encoding in the RocksDBSt…
neilramaswamy Apr 8, 2024
134a139
[SPARK-47681][SQL] Add schema_of_variant expression
chenhao-db Apr 8, 2024
abb7b04
[SPARK-47504][SQL] Resolve AbstractDataType simpleStrings for StringT…
mihailomilosevic2001 Apr 8, 2024
91b2331
[WIP] ListStateTTL implementation
ericm-db Apr 8, 2024
479392a
adding log lines
ericm-db Apr 8, 2024
7aab43e
test cases pass
ericm-db Apr 8, 2024
71f960d
spacing
ericm-db Apr 8, 2024
998764c
using NextIterator instead
ericm-db Apr 8, 2024
1dcb7d8
refactor feedback
ericm-db Apr 9, 2024
47867e7
undoing unnecessary change
ericm-db Apr 9, 2024
cfd30c3
refactor get_ttl_value
ericm-db Apr 9, 2024
4a19cb7
refactor test case
ericm-db Apr 9, 2024
993125c
specific doc for clearIfExpired
ericm-db Apr 9, 2024
fd5200f
moving isExpired to common place
ericm-db Apr 9, 2024
d43ffb1
refactoring to use common utils
ericm-db Apr 9, 2024
30f6094
updating interface header
ericm-db Apr 9, 2024
e9376d9
Map State TTL, Initial Commit
ericm-db Apr 9, 2024
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
Prev Previous commit
Next Next commit
[SPARK-47582][SQL] Migrate Catalyst logInfo with variables to structu…
…red logging framework

### What changes were proposed in this pull request?

Migrate logInfo with variables of the Catalyst module to structured logging framework. This is part1 which transforms the logInfo entries of the following API
```
def logInfo(msg: => String): Unit
```
to
```
def logInfo(entry: LogEntry): Unit
```

### Why are the changes needed?

To enhance Apache Spark's logging system by implementing structured logging.

### Does this PR introduce _any_ user-facing change?

Yes, Spark core logs will contain additional MDC

### How was this patch tested?

Compiler and scala style checks, as well as code review.

### Was this patch authored or co-authored using generative AI tooling?

No

Closes apache#45866 from dtenedor/catalyst-log-info.

Authored-by: Daniel Tenedorio <[email protected]>
Signed-off-by: Gengliang Wang <[email protected]>
  • Loading branch information
dtenedor authored and gengliangwang committed Apr 5, 2024
commit d1ace24f8facf48af8f5d20c2a8b1d1870f281f7
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ object LogKey extends Enumeration {
val BLOCK_MANAGER_ID = Value
val BROADCAST_ID = Value
val BUCKET = Value
val BYTECODE_SIZE = Value
val CATEGORICAL_FEATURES = Value
val CLASS_LOADER = Value
val CLASS_NAME = Value
Expand All @@ -50,6 +51,7 @@ object LogKey extends Enumeration {
val FAILURES = Value
val HOST = Value
val JOB_ID = Value
val JOIN_CONDITION = Value
val LEARNING_RATE = Value
val LINE = Value
val LINE_NUM = Value
Expand All @@ -72,13 +74,17 @@ object LogKey extends Enumeration {
val PATHS = Value
val POD_ID = Value
val PORT = Value
val QUERY_PLAN = Value
val RANGE = Value
val RDD_ID = Value
val REASON = Value
val REDUCE_ID = Value
val REMOTE_ADDRESS = Value
val RETRY_COUNT = Value
val RPC_ADDRESS = Value
val RULE_BATCH_NAME = Value
val RULE_NAME = Value
val RULE_NUMBER_OF_RUNS = Value
val SHUFFLE_BLOCK_INFO = Value
val SHUFFLE_ID = Value
val SHUFFLE_MERGE_ID = Value
Expand All @@ -96,9 +102,12 @@ object LogKey extends Enumeration {
val THREAD_NAME = Value
val TID = Value
val TIMEOUT = Value
val TIME_UNITS = Value
val URI = Value
val USER_NAME = Value
val WATERMARK_CONSTRAINT = Value
val WORKER_URL = Value
val XSD_PATH = Value

type LogKey = Value
}
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,8 @@ case class MessageWithContext(message: String, context: java.util.HashMap[String
resultMap.putAll(mdc.context)
MessageWithContext(message + mdc.message, resultMap)
}

def stripMargin: MessageWithContext = copy(message = message.stripMargin)
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.analysis
import scala.util.control.NonFatal

import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKey._
import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan}
Expand Down Expand Up @@ -107,7 +109,8 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
case _ => None
}
if (stateWatermark.nonEmpty) {
logInfo(s"Condition $joinCondition generated watermark constraint = ${stateWatermark.get}")
logInfo(log"Condition ${MDC(JOIN_CONDITION, joinCondition)} generated " +
log"watermark constraint = ${MDC(WATERMARK_CONSTRAINT, stateWatermark.get)}")
}
stateWatermark
}
Expand Down Expand Up @@ -195,7 +198,8 @@ object StreamingJoinHelper extends PredicateHelper with Logging {
}.reduceLeft(Add(_, _))

// Calculate the constraint value
logInfo(s"Final expression to evaluate constraint:\t$exprWithWatermarkSubstituted")
logInfo(log"Final expression to evaluate " +
log"constraint:\t${MDC(WATERMARK_CONSTRAINT, exprWithWatermarkSubstituted)}")
val constraintValue = exprWithWatermarkSubstituted.eval().asInstanceOf[java.lang.Double]
Some((Double2double(constraintValue) / 1000.0).toLong)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,8 @@ import org.codehaus.janino.util.ClassFile
import org.apache.spark.{SparkException, SparkIllegalArgumentException, TaskContext, TaskKilledException}
import org.apache.spark.executor.InputMetrics
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKey._
import org.apache.spark.internal.MDC
import org.apache.spark.metrics.source.CodegenMetrics
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.encoders.HashableWeakReference
Expand Down Expand Up @@ -1593,7 +1595,8 @@ object CodeGenerator extends Logging {

if (byteCodeSize > DEFAULT_JVM_HUGE_METHOD_LIMIT) {
logInfo("Generated method too long to be JIT compiled: " +
s"${cf.getThisClassName}.${method.getName} is $byteCodeSize bytes")
log"${MDC(CLASS_NAME, cf.getThisClassName)}.${MDC(METHOD_NAME, method.getName)} " +
log"is ${MDC(BYTECODE_SIZE, byteCodeSize)} bytes")
}

byteCodeSize
Expand Down Expand Up @@ -1638,7 +1641,7 @@ object CodeGenerator extends Logging {
val timeMs: Double = duration.toDouble / NANOS_PER_MILLIS
CodegenMetrics.METRIC_SOURCE_CODE_SIZE.update(code.body.length)
CodegenMetrics.METRIC_COMPILATION_TIME.update(timeMs.toLong)
logInfo(s"Code generated in $timeMs ms")
logInfo(log"Code generated in ${MDC(TIME_UNITS, timeMs)} ms")
_compileTime.add(duration)
result
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.collection.mutable

import org.apache.spark.SparkException
import org.apache.spark.internal.LogKey._
import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
Expand Down Expand Up @@ -455,7 +457,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
val filteredRules = batch.rules.filter { rule =>
val exclude = excludedRules.contains(rule.ruleName)
if (exclude) {
logInfo(s"Optimization rule '${rule.ruleName}' is excluded from the optimizer.")
logInfo(log"Optimization rule '${MDC(RULE_NAME, rule.ruleName)}' " +
log"is excluded from the optimizer.")
}
!exclude
}
Expand All @@ -464,8 +467,8 @@ abstract class Optimizer(catalogManager: CatalogManager)
} else if (filteredRules.nonEmpty) {
Some(Batch(batch.name, batch.strategy, filteredRules: _*))
} else {
logInfo(s"Optimization batch '${batch.name}' is excluded from the optimizer " +
s"as all enclosed rules have been excluded.")
logInfo(log"Optimization batch '${MDC(RULE_BATCH_NAME, batch.name)}' " +
log"is excluded from the optimizer as all enclosed rules have been excluded.")
None
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@
package org.apache.spark.sql.catalyst.rules

import org.apache.spark.SparkException
import org.apache.spark.internal.Logging
import org.apache.spark.internal.{Logging, MessageWithContext}
import org.apache.spark.internal.LogKey._
import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.QueryPlanningTracker
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.util.DateTimeConstants.NANOS_PER_SECOND
Expand Down Expand Up @@ -56,10 +58,10 @@ class PlanChangeLogger[TreeType <: TreeNode[_]] extends Logging {
def logRule(ruleName: String, oldPlan: TreeType, newPlan: TreeType): Unit = {
if (!newPlan.fastEquals(oldPlan)) {
if (logRules.isEmpty || logRules.get.contains(ruleName)) {
def message(): String = {
s"""
|=== Applying Rule $ruleName ===
|${sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n")}
def message(): MessageWithContext = {
log"""
|=== Applying Rule ${MDC(RULE_NAME, ruleName)} ===
|${MDC(QUERY_PLAN, sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n"))}
""".stripMargin
}

Expand All @@ -70,14 +72,14 @@ class PlanChangeLogger[TreeType <: TreeNode[_]] extends Logging {

def logBatch(batchName: String, oldPlan: TreeType, newPlan: TreeType): Unit = {
if (logBatches.isEmpty || logBatches.get.contains(batchName)) {
def message(): String = {
def message(): MessageWithContext = {
if (!oldPlan.fastEquals(newPlan)) {
s"""
|=== Result of Batch $batchName ===
|${sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n")}
log"""
|=== Result of Batch ${MDC(RULE_BATCH_NAME, batchName)} ===
|${MDC(QUERY_PLAN, sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n"))}
""".stripMargin
} else {
s"Batch $batchName has no effect."
log"Batch ${MDC(RULE_BATCH_NAME, batchName)} has no effect."
}
}

Expand All @@ -88,26 +90,26 @@ class PlanChangeLogger[TreeType <: TreeNode[_]] extends Logging {
def logMetrics(metrics: QueryExecutionMetrics): Unit = {
val totalTime = metrics.time / NANOS_PER_SECOND.toDouble
val totalTimeEffective = metrics.timeEffective / NANOS_PER_SECOND.toDouble
val message =
s"""
val message: MessageWithContext =
log"""
|=== Metrics of Executed Rules ===
|Total number of runs: ${metrics.numRuns}
|Total time: $totalTime seconds
|Total number of effective runs: ${metrics.numEffectiveRuns}
|Total time of effective runs: $totalTimeEffective seconds
|Total number of runs: ${MDC(RULE_NUMBER_OF_RUNS, metrics.numRuns)}
|Total time: ${MDC(TIME_UNITS, totalTime)} seconds
|Total number of effective runs: ${MDC(RULE_NUMBER_OF_RUNS, metrics.numEffectiveRuns)}
|Total time of effective runs: ${MDC(TIME_UNITS, totalTimeEffective)} seconds
""".stripMargin

logBasedOnLevel(message)
}

private def logBasedOnLevel(f: => String): Unit = {
private def logBasedOnLevel(f: => MessageWithContext): Unit = {
logLevel match {
case "TRACE" => logTrace(f)
case "DEBUG" => logDebug(f)
case "TRACE" => logTrace(f.message)
case "DEBUG" => logDebug(f.message)
case "INFO" => logInfo(f)
case "WARN" => logWarning(f)
case "ERROR" => logError(f)
case _ => logTrace(f)
case _ => logTrace(f.message)
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ import org.apache.hadoop.fs.Path
import org.apache.spark.SparkFiles
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.internal.Logging
import org.apache.spark.internal.LogKey._
import org.apache.spark.internal.MDC

/**
* Utilities for working with XSD validation.
Expand Down Expand Up @@ -56,7 +58,8 @@ object ValidatorUtil extends Logging {
case e: Throwable =>
// Handle case where it was added with sc.addFile
// When they are added via sc.addFile, they are always downloaded to local file system
logInfo(s"$xsdPath was not found, falling back to look up files added by Spark")
logInfo(log"${MDC(XSD_PATH, xsdPath)} was not found, " +
log"falling back to look up files added by Spark")
val f = new File(SparkFiles.get(xsdPath.toString))
if (f.exists()) {
new FileInputStream(f)
Expand Down