Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/18159#discussion_r120350843
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/FileWritingCommand.scala
 ---
    @@ -0,0 +1,108 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution.command
    +
    +import org.apache.spark.SparkContext
    +import org.apache.spark.sql.{Row, SparkSession}
    +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
    +import org.apache.spark.sql.catalyst.plans.logical
    +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
    +import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary
    +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A logical command specialized for writing data out. 
`FileWritingCommand`s are
    + * wrapped in `FileWritingCommandExec` during execution.
    + */
    +trait FileWritingCommand extends logical.Command {
    +
    +  // The caller of `FileWritingCommand` can replace the metrics location 
by providing this external
    +  // metrics structure.
    +  private var _externalMetrics: Option[Map[String, SQLMetric]] = None
    +  private[sql] def withExternalMetrics(map: Map[String, SQLMetric]): 
this.type = {
    +    _externalMetrics = Option(map)
    +    this
    +  }
    +
    +  /**
    +   * Those metrics will be updated once the command finishes writing data 
out. Those metrics will
    +   * be taken by `FileWritingCommandExec` as its metrics when showing in 
UI.
    +   */
    +  def metrics(sparkContext: SparkContext): Map[String, SQLMetric] = 
_externalMetrics.getOrElse {
    +    Map(
    +      // General metrics.
    +      "avgTime" -> SQLMetrics.createMetric(sparkContext, "average writing 
time (ms)"),
    +      "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of 
written files"),
    +      "numOutputBytes" -> SQLMetrics.createMetric(sparkContext, "bytes of 
written output"),
    +      "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of 
output rows"),
    +      "numParts" -> SQLMetrics.createMetric(sparkContext, "number of 
dynamic part")
    +    )
    +  }
    +
    +  /**
    +   * Callback function that update metrics collected from the writing 
operation.
    +   */
    +  private[sql] def postDriverMetrics(sparkContext: SparkContext, metrics: 
Map[String, SQLMetric])
    +      (writeSummaries: Seq[ExecutedWriteSummary]): Unit = {
    +    var numPartitions = 0
    +    var numFiles = 0
    +    var totalNumBytes: Long = 0L
    +    var totalNumOutput: Long = 0L
    +
    +    writeSummaries.foreach { summary =>
    +      numPartitions += summary.updatedPartitions.size
    +      numFiles += summary.numOutputFile
    +      totalNumBytes += summary.numOutputBytes
    +      totalNumOutput += summary.numOutputRows
    +    }
    +
    +    // The time for writing individual file can be zero if it's less than 
1 ms. Zero values can
    +    // lower actual time of writing when calculating average, so excluding 
them.
    +    val writingTime =
    +      Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 
0))).toLong
    +
    +    val metricsNames = metrics.keys.toSeq.sorted
    +    val metricsValues = Seq(writingTime, numFiles, totalNumBytes, 
totalNumOutput, numPartitions)
    +    metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2))
    +
    +    val executionId = 
sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
    +    SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, 
metricsNames.map(metrics(_)))
    +  }
    +
    +  def run(
    +    sparkSession: SparkSession,
    +    children: Seq[SparkPlan],
    +    metrics: Map[String, SQLMetric],
    +    metricsCallback: (Seq[ExecutedWriteSummary]) => Unit): Seq[Row]
    --- End diff --
    
    The case is more complicated...
    
    We have those commands which don't write the data but invoke another 
commands to do that. The execution data for showing on UI is bound to the 
original commands, but the invoked commands. That's why we need to pass metrics 
or callback function here...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to