Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-38615 SQL Error Attribution Framework
  3. SPARK-39166

Provide runtime error query context for Binary Arithmetic when WSCG is off

    XMLWordPrintableJSON

Details

    • Sub-task
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.3.0
    • 3.3.0
    • SQL
    • None

    Description

      Currently, for most of the cases, the project https://issues.apache.org/jira/browse/SPARK-38615 is able to show where the runtime errors happen within the original query.
      However, after trying on production, I found that the following queries won't show where the divide by 0 error happens

      create table aggTest(i int, j int, k int, d date) using parquet
      insert into aggTest values(1, 2, 0, date'2022-01-01')
      select sum(j)/sum(k),percentile(i, 0.9) from aggTest group by d

      With `percentile` function in the query, the plan can't execute with whole stage codegen. Thus the child plan of `Project` is serialized to executors for execution, from ProjectExec:

        protected override def doExecute(): RDD[InternalRow] = {
          child.execute().mapPartitionsWithIndexInternal { (index, iter) =>
            val project = UnsafeProjection.create(projectList, child.output)
            project.initialize(index)
            iter.map(project)
          }
        }

      Note that the `TreeNode.origin` is not serialized to executors since `TreeNode` doesn't extend the trait `Serializable`, which results in an empty query context on errors. For more details, please read https://issues.apache.org/jira/browse/SPARK-39140

      A dummy fix is to make `TreeNode` extend the trait `Serializable`. However, it can be performance regression if the query text is long (every `TreeNode` carries it for serialization). 
      A better fix is to introduce a new trait `SupportQueryContext` and materialize the truncated query context for special expressions. This jira targets on binary arithmetic expressions only. I will create follow-ups for the remaining expressions which support runtime error query context.

      Attachments

        Activity

          People

            Gengliang.Wang Gengliang Wang
            Gengliang.Wang Gengliang Wang
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: