Haejoon Lee created SPARK-48752:
-----------------------------------

             Summary: Introduce `pyspark.logging` for improved structured 
logging for PySpark
                 Key: SPARK-48752
                 URL: https://issues.apache.org/jira/browse/SPARK-48752
             Project: Spark
          Issue Type: Bug
          Components: PySpark
    Affects Versions: 4.0.0
            Reporter: Haejoon Lee


We introduced structured logging from 
[#45729|https://github.com/apache/spark/pull/45729], but PySpark log is still 
hard to figure out in the current structured log, because it is hidden and 
mixed within bunch of complex JVM stacktraces and it's also not very 
Python-friendly:

 
{code:java}
{
  "ts": "2024-06-28T10:53:48.528Z",
  "level": "ERROR",
  "msg": "Exception in task 7.0 in stage 0.0 (TID 7)",
  "context": {
    "task_name": "task 7.0 in stage 0.0 (TID 7)"
  },
  "exception": {
    "class": "org.apache.spark.SparkArithmeticException",
    "msg": "[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate 
divisor being 0 and return NULL instead. If necessary set 
\"spark.sql.ansi.enabled\" to \"false\" to bypass this error. SQLSTATE: 
22012\n== DataFrame ==\n\"__truediv__\" was called 
from\n/.../spark/python/test_error_context.py:17\n",
    "stacktrace": [
      {
        "class": "org.apache.spark.sql.errors.QueryExecutionErrors$",
        "method": "divideByZeroError",
        "file": "QueryExecutionErrors.scala",
        "line": 203
      },
      {
        "class": "org.apache.spark.sql.errors.QueryExecutionErrors",
        "method": "divideByZeroError",
        "file": "QueryExecutionErrors.scala",
        "line": -1
      },
      {
        "class": 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1",
        "method": "project_doConsume_0$",
        "file": null,
        "line": -1
      },
      {
        "class": 
"org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1",
        "method": "processNext",
        "file": null,
        "line": -1
      },
      {
        "class": "org.apache.spark.sql.execution.BufferedRowIterator",
        "method": "hasNext",
        "file": "BufferedRowIterator.java",
        "line": 43
      },
      {
        "class": 
"org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvaluator$$anon$1",
        "method": "hasNext",
        "file": "WholeStageCodegenEvaluatorFactory.scala",
        "line": 50
      },
      {
        "class": "org.apache.spark.sql.execution.SparkPlan",
        "method": "$anonfun$getByteArrayRdd$1",
        "file": "SparkPlan.scala",
        "line": 388
      },
      {
        "class": "org.apache.spark.rdd.RDD",
        "method": "$anonfun$mapPartitionsInternal$2",
        "file": "RDD.scala",
        "line": 896
      },
      {
        "class": "org.apache.spark.rdd.RDD",
        "method": "$anonfun$mapPartitionsInternal$2$adapted",
        "file": "RDD.scala",
        "line": 896
      },
      {
        "class": "org.apache.spark.rdd.MapPartitionsRDD",
        "method": "compute",
        "file": "MapPartitionsRDD.scala",
        "line": 52
      },
      {
        "class": "org.apache.spark.rdd.RDD",
        "method": "computeOrReadCheckpoint",
        "file": "RDD.scala",
        "line": 369
      },
      {
        "class": "org.apache.spark.rdd.RDD",
        "method": "iterator",
        "file": "RDD.scala",
        "line": 333
      },
      {
        "class": "org.apache.spark.scheduler.ResultTask",
        "method": "runTask",
        "file": "ResultTask.scala",
        "line": 93
      },
      {
        "class": "org.apache.spark.TaskContext",
        "method": "runTaskWithListeners",
        "file": "TaskContext.scala",
        "line": 171
      },
      {
        "class": "org.apache.spark.scheduler.Task",
        "method": "run",
        "file": "Task.scala",
        "line": 146
      },
      {
        "class": "org.apache.spark.executor.Executor$TaskRunner",
        "method": "$anonfun$run$5",
        "file": "Executor.scala",
        "line": 644
      },
      {
        "class": "org.apache.spark.util.SparkErrorUtils",
        "method": "tryWithSafeFinally",
        "file": "SparkErrorUtils.scala",
        "line": 64
      },
      {
        "class": "org.apache.spark.util.SparkErrorUtils",
        "method": "tryWithSafeFinally$",
        "file": "SparkErrorUtils.scala",
        "line": 61
      },
      {
        "class": "org.apache.spark.util.Utils$",
        "method": "tryWithSafeFinally",
        "file": "Utils.scala",
        "line": 99
      },
      {
        "class": "org.apache.spark.executor.Executor$TaskRunner",
        "method": "run",
        "file": "Executor.scala",
        "line": 647
      },
      {
        "class": "java.util.concurrent.ThreadPoolExecutor",
        "method": "runWorker",
        "file": "ThreadPoolExecutor.java",
        "line": 1136
      },
      {
        "class": "java.util.concurrent.ThreadPoolExecutor$Worker",
        "method": "run",
        "file": "ThreadPoolExecutor.java",
        "line": 635
      },
      {
        "class": "java.lang.Thread",
        "method": "run",
        "file": "Thread.java",
        "line": 840
      }
    ]
  },
  "logger": "Executor"
} {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to