-
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 1 commit
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 |
---|---|---|
|
@@ -17,8 +17,6 @@ | |
|
||
package org.apache.spark.sql.execution.streaming | ||
|
||
import scala.math.max | ||
|
||
import org.apache.spark.rdd.RDD | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} | ||
|
@@ -28,24 +26,48 @@ import org.apache.spark.sql.types.MetadataBuilder | |
import org.apache.spark.unsafe.types.CalendarInterval | ||
import org.apache.spark.util.AccumulatorV2 | ||
|
||
/** Tracks the maximum positive long seen. */ | ||
class MaxLong(protected var currentValue: Long = 0) | ||
extends AccumulatorV2[Long, Long] { | ||
/** Class for collecting event time stats with an accumulator */ | ||
case class EventTimeStats(var max: Long, var min: Long, var sum: Long, var count: Long) { | ||
def add(eventTime: Long): Unit = { | ||
this.max = math.max(this.max, eventTime) | ||
this.min = math.min(this.min, eventTime) | ||
this.sum += eventTime | ||
this.count += 1 | ||
} | ||
|
||
def merge(that: EventTimeStats): Unit = { | ||
this.max = math.max(this.max, that.max) | ||
this.min = math.min(this.min, that.min) | ||
this.sum += that.sum | ||
this.count += that.count | ||
} | ||
|
||
def avg: Long = (sum.toDouble / count).toLong | ||
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. nit: why not use |
||
} | ||
|
||
object EventTimeStats { | ||
def zero: EventTimeStats = EventTimeStats( | ||
max = Long.MinValue, min = Long.MaxValue, sum = 0L, count = 0L) | ||
} | ||
|
||
/** Accumulator that collects stats on event time in a batch. */ | ||
class EventTimeStatsAccum(protected var currentStats: EventTimeStats = EventTimeStats.zero) | ||
extends AccumulatorV2[Long, EventTimeStats] { | ||
|
||
override def isZero: Boolean = value == 0 | ||
override def value: Long = currentValue | ||
override def copy(): AccumulatorV2[Long, Long] = new MaxLong(currentValue) | ||
override def isZero: Boolean = value == EventTimeStats.zero | ||
override def value: EventTimeStats = currentStats | ||
override def copy(): AccumulatorV2[Long, EventTimeStats] = new EventTimeStatsAccum(currentStats) | ||
|
||
override def reset(): Unit = { | ||
currentValue = 0 | ||
currentStats = EventTimeStats.zero | ||
} | ||
|
||
override def add(v: Long): Unit = { | ||
currentValue = max(v, value) | ||
currentStats.add(v) | ||
} | ||
|
||
override def merge(other: AccumulatorV2[Long, Long]): Unit = { | ||
currentValue = max(value, other.value) | ||
override def merge(other: AccumulatorV2[Long, EventTimeStats]): Unit = { | ||
currentStats.merge(other.value) | ||
} | ||
} | ||
|
||
|
@@ -61,15 +83,14 @@ case class EventTimeWatermarkExec( | |
delay: CalendarInterval, | ||
child: SparkPlan) extends SparkPlan { | ||
|
||
// TODO: Use Spark SQL Metrics? | ||
val maxEventTime = new MaxLong | ||
sparkContext.register(maxEventTime) | ||
val eventTimeStats = new EventTimeStatsAccum() | ||
sparkContext.register(eventTimeStats) | ||
|
||
override protected def doExecute(): RDD[InternalRow] = { | ||
child.execute().mapPartitions { iter => | ||
val getEventTime = UnsafeProjection.create(eventTime :: Nil, child.output) | ||
iter.map { row => | ||
maxEventTime.add(getEventTime(row).getLong(0)) | ||
eventTimeStats.add((getEventTime(row).getLong(0).toDouble / 1000).toLong) | ||
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. nit: could be |
||
row | ||
} | ||
} | ||
|
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._ | ||
|
@@ -31,27 +32,6 @@ import org.json4s.jackson.JsonMethods._ | |
import org.apache.spark.annotation.Experimental | ||
import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
|
||
/** | ||
* :: Experimental :: | ||
* Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. | ||
*/ | ||
@Experimental | ||
class StateOperatorProgress private[sql]( | ||
val numRowsTotal: Long, | ||
val numRowsUpdated: Long) { | ||
|
||
/** The compact JSON representation of this progress. */ | ||
def json: String = compact(render(jsonValue)) | ||
|
||
/** The pretty (i.e. indented) JSON representation of this progress. */ | ||
def prettyJson: String = pretty(render(jsonValue)) | ||
|
||
private[sql] def jsonValue: JValue = { | ||
("numRowsTotal" -> JInt(numRowsTotal)) ~ | ||
("numRowsUpdated" -> JInt(numRowsUpdated)) | ||
} | ||
} | ||
|
||
/** | ||
* :: Experimental :: | ||
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 below to keep the StreamingQueryProgress class first in the file. More important code first. 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. Thats actually the opposite of how most code in SQL is laid out, so I think it would be better to avoid this change. The logic here is declarations that are use later should come first (references before declaration make it harder to read), and stuff at the end of the file is kind of hidden. 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. aah, then for consistency, SourceProgress and SinkProgress should also be before StreamingQueryProgress. But thats a bigger change should be done in a different PR. |
||
* Information about progress made in the execution of a [[StreamingQuery]] during | ||
|
@@ -61,13 +41,13 @@ 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 triggerTimestamp Timestamp (ms) of the beginning of the trigger. | ||
* @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 queryTimestamps Statistics of event time seen in this batch | ||
* @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 | ||
|
@@ -77,10 +57,10 @@ class StreamingQueryProgress private[sql]( | |
val id: UUID, | ||
val runId: UUID, | ||
val name: String, | ||
val timestamp: String, | ||
val triggerTimestamp: String, | ||
val batchId: Long, | ||
val durationMs: ju.Map[String, java.lang.Long], | ||
val currentWatermark: Long, | ||
val durationMs: ju.Map[String, JLong], | ||
val queryTimestamps: ju.Map[String, String], | ||
val stateOperators: Array[StateOperatorProgress], | ||
val sources: Array[SourceProgress], | ||
val sink: SinkProgress) { | ||
|
@@ -107,18 +87,23 @@ class StreamingQueryProgress private[sql]( | |
if (value.isNaN || value.isInfinity) JNothing else JDouble(value) | ||
} | ||
|
||
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)) } | ||
.foldLeft("" -> JNothing: JObject)(_ ~ _) | ||
} | ||
|
||
("id" -> JString(id.toString)) ~ | ||
("runId" -> JString(runId.toString)) ~ | ||
("name" -> JString(name)) ~ | ||
("timestamp" -> JString(timestamp)) ~ | ||
("triggerTimestamp" -> JString(triggerTimestamp)) ~ | ||
("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))) ~ | ||
("queryTimestamps" -> safeMapToJValue[String](queryTimestamps, s => JString(s))) ~ | ||
("stateOperators" -> JArray(stateOperators.map(_.jsonValue).toList)) ~ | ||
("sources" -> JArray(sources.map(_.jsonValue).toList)) ~ | ||
("sink" -> sink.jsonValue) | ||
|
@@ -200,3 +185,24 @@ class SinkProgress protected[sql]( | |
("description" -> JString(description)) | ||
} | ||
} | ||
|
||
/** | ||
* :: Experimental :: | ||
* Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. | ||
*/ | ||
@Experimental | ||
class StateOperatorProgress private[sql]( | ||
val numRowsTotal: Long, | ||
val numRowsUpdated: Long) { | ||
|
||
/** The compact JSON representation of this progress. */ | ||
def json: String = compact(render(jsonValue)) | ||
|
||
/** The pretty (i.e. indented) JSON representation of this progress. */ | ||
def prettyJson: String = pretty(render(jsonValue)) | ||
|
||
private[sql] def jsonValue: JValue = { | ||
("numRowsTotal" -> JInt(numRowsTotal)) ~ | ||
("numRowsUpdated" -> JInt(numRowsUpdated)) | ||
} | ||
} |
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