-
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-17731][SQL][STREAMING] Metrics for structured streaming #15307
Changes from 13 commits
deb9a3f
f3c5dcb
6106a9d
3c0c7df
3b063a2
b31b606
e5a5b7e
348b59e
43e1ab1
62f5b0b
4074428
2bcbcd2
2d00f26
d69935c
05f22d7
bbd0d8b
02603c7
9fd6815
f5732a5
e708b3b
a47c73a
f883b48
713f773
bad5644
8bb554b
dfdca0c
2918525
10d1c24
8537783
1655c96
3984d54
5e0aa57
e8ebfc6
b747b23
dca9939
a7e20cc
af5854c
3d7c71a
8b4bce8
35bf508
4c08d56
38ac35e
42bc7bf
00a7415
cafbeb7
49da4b1
839d402
5972212
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 |
---|---|---|
|
@@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.errors._ | |
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection | ||
import org.apache.spark.sql.execution | ||
import org.apache.spark.sql.execution.metric.SQLMetrics | ||
import org.apache.spark.sql.execution.streaming.state._ | ||
import org.apache.spark.sql.execution.SparkPlan | ||
|
||
|
@@ -56,7 +57,12 @@ case class StateStoreRestoreExec( | |
child: SparkPlan) | ||
extends execution.UnaryExecNode with StatefulOperator { | ||
|
||
override lazy val metrics = Map( | ||
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) | ||
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 metric names should probably be in a separate, centralized list of constants. Users will want a single place in the API docs to find a list of all available metrics, and the list is likely to change quite frequently as Structured Streaming evolves. |
||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
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. you don't need to initialize 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 numOutputRows = longMetric("numOutputRows") | ||
|
||
child.execute().mapPartitionsWithStateStore( | ||
getStateId.checkpointLocation, | ||
operatorId = getStateId.operatorId, | ||
|
@@ -69,6 +75,7 @@ case class StateStoreRestoreExec( | |
iter.flatMap { row => | ||
val key = getKey(row) | ||
val savedState = store.get(key) | ||
numOutputRows += 1 | ||
row +: savedState.toSeq | ||
} | ||
} | ||
|
@@ -86,7 +93,13 @@ case class StateStoreSaveExec( | |
child: SparkPlan) | ||
extends execution.UnaryExecNode with StatefulOperator { | ||
|
||
override 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. Why is this lazy? 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 just followed the convention - https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala#L39 I am not sure but I guess it was to delay the creation of accumulators until the plan is actually executed. |
||
"numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), | ||
"numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), | ||
"numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows")) | ||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
metrics // force lazy init at driver | ||
assert(returnAllStates.nonEmpty, | ||
"Incorrect planning in IncrementalExecution, returnAllStates have not been set") | ||
val saveAndReturnFunc = if (returnAllStates.get) saveAndReturnAll _ else saveAndReturnUpdated _ | ||
|
@@ -111,13 +124,18 @@ case class StateStoreSaveExec( | |
private def saveAndReturnUpdated( | ||
store: StateStore, | ||
iter: Iterator[InternalRow]): Iterator[InternalRow] = { | ||
val numOutputRows = longMetric("numOutputRows") | ||
val numTotalStateRows = longMetric("numTotalStateRows") | ||
val numUpdatedStateRows = longMetric("numUpdatedStateRows") | ||
|
||
new Iterator[InternalRow] { | ||
private[this] val baseIterator = iter | ||
private[this] val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) | ||
|
||
override def hasNext: Boolean = { | ||
if (!baseIterator.hasNext) { | ||
store.commit() | ||
numTotalStateRows += store.numKeys() | ||
false | ||
} else { | ||
true | ||
|
@@ -128,6 +146,8 @@ case class StateStoreSaveExec( | |
val row = baseIterator.next().asInstanceOf[UnsafeRow] | ||
val key = getKey(row) | ||
store.put(key.copy(), row.copy()) | ||
numOutputRows += 1 | ||
numUpdatedStateRows += 1 | ||
row | ||
} | ||
} | ||
|
@@ -142,12 +162,21 @@ case class StateStoreSaveExec( | |
store: StateStore, | ||
iter: Iterator[InternalRow]): Iterator[InternalRow] = { | ||
val getKey = GenerateUnsafeProjection.generate(keyExpressions, child.output) | ||
val numOutputRows = longMetric("numOutputRows") | ||
val numTotalStateRows = longMetric("numTotalStateRows") | ||
val numUpdatedStateRows = longMetric("numUpdatedStateRows") | ||
|
||
while (iter.hasNext) { | ||
val row = iter.next().asInstanceOf[UnsafeRow] | ||
val key = getKey(row) | ||
store.put(key.copy(), row.copy()) | ||
numUpdatedStateRows += 1 | ||
} | ||
store.commit() | ||
store.iterator().map(_._2.asInstanceOf[InternalRow]) | ||
numTotalStateRows += store.numKeys() | ||
store.iterator().map { case (k, v) => | ||
numOutputRows += 1 | ||
v.asInstanceOf[InternalRow] | ||
} | ||
} | ||
} |
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.
?
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.
somehow happened through a master merge. Fixed.