Skip to content
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

Closed
wants to merge 48 commits into from
Closed
Show file tree
Hide file tree
Changes from 13 commits
Commits
Show all changes
48 commits
Select commit Hold shift + click to select a range
deb9a3f
Added metrics
tdas Sep 30, 2016
f3c5dcb
Merge remote-tracking branch 'apache-github/master' into HEAD
tdas Sep 30, 2016
6106a9d
Mima excludes
tdas Sep 30, 2016
3c0c7df
Added queryStatus, and state metric tests in listener
tdas Sep 30, 2016
3b063a2
Fixed unit test
tdas Sep 30, 2016
b31b606
More Mima excludes
tdas Sep 30, 2016
e5a5b7e
Added tests, and addressed some comments.
tdas Oct 3, 2016
348b59e
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 3, 2016
43e1ab1
Addressed comments, and added more tests
tdas Oct 4, 2016
62f5b0b
Fix polling delay
tdas Oct 4, 2016
4074428
Added synchronization
tdas Oct 4, 2016
2bcbcd2
Renamed *info to *status for consistency, except StreamingQueryInfo
tdas Oct 4, 2016
2d00f26
Added exclude
tdas Oct 4, 2016
d69935c
Added latency.sourceGetBatch
tdas Oct 5, 2016
05f22d7
Made sure status objects are private[sql]
tdas Oct 5, 2016
bbd0d8b
Addressed comments
tdas Oct 5, 2016
02603c7
Minor
tdas Oct 5, 2016
9fd6815
Fixed unnecessary mima change
tdas Oct 5, 2016
f5732a5
Addressed more comments
tdas Oct 5, 2016
e708b3b
Added periodic warning
tdas Oct 5, 2016
a47c73a
Remove unnecessary test
tdas Oct 5, 2016
f883b48
Removed locks
tdas Oct 5, 2016
713f773
Removed output rate, renamed StreamingQueryInfo -> StreamingQueryStat…
tdas Oct 6, 2016
bad5644
Added Mima excludes
tdas Oct 6, 2016
8bb554b
More refactoring, and file num rows test
tdas Oct 6, 2016
dfdca0c
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 6, 2016
2918525
Added tests for sources
tdas Oct 7, 2016
10d1c24
Fixed mima
tdas Oct 7, 2016
8537783
Pretty strings from statuses
tdas Oct 7, 2016
1655c96
Fixed mima and improved status desc
tdas Oct 7, 2016
3984d54
Added python APIs for StreamingQueryStatus, SourceStatus, SinkStatus
tdas Oct 7, 2016
5e0aa57
Fixed StreamingQuerySuite
tdas Oct 7, 2016
e8ebfc6
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 10, 2016
b747b23
Fixed python 3.4 tests
tdas Oct 11, 2016
dca9939
Addressed comments
tdas Oct 11, 2016
a7e20cc
Merge remote-tracking branch 'apache-github/master' into SPARK-17731
tdas Oct 11, 2016
af5854c
Fixed style
tdas Oct 11, 2016
3d7c71a
Fixed more issues
tdas Oct 11, 2016
8b4bce8
One more comment
tdas Oct 11, 2016
35bf508
Fixed python 3 tests
tdas Oct 12, 2016
4c08d56
Used conf correctly
tdas Oct 12, 2016
38ac35e
Addressed some comments
tdas Oct 12, 2016
42bc7bf
Addressed comments
tdas Oct 12, 2016
00a7415
Fixed test
tdas Oct 13, 2016
cafbeb7
Addressed more comments
tdas Oct 13, 2016
49da4b1
Fixed trigger details bug
tdas Oct 13, 2016
839d402
Removed unnecessary imports
tdas Oct 13, 2016
5972212
Updated python docs
tdas Oct 13, 2016
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 8 additions & 1 deletion project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,14 @@ object MimaExcludes {
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.getFunction"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.databaseExists"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.tableExists"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists")
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.columnExists"),
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

?

Copy link
Contributor Author

@tdas tdas Oct 5, 2016

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.


// [SPARK-17731] Metrics for structured streaming
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"),
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo.this"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status")
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,6 +164,13 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product {
ret
}

/**
* Returns a Seq containing the leaves in this tree.
*/
def collectLeaves(): Seq[BaseType] = {
this.collect { case p if p.children.isEmpty => p }
}

/**
* Finds and returns the first [[TreeNode]] of the tree for which the given partial function
* is defined (pre-order), and applies the partial function to it.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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"))
Copy link
Contributor

@frreiss frreiss Oct 4, 2016

Choose a reason for hiding this comment

The 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.
Edit: On second thought, looks like these string literals are for internal use, and the user-visible constants are at the bottom of StreamMetrics.scala.


override protected def doExecute(): RDD[InternalRow] = {
Copy link
Contributor

@brkyvz brkyvz Oct 11, 2016

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you don't need to initialize metrics this time like below?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

longMetrics("...") forces metrics to be initialized.

val numOutputRows = longMetric("numOutputRows")

child.execute().mapPartitionsWithStateStore(
getStateId.checkpointLocation,
operatorId = getStateId.operatorId,
Expand All @@ -69,6 +75,7 @@ case class StateStoreRestoreExec(
iter.flatMap { row =>
val key = getKey(row)
val savedState = store.get(key)
numOutputRows += 1
row +: savedState.toSeq
}
}
Expand All @@ -86,7 +93,13 @@ case class StateStoreSaveExec(
child: SparkPlan)
extends execution.UnaryExecNode with StatefulOperator {

override lazy val metrics = Map(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why is this lazy?

Copy link
Contributor Author

Choose a reason for hiding this comment

The 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 _
Expand All @@ -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
Expand All @@ -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
}
}
Expand All @@ -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]
}
}
}
Loading