-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-13916][SQL] Add a metric to WholeStageCodegen to measure duration. #11741
Changes from 3 commits
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 |
---|---|---|
|
@@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.Rule | |
import org.apache.spark.sql.catalyst.util.toCommentSafeString | ||
import org.apache.spark.sql.execution.aggregate.TungstenAggregate | ||
import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, SortMergeJoin} | ||
import org.apache.spark.sql.execution.metric.LongSQLMetricValue | ||
import org.apache.spark.sql.execution.metric.{LongSQLMetricValue, SQLMetrics} | ||
import org.apache.spark.sql.internal.SQLConf | ||
|
||
/** | ||
|
@@ -263,6 +263,10 @@ case class InputAdapter(child: SparkPlan) extends UnaryNode with CodegenSupport | |
override def treeChildren: Seq[SparkPlan] = Nil | ||
} | ||
|
||
object WholeStageCodegen { | ||
val PIPELINE_DURATION_METRIC = "duration" | ||
} | ||
|
||
/** | ||
* WholeStageCodegen compile a subtree of plans that support codegen together into single Java | ||
* function. | ||
|
@@ -300,6 +304,10 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup | |
override def outputPartitioning: Partitioning = child.outputPartitioning | ||
override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
||
override private[sql] lazy val metrics = Map( | ||
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 does this work if there are two WholeStageCodegens in a single stage? 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. Doesn't matter. That would be similar to have two projects in a single stage. The metrics are per instance. |
||
"pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, | ||
WholeStageCodegen.PIPELINE_DURATION_METRIC)) | ||
|
||
override def doExecute(): RDD[InternalRow] = { | ||
val ctx = new CodegenContext | ||
val code = child.asInstanceOf[CodegenSupport].produce(ctx, this) | ||
|
@@ -338,6 +346,8 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup | |
logDebug(s"${CodeFormatter.format(cleanedSource)}") | ||
CodeGenerator.compile(cleanedSource) | ||
|
||
val durationMs = longMetric("pipelineTime") | ||
|
||
val rdds = child.asInstanceOf[CodegenSupport].upstreams() | ||
assert(rdds.size <= 2, "Up to two upstream RDDs can be supported") | ||
if (rdds.length == 1) { | ||
|
@@ -346,7 +356,11 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup | |
val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator] | ||
buffer.init(Array(iter)) | ||
new Iterator[InternalRow] { | ||
override def hasNext: Boolean = buffer.hasNext | ||
override def hasNext: Boolean = { | ||
val v = buffer.hasNext | ||
if (!v) durationMs += buffer.durationMs() | ||
v | ||
} | ||
override def next: InternalRow = buffer.next() | ||
} | ||
} | ||
|
@@ -357,7 +371,11 @@ case class WholeStageCodegen(child: SparkPlan) extends UnaryNode with CodegenSup | |
val buffer = clazz.generate(references).asInstanceOf[BufferedRowIterator] | ||
buffer.init(Array(leftIter, rightIter)) | ||
new Iterator[InternalRow] { | ||
override def hasNext: Boolean = buffer.hasNext | ||
override def hasNext: Boolean = { | ||
val v = buffer.hasNext | ||
if (!v) durationMs += buffer.durationMs() | ||
v | ||
} | ||
override def next: InternalRow = buffer.next() | ||
} | ||
} | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -122,7 +122,7 @@ private class LongSQLMetricParam(val stringValue: Seq[Long] => String, initialVa | |
|
||
private object LongSQLMetricParam extends LongSQLMetricParam(_.sum.toString, 0L) | ||
|
||
private object StaticsLongSQLMetricParam extends LongSQLMetricParam( | ||
private object StatisticsBytesSQLMetricParam extends LongSQLMetricParam( | ||
(values: Seq[Long]) => { | ||
// This is a workaround for SPARK-11013. | ||
// We use -1 as initial value of the accumulator, if the accumulator is valid, we will update | ||
|
@@ -140,6 +140,24 @@ private object StaticsLongSQLMetricParam extends LongSQLMetricParam( | |
s"\n$sum ($min, $med, $max)" | ||
}, -1L) | ||
|
||
private object StatisticsTimingSQLMetricParam extends LongSQLMetricParam( | ||
(values: Seq[Long]) => { | ||
// This is a workaround for SPARK-11013. | ||
// We use -1 as initial value of the accumulator, if the accumulator is valid, we will update | ||
// it at the end of task and the value will be at least 0. | ||
val validValues = values.filter(_ >= 0) | ||
val Seq(sum, min, med, max) = { | ||
val metric = if (validValues.length == 0) { | ||
Seq.fill(4)(0L) | ||
} else { | ||
val sorted = validValues.sorted | ||
Seq(sorted.sum, sorted(0), sorted(validValues.length / 2), sorted(validValues.length - 1)) | ||
} | ||
metric.map(Utils.msDurationToString) | ||
} | ||
s"\n$sum ($min, $med, $max)" | ||
}, -1L) | ||
|
||
private[sql] object SQLMetrics { | ||
|
||
// Identifier for distinguishing SQL metrics from other accumulators | ||
|
@@ -168,15 +186,24 @@ private[sql] object SQLMetrics { | |
// The final result of this metric in physical operator UI may looks like: | ||
// data size total (min, med, max): | ||
// 100GB (100MB, 1GB, 10GB) | ||
createLongMetric(sc, s"$name total (min, med, max)", StaticsLongSQLMetricParam) | ||
createLongMetric(sc, s"$name total (min, med, max)", StatisticsBytesSQLMetricParam) | ||
} | ||
|
||
def createTimingMetric(sc: SparkContext, name: String): LongSQLMetric = { | ||
// The final result of this metric in physical operator UI may looks like: | ||
// duration(min, med, max): | ||
// 1s (800msMB, 1s, 2s) | ||
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. msMB? |
||
createLongMetric(sc, s"$name total (min, med, max)", StatisticsTimingSQLMetricParam) | ||
} | ||
|
||
def getMetricParam(metricParamName: String): SQLMetricParam[SQLMetricValue[Any], Any] = { | ||
val longSQLMetricParam = Utils.getFormattedClassName(LongSQLMetricParam) | ||
val staticsSQLMetricParam = Utils.getFormattedClassName(StaticsLongSQLMetricParam) | ||
val bytesSQLMetricParam = Utils.getFormattedClassName(StatisticsBytesSQLMetricParam) | ||
val timingsSQLMetricParam = Utils.getFormattedClassName(StatisticsTimingSQLMetricParam) | ||
val metricParam = metricParamName match { | ||
case `longSQLMetricParam` => LongSQLMetricParam | ||
case `staticsSQLMetricParam` => StaticsLongSQLMetricParam | ||
case `bytesSQLMetricParam` => StatisticsBytesSQLMetricParam | ||
case `timingsSQLMetricParam` => StatisticsTimingSQLMetricParam | ||
} | ||
metricParam.asInstanceOf[SQLMetricParam[SQLMetricValue[Any], Any]] | ||
} | ||
|
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.
we should use nanoTime. See https://github.com/databricks/scala-style-guide#misc_currentTimeMillis_vs_nanoTime