-
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-18834][SS] Expose event time stats through StreamingQueryProgress #16258
Changes from all 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 |
---|---|---|
|
@@ -360,6 +360,24 @@ class StreamExecution( | |
if (hasNewData) { | ||
// Current batch timestamp in milliseconds | ||
offsetSeqMetadata.batchTimestampMs = triggerClock.getTimeMillis() | ||
// Update the eventTime watermark if we find one in the plan. | ||
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 moved this such that the watermark is updated before starting a batch, rather than after finishing a batch. This keeps batchWatermarkMs consistent with batchTimestampsMs, both are set before starting a batch, and reduces complexities in the ProgressReporter. 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. +1, this makes sense. |
||
if (lastExecution != null) { | ||
lastExecution.executedPlan.collect { | ||
case e: EventTimeWatermarkExec if e.eventTimeStats.value.count > 0 => | ||
logDebug(s"Observed event time stats: ${e.eventTimeStats.value}") | ||
e.eventTimeStats.value.max - e.delay.milliseconds | ||
}.headOption.foreach { newWatermarkMs => | ||
if (newWatermarkMs > offsetSeqMetadata.batchWatermarkMs) { | ||
logInfo(s"Updating eventTime watermark to: $newWatermarkMs ms") | ||
offsetSeqMetadata.batchWatermarkMs = newWatermarkMs | ||
} else { | ||
logDebug( | ||
s"Event time didn't move: $newWatermarkMs < " + | ||
s"${offsetSeqMetadata.batchWatermarkMs}") | ||
} | ||
} | ||
} | ||
|
||
updateStatusMessage("Writing offsets to log") | ||
reportTimeTaken("walCommit") { | ||
assert(offsetLog.add( | ||
|
@@ -462,21 +480,6 @@ class StreamExecution( | |
sink.addBatch(currentBatchId, nextBatch) | ||
} | ||
|
||
// Update the eventTime watermark if we find one in the plan. | ||
lastExecution.executedPlan.collect { | ||
case e: EventTimeWatermarkExec => | ||
logTrace(s"Maximum observed eventTime: ${e.maxEventTime.value}") | ||
(e.maxEventTime.value / 1000) - e.delay.milliseconds() | ||
}.headOption.foreach { newWatermark => | ||
if (newWatermark > offsetSeqMetadata.batchWatermarkMs) { | ||
logInfo(s"Updating eventTime watermark to: $newWatermark ms") | ||
offsetSeqMetadata.batchWatermarkMs = newWatermark | ||
} else { | ||
logTrace(s"Event time didn't move: $newWatermark < " + | ||
s"$offsetSeqMetadata.currentEventTimeWatermark") | ||
} | ||
} | ||
|
||
awaitBatchLock.lock() | ||
try { | ||
// Wake up any threads that are waiting for the stream to progress. | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -18,6 +18,7 @@ | |
package org.apache.spark.sql.streaming | ||
|
||
import java.{util => ju} | ||
import java.lang.{Long => JLong} | ||
import java.util.UUID | ||
|
||
import scala.collection.JavaConverters._ | ||
|
@@ -29,7 +30,6 @@ import org.json4s.JsonDSL._ | |
import org.json4s.jackson.JsonMethods._ | ||
|
||
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
|
||
/** | ||
* :: Experimental :: | ||
|
@@ -61,13 +61,20 @@ class StateOperatorProgress private[sql]( | |
* @param id An unique query id that persists across restarts. See `StreamingQuery.id()`. | ||
* @param runId A query id that is unique for every start/restart. See `StreamingQuery.runId()`. | ||
* @param name User-specified name of the query, null if not specified. | ||
* @param timestamp Timestamp (ms) of the beginning of the trigger. | ||
* @param timestamp Beginning time of the trigger in ISO8601 format, i.e. UTC timestamps. | ||
* @param batchId A unique id for the current batch of data being processed. Note that in the | ||
* case of retries after a failure a given batchId my be executed more than once. | ||
* Similarly, when there is no data to be processed, the batchId will not be | ||
* incremented. | ||
* @param durationMs The amount of time taken to perform various operations in milliseconds. | ||
* @param currentWatermark The current event time watermark in milliseconds | ||
* @param eventTime Statistics of event time seen in this batch. It may contain the following keys: | ||
* { | ||
* "max" -> "2016-12-05T20:54:20.827Z" // maximum event time seen in this trigger | ||
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. Hi all, I am just leaving a comment as a gentle reminder to note that we probably should replace
|
||
* "min" -> "2016-12-05T20:54:20.827Z" // minimum event time seen in this trigger | ||
* "avg" -> "2016-12-05T20:54:20.827Z" // average event time seen in this trigger | ||
* "watermark" -> "2016-12-05T20:54:20.827Z" // watermark used in this trigger | ||
* } | ||
* All timestamps are in ISO8601 format, i.e. UTC timestamps. | ||
* @param stateOperators Information about operators in the query that store state. | ||
* @param sources detailed statistics on data being read from each of the streaming sources. | ||
* @since 2.1.0 | ||
|
@@ -79,8 +86,8 @@ class StreamingQueryProgress private[sql]( | |
val name: String, | ||
val timestamp: String, | ||
val batchId: Long, | ||
val durationMs: ju.Map[String, java.lang.Long], | ||
val currentWatermark: Long, | ||
val durationMs: ju.Map[String, JLong], | ||
val eventTime: ju.Map[String, String], | ||
val stateOperators: Array[StateOperatorProgress], | ||
val sources: Array[SourceProgress], | ||
val sink: SinkProgress) { | ||
|
@@ -107,18 +114,22 @@ class StreamingQueryProgress private[sql]( | |
if (value.isNaN || value.isInfinity) JNothing else JDouble(value) | ||
} | ||
|
||
/** Convert map to JValue while handling empty maps. Also, this sorts the keys. */ | ||
def safeMapToJValue[T](map: ju.Map[String, T], valueToJValue: T => JValue): JValue = { | ||
if (map.isEmpty) return JNothing | ||
val keys = map.asScala.keySet.toSeq.sorted | ||
keys.map { k => k -> valueToJValue(map.get(k)) : JObject }.reduce(_ ~ _) | ||
} | ||
|
||
("id" -> JString(id.toString)) ~ | ||
("runId" -> JString(runId.toString)) ~ | ||
("name" -> JString(name)) ~ | ||
("timestamp" -> JString(timestamp)) ~ | ||
("numInputRows" -> JInt(numInputRows)) ~ | ||
("inputRowsPerSecond" -> safeDoubleToJValue(inputRowsPerSecond)) ~ | ||
("processedRowsPerSecond" -> safeDoubleToJValue(processedRowsPerSecond)) ~ | ||
("durationMs" -> durationMs | ||
.asScala | ||
.map { case (k, v) => k -> JInt(v.toLong): JObject } | ||
.reduce(_ ~ _)) ~ | ||
("currentWatermark" -> JInt(currentWatermark)) ~ | ||
("durationMs" -> safeMapToJValue[JLong](durationMs, v => JInt(v.toLong))) ~ | ||
("eventTime" -> safeMapToJValue[String](eventTime, s => JString(s))) ~ | ||
("stateOperators" -> JArray(stateOperators.map(_.jsonValue).toList)) ~ | ||
("sources" -> JArray(sources.map(_.jsonValue).toList)) ~ | ||
("sink" -> sink.jsonValue) | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -44,7 +44,12 @@ class StreamingQueryStatusAndProgressSuite extends SparkFunSuite { | |
| "durationMs" : { | ||
| "total" : 0 | ||
| }, | ||
| "currentWatermark" : 3, | ||
| "eventTime" : { | ||
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. awesome! |
||
| "avg" : "2016-12-05T20:54:20.827Z", | ||
| "max" : "2016-12-05T20:54:20.827Z", | ||
| "min" : "2016-12-05T20:54:20.827Z", | ||
| "watermark" : "2016-12-05T20:54:20.827Z" | ||
| }, | ||
| "stateOperators" : [ { | ||
| "numRowsTotal" : 0, | ||
| "numRowsUpdated" : 1 | ||
|
@@ -76,7 +81,6 @@ class StreamingQueryStatusAndProgressSuite extends SparkFunSuite { | |
| "durationMs" : { | ||
| "total" : 0 | ||
| }, | ||
| "currentWatermark" : 3, | ||
| "stateOperators" : [ { | ||
| "numRowsTotal" : 0, | ||
| "numRowsUpdated" : 1 | ||
|
@@ -134,7 +138,11 @@ object StreamingQueryStatusAndProgressSuite { | |
timestamp = "2016-12-05T20:54:20.827Z", | ||
batchId = 2L, | ||
durationMs = Map("total" -> 0L).mapValues(long2Long).asJava, | ||
currentWatermark = 3L, | ||
eventTime = Map( | ||
"max" -> "2016-12-05T20:54:20.827Z", | ||
"min" -> "2016-12-05T20:54:20.827Z", | ||
"avg" -> "2016-12-05T20:54:20.827Z", | ||
"watermark" -> "2016-12-05T20:54:20.827Z").asJava, | ||
stateOperators = Array(new StateOperatorProgress(numRowsTotal = 0, numRowsUpdated = 1)), | ||
sources = Array( | ||
new SourceProgress( | ||
|
@@ -156,7 +164,7 @@ object StreamingQueryStatusAndProgressSuite { | |
timestamp = "2016-12-05T20:54:20.827Z", | ||
batchId = 2L, | ||
durationMs = Map("total" -> 0L).mapValues(long2Long).asJava, | ||
currentWatermark = 3L, | ||
eventTime = Map.empty[String, String].asJava, // empty maps should be handled correctly | ||
stateOperators = Array(new StateOperatorProgress(numRowsTotal = 0, numRowsUpdated = 1)), | ||
sources = Array( | ||
new SourceProgress( | ||
|
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.
nit: please document the time unit.
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.
Documented it in EventTimeWatermarkExec