-
Notifications
You must be signed in to change notification settings - Fork 29.2k
[SPARK-20703][SQL] Associate metrics with data writes onto DataFrameWriter operations #18159
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 13 commits
a6efa73
2ff2fa1
143764c
43c17c0
8bb02d9
778007d
9819f01
6e61bd6
e8acb49
de069e7
775d2c2
f670ae3
0af718d
9b87b4f
6e8ffc5
9d8d77f
8926156
1defb3a
32f4abf
6ea6bbb
f575f0c
8380023
dfaa4f1
04e79d9
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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 | ||
| } | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's not clear to me why we need this and the comment is not helpful.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I've explained the purpose of this in below comments. Once a command invokes another command to write data. We have to update the metrics in the physical plan of the first command, not the second. Before we invoke the second command, we have to replace the metrics in the physical plan's metrics with the metrics in the first one (i.e., the external metrics). It seems to me that you miss the relationship that there is two different commands.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Indeed, I missed it at first, but then I got it. See below: #18159 (comment)
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I should have better code comment on this part too. I'll add it in next commit. |
||
|
|
||
| /** | ||
| * 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. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Seems arbitrary to exclude 0 values. 1ms values also bring the average down, yet you're keeping those.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Excluding 0 values because 0 values can make the average writing time as 0. It seems to me that it doesn't really make sense to show 0 writing time. Seems it indicates the writing doesn't cost time. 1ms brings the average down, but you still get a meaningful time value. |
||
| val writingTime = | ||
| Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong | ||
|
|
||
| val metricsNames = metrics.keys.toSeq.sorted | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. how do you guarantee the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| val metricsValues = Seq(writingTime, numFiles, totalNumBytes, totalNumOutput, numPartitions) | ||
| metricsNames.zip(metricsValues).foreach(x => metrics(x._1).add(x._2)) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This doesn't make sense if
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I am not sure if I correctly understand your point. The external metrics are always coming from the same kind of trait. It's not coming from user, how it's user-provided map? The order of keys returned by
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I meant user as in caller of this function. This function only works when the input satisfies some requirements and silently fails otherwise. I get it that it's a private method, but it's very error prone. Imagine I want to extend this by adding another metric called At the very least, leave a comment saying that the
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. For metrics purpose, this change is a bit too large. I'd try not to increase the complexity for now. I added a comment for this. |
||
|
|
||
| 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] | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why does this take 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
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. 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. The invoked commands will update the metrics in its physical plans, not the caller's. If we don't pass the metrics or callback function, you won't actually update the correct metrics. That's why we need to pass metrics or callback function here...
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I understand that, but I think it can still be simplified. See below: #18159 (comment) |
||
| } | ||
|
|
||
| /** | ||
| * A physical operator specialized to execute the run method of a `FileWritingCommand`, | ||
| * save the result to prevent multiple executions, and record necessary metrics for UI. | ||
| */ | ||
| case class FileWritingCommandExec( | ||
| cmd: FileWritingCommand, | ||
| children: Seq[SparkPlan]) extends CommandExec { | ||
|
|
||
| override lazy val metrics = cmd.metrics(sqlContext.sparkContext) | ||
|
|
||
| protected[sql] lazy val invokeCommand: Seq[Row] = | ||
| cmd.run(sqlContext.sparkSession, children, metrics, | ||
| cmd.postDriverMetrics(sqlContext.sparkContext, metrics)) | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -35,20 +35,16 @@ import org.apache.spark.sql.types._ | |
| * wrapped in `ExecutedCommand` during execution. | ||
| */ | ||
| trait RunnableCommand extends logical.Command { | ||
| def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { | ||
| throw new NotImplementedError | ||
| } | ||
|
|
||
| def run(sparkSession: SparkSession): Seq[Row] = { | ||
| throw new NotImplementedError | ||
| } | ||
| def run(sparkSession: SparkSession): Seq[Row] | ||
| } | ||
|
|
||
| /** | ||
| * A physical operator that executes the run method of a `RunnableCommand` and | ||
| * A physical operator that executes the run method of a `logical.Command` and | ||
| * saves the result to prevent multiple executions. | ||
| */ | ||
| case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan { | ||
| trait CommandExec extends SparkPlan { | ||
| val cmd: logical.Command | ||
|
|
||
| /** | ||
| * A concrete command should override this lazy field to wrap up any side effects caused by the | ||
| * command or any other computation that should be evaluated exactly once. The value of this field | ||
|
|
@@ -60,14 +56,11 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e | |
| */ | ||
| protected[sql] lazy val sideEffectResult: Seq[InternalRow] = { | ||
| val converter = CatalystTypeConverters.createToCatalystConverter(schema) | ||
| val rows = if (children.isEmpty) { | ||
| cmd.run(sqlContext.sparkSession) | ||
| } else { | ||
| cmd.run(sqlContext.sparkSession, children) | ||
| } | ||
| rows.map(converter(_).asInstanceOf[InternalRow]) | ||
| invokeCommand.map(converter(_).asInstanceOf[InternalRow]) | ||
| } | ||
|
|
||
| protected[sql] val invokeCommand: Seq[Row] | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the name looks weird for a
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Sure. Sounds good. |
||
|
|
||
| override def innerChildren: Seq[QueryPlan[_]] = cmd.innerChildren | ||
|
|
||
| override def output: Seq[Attribute] = cmd.output | ||
|
|
@@ -85,6 +78,15 @@ case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) e | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * A physical operator specialized to execute the run method of a `RunnableCommand` and | ||
| * save the result to prevent multiple executions. | ||
| */ | ||
| case class ExecutedCommandExec(cmd: RunnableCommand) extends CommandExec { | ||
| protected[sql] lazy val invokeCommand: Seq[Row] = cmd.run(sqlContext.sparkSession) | ||
| override def children: Seq[SparkPlan] = Nil | ||
| } | ||
|
|
||
| /** | ||
| * An explain command for users to see how a command will be executed. | ||
| * | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -32,10 +32,12 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute | |
| import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap | ||
| import org.apache.spark.sql.execution.command.FileWritingCommandExec | ||
| import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat | ||
| import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider | ||
| import org.apache.spark.sql.execution.datasources.json.JsonFileFormat | ||
| import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat | ||
| import org.apache.spark.sql.execution.metric.SQLMetric | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.sources._ | ||
| import org.apache.spark.sql.streaming.OutputMode | ||
|
|
@@ -436,7 +438,10 @@ case class DataSource( | |
| * Writes the given [[LogicalPlan]] out to this [[DataSource]] and returns a [[BaseRelation]] for | ||
| * the following reading. | ||
| */ | ||
| def writeAndRead(mode: SaveMode, data: LogicalPlan): BaseRelation = { | ||
| def writeAndRead( | ||
| mode: SaveMode, | ||
| data: LogicalPlan, | ||
| externalMetrics: Option[Map[String, SQLMetric]] = None): BaseRelation = { | ||
| if (data.schema.map(_.dataType).exists(_.isInstanceOf[CalendarIntervalType])) { | ||
| throw new AnalysisException("Cannot save interval data type into external storage.") | ||
| } | ||
|
|
@@ -446,7 +451,12 @@ case class DataSource( | |
| dataSource.createRelation( | ||
| sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) | ||
| case format: FileFormat => | ||
| sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)).toRdd | ||
| val qe = sparkSession.sessionState.executePlan(planForWritingFileFormat(format, mode, data)) | ||
| qe.executedPlan.transform { | ||
| case f: FileWritingCommandExec => | ||
| val newCmd = f.cmd.withExternalMetrics(externalMetrics.getOrElse(null)) | ||
| FileWritingCommandExec(newCmd, f.children) | ||
| }.execute() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't think this is much of an improvement over the previous hack.. See above comments, try to avoid changing this code at all.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Can you provide specified suggestion regarding this? I don't think there is a possible way to avoid this. Those commands are invoking another commands to write the data out. We want to have the metrics updated is the metrics in the original physical node, not the invoked one. To do this, we either pass the callback or the metrics in the original plan into the invoked one. Do you think we have other choice?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hm.. So it looks like you do this in 2 places: For the latter case, I'd pass For the former, I'm not sure, as I see
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yeah, I agree that's not a good way to go.. |
||
| // Replace the schema with that of the DataFrame we just wrote out to avoid re-inferring | ||
| copy(userSpecifiedSchema = Some(data.schema.asNullable)).resolveRelation() | ||
| case _ => | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we require array type parameter? We are calling
sizetwice in this method, which can be very slow if the input is not indexed seq.