-
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 19 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 |
|---|---|---|
|
|
@@ -1002,6 +1002,15 @@ private[spark] object Utils extends Logging { | |
| } | ||
| } | ||
|
|
||
| /** | ||
| * Lists files recursively. | ||
| */ | ||
| def recursiveList(f: File): Array[File] = { | ||
| require(f.isDirectory) | ||
| val current = f.listFiles | ||
| current ++ current.filter(_.isDirectory).flatMap(recursiveList) | ||
| } | ||
|
|
||
| /** | ||
| * Delete a file or directory and its contents recursively. | ||
| * Don't follow directories if they are symlinks. | ||
|
|
@@ -2659,6 +2668,21 @@ private[spark] object Utils extends Logging { | |
| redact(redactionPattern, kvs.toArray) | ||
| } | ||
|
|
||
| /** | ||
| * Computes the average of all elements in an `Iterable`. If there is no element, returns 0. | ||
| */ | ||
| def average[T](ts: Iterable[T])(implicit num: Numeric[T]): Double = { | ||
| if (ts.isEmpty) { | ||
| 0.0 | ||
| } else { | ||
| var count = 0 | ||
| val sum = ts.reduce { (sum, ele) => | ||
| count += 1 | ||
| num.plus(sum, ele) | ||
| } | ||
| num.toDouble(sum) / (count + 1) | ||
|
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. Is this better? Now we don't call |
||
| } | ||
| } | ||
| } | ||
|
|
||
| private[util] object CallerContext extends Logging { | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,24 +19,65 @@ package org.apache.spark.sql.execution.command | |
|
|
||
| import java.util.UUID | ||
|
|
||
| import org.apache.spark.SparkContext | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.{Row, SparkSession} | ||
| import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | ||
| import org.apache.spark.sql.catalyst.errors.TreeNodeException | ||
| import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} | ||
| import org.apache.spark.sql.catalyst.plans.{logical, QueryPlan} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} | ||
| import org.apache.spark.sql.execution.datasources.ExecutedWriteSummary | ||
| import org.apache.spark.sql.execution.debug._ | ||
| import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} | ||
| import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} | ||
| import org.apache.spark.sql.streaming.OutputMode | ||
| import org.apache.spark.sql.types._ | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| /** | ||
| * A logical command that is executed for its side-effects. `RunnableCommand`s are | ||
| * wrapped in `ExecutedCommand` during execution. | ||
| */ | ||
| trait RunnableCommand extends logical.Command { | ||
|
|
||
| // The map used to record the metrics of running the command. This will be passed to | ||
| // `ExecutedCommand` during query planning. | ||
| private[sql] lazy val metrics: Map[String, SQLMetric] = Map.empty | ||
|
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. usually we don't need
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. Oh. got it. |
||
|
|
||
| /** | ||
| * Callback function that update metrics collected from the writing operation. | ||
| */ | ||
| protected def callbackMetricsUpdater(writeSummaries: Seq[ExecutedWriteSummary]): Unit = { | ||
|
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 think it's more reasonable to do this in
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. Ok. I created a trait for 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. how about
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. Good for me. |
||
| val sparkContext = SparkContext.getActive.get | ||
| 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 | ||
|
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 only happens if a partition is very small, right?
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 guess this should be rare?
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. Yeah, it's for small partition like the test case.
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. Ok. Since it should be rare, I removed this part so the codes can be simpler. |
||
| // lower actual time of writing to zero when calculating average, so excluding them. | ||
| val avgWritingTime = | ||
| Utils.average(writeSummaries.flatMap(_.writingTimePerFile.filter(_ > 0))).toLong | ||
|
|
||
| metrics("avgTime").add(avgWritingTime) | ||
| metrics("numFiles").add(numFiles) | ||
| metrics("numOutputBytes").add(totalNumBytes) | ||
| metrics("numOutputRows").add(totalNumOutput) | ||
| metrics("numParts").add(numPartitions) | ||
|
|
||
| val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) | ||
| SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toList) | ||
| } | ||
|
|
||
| def run(sparkSession: SparkSession, children: Seq[SparkPlan]): Seq[Row] = { | ||
| throw new NotImplementedError | ||
| } | ||
|
|
@@ -49,8 +90,14 @@ trait RunnableCommand extends logical.Command { | |
| /** | ||
| * A physical operator that executes the run method of a `RunnableCommand` and | ||
| * saves the result to prevent multiple executions. | ||
| * | ||
| * @param cmd the `RunnableCommand` this operator will run. | ||
| * @param children the children physical plans ran by the `RunnableCommand`. | ||
| */ | ||
| case class ExecutedCommandExec(cmd: RunnableCommand, children: Seq[SparkPlan]) extends SparkPlan { | ||
|
|
||
| override lazy val metrics: Map[String, SQLMetric] = cmd.metrics | ||
|
|
||
| /** | ||
| * 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 | ||
|
|
||
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.