Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-23483 Feature parity for Python vs Scala APIs
  3. SPARK-23942

PySpark's collect doesn't trigger QueryExecutionListener

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.0
    • 2.3.1, 2.4.0
    • PySpark, SQL
    • None

    Description

      For example, if you have an custom query execution listener:

      package org.apache.spark.sql
      
      import org.apache.spark.internal.Logging
      import org.apache.spark.sql.execution.QueryExecution
      import org.apache.spark.sql.util.QueryExecutionListener
      
      
      class TestQueryExecutionListener extends QueryExecutionListener with Logging {
        override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = {
          logError("Look at me! I'm 'onSuccess'")
        }
      
        override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { }
      }
      

      and set "spark.sql.queryExecutionListeners org.apache.spark.sql.TestQueryExecutionListener",

      >>> sql("SELECT * FROM range(1)").collect()
      [Row(id=0)]
      
      >>> spark.conf.set("spark.sql.execution.arrow.enabled", "true")
      >>> sql("SELECT * FROM range(1)").toPandas()
         id
      0   0
      

      Seems other actions like show and etc fine in Scala side too:

      >>> sql("SELECT * FROM range(1)").show()
      18/04/09 17:02:04 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
      +---+
      | id|
      +---+
      |  0|
      +---+
      
      scala> sql("SELECT * FROM range(1)").collect()
      18/04/09 16:58:41 ERROR TestQueryExecutionListener: Look at me! I'm 'onSuccess'
      res1: Array[org.apache.spark.sql.Row] = Array([0])
      

      Attachments

        Activity

          People

            gurwls223 Hyukjin Kwon
            gurwls223 Hyukjin Kwon
            Votes:
            0 Vote for this issue
            Watchers:
            1 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: