Github user adrian-ionescu commented on a diff in the pull request:
https://github.com/apache/spark/pull/18159#discussion_r120334376
--- 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 --
Why does this take `metrics` as a parameter, when the trait already has a
`metrics` member?
A big part of this patch is just about passing down metrics through this
interface.. It also makes it quite hard to follow. Is there no way we can avoid
this? It would be a significant improvement if you could find a solution.
Basically try to specify the metrics (either the Map you have, or None)
together with the corresponding callback function just once, when you first
instantiate `FileWritingCommandExec` in `SparkStrategies.scala`. Then you won't
need to pass around metrics all over the place and you also won't need the
`transform ... withExternalMetrics` hack.
---
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 [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]