-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-8861][SQL]Add accumulators to SparkPlan operations #7590
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 4 commits
1a4c9de
0c89d22
9629ce0
1849fe3
1d5fb86
8b9a36c
d5b070e
a5455b1
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 |
|---|---|---|
|
|
@@ -1234,12 +1234,25 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli | |
| * driver can access the accumulator's `value`. | ||
| */ | ||
| def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) | ||
| : Accumulator[T] = { | ||
| : Accumulator[T] = { | ||
| val acc = new Accumulator(initialValue, param, Some(name)) | ||
| cleaner.foreach(_.registerAccumulatorForCleanup(acc)) | ||
| acc | ||
| } | ||
|
|
||
| /** | ||
| * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display | ||
|
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. Should probably give this a different Scaladoc in order to differentiate it from the non-internal Accumulator and explain what's different.
Contributor
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. Will also document that |
||
| * in the Spark UI, and which reports its values to the driver via heartbeats. Tasks can "add" | ||
| * values to the accumulator using the `+=` method. Only the driver can access the accumulator's | ||
| * `value`. | ||
| */ | ||
| private[spark] def internalAccumulator[T](initialValue: T, name: String)( | ||
| implicit param: AccumulatorParam[T]): Accumulator[T] = { | ||
| val acc = new Accumulator(initialValue, param, Some(name), true) | ||
| cleaner.foreach(_.registerAccumulatorForCleanup(acc)) | ||
| acc | ||
| } | ||
|
|
||
| /** | ||
| * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values | ||
| * with `+=`. Only the driver can access the accumuable's `value`. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,7 +19,7 @@ package org.apache.spark.sql.execution | |
|
|
||
| import scala.collection.mutable.ArrayBuffer | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.{Accumulator, Logging} | ||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.rdd.{RDD, RDDOperationScope} | ||
| import org.apache.spark.sql.SQLContext | ||
|
|
@@ -51,6 +51,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
|
|
||
| protected def sparkContext = sqlContext.sparkContext | ||
|
|
||
| protected val metricToAccumulator = Map( | ||
| "numTuples"->sparkContext.internalAccumulator(0L, "number of tuples")) | ||
|
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. style: space around
Contributor
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 |
||
|
|
||
| // sqlContext will be null when we are being deserialized on the slaves. In this instance | ||
| // the value of codegenEnabled will be set by the desserializer after the constructor has run. | ||
| val codegenEnabled: Boolean = if (sqlContext != null) { | ||
|
|
@@ -91,6 +94,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ | |
| */ | ||
| def canProcessSafeRows: Boolean = true | ||
|
|
||
| /** | ||
| * Returns accumulators for metrics. The key of the Map is the metric's name and the value is the | ||
| * current value of the metric. | ||
| */ | ||
| def accumulators: Map[String, Accumulator[_]] = metricToAccumulator | ||
|
|
||
| /** | ||
| * Returns the result of this query as an RDD[InternalRow] by delegating to doExecute | ||
| * after adding query plan information to created RDDs for visualization. | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,20 +17,19 @@ | |
|
|
||
| package org.apache.spark.sql.execution | ||
|
|
||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.annotation.DeveloperApi | ||
| import org.apache.spark.rdd.{RDD, ShuffledRDD} | ||
| import org.apache.spark.shuffle.sort.SortShuffleManager | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.CatalystTypeConverters | ||
| import org.apache.spark.sql.Row | ||
| import org.apache.spark.sql.catalyst.errors._ | ||
| import org.apache.spark.sql.catalyst.expressions._ | ||
| import org.apache.spark.sql.Row | ||
| import org.apache.spark.sql.catalyst.plans.physical._ | ||
| import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} | ||
| import org.apache.spark.sql.types.StructType | ||
| import org.apache.spark.util.collection.ExternalSorter | ||
| import org.apache.spark.util.collection.unsafe.sort.PrefixComparator | ||
| import org.apache.spark.util.{CompletionIterator, MutablePair} | ||
| import org.apache.spark.{HashPartitioner, SparkEnv} | ||
| import org.apache.spark.{Accumulator, HashPartitioner, SparkEnv} | ||
|
|
||
| /** | ||
| * :: DeveloperApi :: | ||
|
|
@@ -43,7 +42,9 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends | |
|
|
||
| protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => | ||
| val reusableProjection = buildProjection() | ||
| iter.map(reusableProjection) | ||
| val result = iter.map(reusableProjection) | ||
| accumulators("numTuples").asInstanceOf[Accumulator[Long]] += result.length | ||
|
||
| result | ||
| } | ||
|
|
||
| override def outputOrdering: Seq[SortOrder] = child.outputOrdering | ||
|
|
||
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.
should we make this constructor
private[spark]so users can't setinternal = true?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.
+1
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.
OK.