From deb9a3fc1dde122691dc3d309cbb737cb4e0f9b9 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 29 Sep 2016 20:22:19 -0700 Subject: [PATCH 01/43] Added metrics --- .../streaming/StatefulAggregate.scala | 30 ++- .../execution/streaming/StreamExecution.scala | 163 +++++++++--- .../execution/streaming/StreamMetrics.scala | 250 ++++++++++++++++++ .../sql/execution/streaming/memory.scala | 26 +- .../state/HDFSBackedStateStoreProvider.scala | 2 + .../streaming/state/StateStore.scala | 3 + .../spark/sql/streaming/SinkStatus.scala | 6 +- .../spark/sql/streaming/SourceStatus.scala | 8 +- .../sql/streaming/StreamingQueryInfo.scala | 11 +- .../streaming/StreamingMetricsSuite.scala | 95 +++++++ .../streaming/state/StateStoreSuite.scala | 5 + .../spark/sql/streaming/StreamTest.scala | 1 + .../streaming/StreamingAggregationSuite.scala | 54 ++++ .../StreamingQueryListenerSuite.scala | 187 ++++++++++--- .../sql/streaming/StreamingQuerySuite.scala | 39 ++- 15 files changed, 777 insertions(+), 103 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index 4d0283fbef1d0..b27ece848c98c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -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")) + override protected def doExecute(): RDD[InternalRow] = { + 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,6 +93,11 @@ case class StateStoreSaveExec( child: SparkPlan) extends execution.UnaryExecNode with StatefulOperator { + override lazy val metrics = Map( + "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] = { assert(returnAllStates.nonEmpty, "Incorrect planning in IncrementalExecution, returnAllStates have not been set") @@ -111,6 +123,10 @@ 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) @@ -118,6 +134,7 @@ case class StateStoreSaveExec( override def hasNext: Boolean = { if (!baseIterator.hasNext) { store.commit() + numTotalStateRows += store.numKeys() false } else { true @@ -128,6 +145,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 +161,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] + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 66fb5a4bdeb7f..57fa29da4d8dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ @@ -57,6 +57,7 @@ class StreamExecution( extends StreamingQuery with Logging { import org.apache.spark.sql.streaming.StreamingQueryListener._ + import StreamMetrics._ private val pollingDelayMs = sparkSession.conf.get(SQLConf.STREAMING_POLLING_DELAY) @@ -105,11 +106,14 @@ class StreamExecution( var lastExecution: QueryExecution = null @volatile - var streamDeathCause: StreamingQueryException = null + private var streamDeathCause: StreamingQueryException = null /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() + private val streamMetrics = new StreamMetrics(uniqueSources.toSet, triggerClock, + "%s.StructuredStreamingMetrics.%s".format(sparkSession.sparkContext.appName, name)) + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using @@ -140,12 +144,23 @@ class StreamExecution( override def sourceStatuses: Array[SourceStatus] = { val localAvailableOffsets = availableOffsets sources.map(s => - new SourceStatus(s.toString, localAvailableOffsets.get(s).map(_.toString))).toArray + new SourceStatus( + s.toString, + localAvailableOffsets.get(s).map(_.toString), + streamMetrics.currentSourceInputRate(s), + streamMetrics.currentSourceProcessingRate(s), + streamMetrics.currentSourceTriggerInfo(s)) + ).toArray } /** Returns current status of the sink. */ - override def sinkStatus: SinkStatus = - new SinkStatus(sink.toString, committedOffsets.toCompositeOffset(sources).toString) + override def sinkStatus: SinkStatus = { + new SinkStatus( + sink.toString, + committedOffsets.toCompositeOffset(sources).toString, + streamMetrics.currentOutputRate()) + } + /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ override def exception: Option[StreamingQueryException] = Option(streamDeathCause) @@ -176,6 +191,7 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE + sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) postEvent(new QueryStarted(this.toInfo)) // Assumption: Does not throw exception. // Unblock starting thread @@ -185,25 +201,36 @@ class StreamExecution( SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { - if (isActive) { - if (currentBatchId < 0) { - // We'll do this initialization only once - populateStartOffsets() - logDebug(s"Stream running from $committedOffsets to $availableOffsets") + streamMetrics.reportTriggerStarted(currentBatchId) + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Finding new data from sources") + val isTerminated = timeIt(TRIGGER_LATENCY) { + if (isActive) { + if (currentBatchId < 0) { + // We'll do this initialization only once + populateStartOffsets() + logDebug(s"Stream running from $committedOffsets to $availableOffsets") + } else { + constructNextBatch() + } + if (dataAvailable) { + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Processing new data") + streamMetrics.reportTriggerInfo(DATA_AVAILABLE, true) + runBatch() + // We'll increase currentBatchId after we complete processing current batch's data + currentBatchId += 1 + } else { + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "No new data") + streamMetrics.reportTriggerInfo(DATA_AVAILABLE, false) + Thread.sleep(100) + } + true } else { - constructNextBatch() + false } - if (dataAvailable) { - runBatch() - // We'll increase currentBatchId after we complete processing current batch's data - currentBatchId += 1 - } else { - Thread.sleep(pollingDelayMs) - } - true - } else { - false } + streamMetrics.reportTriggerFinished() + postEvent(new QueryProgress(this.toInfo)) + isTerminated }) } catch { case _: InterruptedException if state == TERMINATED => // interrupted by stop() @@ -217,6 +244,8 @@ class StreamExecution( } finally { state = TERMINATED sparkSession.streams.notifyQueryTermination(StreamExecution.this) + streamMetrics.stop() + sparkSession.sparkContext.env.metricsSystem.removeSource(streamMetrics) postEvent(new QueryTerminated( this.toInfo, exception.map(_.getMessage), @@ -276,8 +305,14 @@ class StreamExecution( val hasNewData = { awaitBatchLock.lock() try { - val newData = uniqueSources.flatMap(s => s.getOffset.map(o => s -> o)) - availableOffsets ++= newData + timeIt(GET_OFFSET_LATENCY) { + val latestOffsets: Map[Source, Option[Offset]] = uniqueSources.map { s => + timeIt(s, SOURCE_GET_OFFSET_LATENCY) { + (s, s.getOffset) + } + }.toMap + availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) + } if (dataAvailable) { true @@ -290,9 +325,11 @@ class StreamExecution( } } if (hasNewData) { - assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), - s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") - logInfo(s"Committed offsets for batch $currentBatchId.") + timeIt(OFFSET_WAL_WRITE_LATENCY) { + assert(offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), + s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") + logInfo(s"Committed offsets for batch $currentBatchId.") + } } else { awaitBatchLock.lock() try { @@ -302,6 +339,7 @@ class StreamExecution( awaitBatchLock.unlock() } } + streamMetrics.reportTimestamp(GET_OFFSET_TIMESTAMP) } /** @@ -313,14 +351,18 @@ class StreamExecution( // TODO: Move this to IncrementalExecution. // Request unprocessed data from all sources. - val newData = availableOffsets.flatMap { - case (source, available) if committedOffsets.get(source).map(_ < available).getOrElse(true) => - val current = committedOffsets.get(source) - val batch = source.getBatch(current, available) - logDebug(s"Retrieving data from $source: $current -> $available") - Some(source -> batch) - case _ => None - }.toMap + val newData = timeIt(GET_BATCH_LATENCY) { + availableOffsets.flatMap { + case (source, available) + if committedOffsets.get(source).map(_ < available).getOrElse(true) => + val current = committedOffsets.get(source) + val batch = source.getBatch(current, available) + logDebug(s"Retrieving data from $source: $current -> $available") + Some(source -> batch) + case _ => None + } + } + streamMetrics.reportTimestamp(GET_BATCH_TIMESTAMP) // A list of attributes that will need to be updated. var replacements = new ArrayBuffer[(Attribute, Attribute)] @@ -353,13 +395,14 @@ class StreamExecution( checkpointFile("state"), currentBatchId) - lastExecution.executedPlan + val executedPlan = lastExecution.executedPlan // Force the lazy generation of execution plan val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000 logDebug(s"Optimized batch in ${optimizerTime}ms") val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId, nextBatch) + reportMetrics(executedPlan) awaitBatchLock.lock() try { @@ -373,7 +416,6 @@ class StreamExecution( logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. committedOffsets ++= availableOffsets - postEvent(new QueryProgress(this.toInfo)) } private def postEvent(event: StreamingQueryListener.Event) { @@ -503,12 +545,59 @@ class StreamExecution( """.stripMargin } + private def reportMetrics(executedPlan: SparkPlan): Unit = { + val execPlanLeaves = executedPlan.collect { case p if p.children.isEmpty => p } + val sourceToNumInputRows = if (execPlanLeaves.size == sources.size) { + sources.zip(execPlanLeaves).flatMap { case (s, leaf) => + leaf.metrics.get("numOutputRows").map { m => s -> m.value } + }.toMap + } else { + logWarning( + "Could not report metrics as number of sources did not match number of leaves in" + + s" execution plan: sources(${sources.size}) = ${sources.mkString(", ")};" + + s" leaves(${execPlanLeaves.size}) = ${execPlanLeaves.mkString(",")}") + Map.empty[Source, Long] + } + val numOutputRows = executedPlan.metrics.get("numOutputRows").map(_.value) + streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) + + val stateNodes = executedPlan.collect { case p if p.isInstanceOf[StateStoreSaveExec] => p } + stateNodes.zipWithIndex.foreach { case (s, i) => + streamMetrics.reportTriggerInfo(NUM_TOTAL_STATE_ROWS(i), + s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) + streamMetrics.reportTriggerInfo(NUM_UPDATED_STATE_ROWS(i), + s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) + } + } + + private def timeIt[T](triggerInfoKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + streamMetrics.reportLatency(triggerInfoKey, math.max(endTime - startTime, 0)) + result + } + + private def timeIt[T](source: Source, triggerInfoKey: String)(body: => T): T = { + val startTime = triggerClock.getTimeMillis() + val result = body + val endTime = triggerClock.getTimeMillis() + streamMetrics.reportLatency(source, triggerInfoKey, math.max(endTime - startTime, 0)) + result + } + private def toInfo: StreamingQueryInfo = { new StreamingQueryInfo( this.name, this.id, + triggerClock.getTimeMillis(), + streamMetrics.currentInputRate, + streamMetrics.currentProcessingRate, + streamMetrics.currentOutputRate, + streamMetrics.currentLatency, this.sourceStatuses, - this.sinkStatus) + this.sinkStatus, + streamMetrics.currentTriggerInfo) } trait State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala new file mode 100644 index 0000000000000..f5fe41586ae41 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -0,0 +1,250 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import scala.collection.mutable + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.internal.Logging +import org.apache.spark.metrics.source.{Source => CodahaleSource} +import org.apache.spark.util.Clock + +class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceName: String) + extends CodahaleSource with Logging { + + import StreamMetrics._ + + // Trigger infos + private val triggerInfo = new mutable.HashMap[String, String] + private val sourceTriggerInfo = new mutable.HashMap[Source, mutable.HashMap[String, String]] + + // Rate estimators for sources and sinks + private val inputRates = new mutable.HashMap[Source, RateCalculator] + private val processingRates = new mutable.HashMap[Source, RateCalculator] + private val outputRate = new RateCalculator + + // Number of input rows in the current trigger + private val numInputRows = new mutable.HashMap[Source, Long] + private var numOutputRows: Option[Long] = None + private var currentTriggerStartTimestamp: Long = -1 + private var previousTriggerStartTimestamp: Long = -1 + private var latency: Option[Double] = None + + override val sourceName: String = codahaleSourceName + override val metricRegistry: MetricRegistry = new MetricRegistry + + // =========== Initialization =========== + + registerGauge("inputRate.total", currentInputRate) + registerGauge("processingRate.total", () => currentProcessingRate) + registerGauge("outputRate.total", () => currentOutputRate) + registerGauge("latencyMs", () => currentLatency().getOrElse(-1.0)) + + sources.foreach { s => + inputRates.put(s, new RateCalculator) + processingRates.put(s, new RateCalculator) + sourceTriggerInfo.put(s, new mutable.HashMap[String, String]) + + registerGauge(s"inputRate.${s.toString}", () => currentSourceInputRate(s)) + registerGauge(s"processingRate.${s.toString}", () => currentSourceProcessingRate(s)) + } + + // =========== Setter methods =========== + + def reportTriggerStarted(triggerId: Long): Unit = synchronized { + numInputRows.clear() + numOutputRows = None + triggerInfo.clear() + sourceTriggerInfo.values.foreach(_.clear()) + + reportTriggerInfo(TRIGGER_ID, triggerId) + sources.foreach(s => reportSourceTriggerInfo(s, TRIGGER_ID, triggerId)) + reportTriggerInfo(ACTIVE, true) + currentTriggerStartTimestamp = triggerClock.getTimeMillis() + reportTriggerInfo(START_TIMESTAMP, currentTriggerStartTimestamp) + } + + def reportTimestamp(key: String): Unit = synchronized { + triggerInfo.put(key, triggerClock.getTimeMillis().toString) + } + + def reportLatency(key: String, latencyMs: Long): Unit = synchronized { + triggerInfo.put(key, latencyMs.toString) + } + + def reportLatency(source: Source, key: String, latencyMs: Long): Unit = synchronized { + sourceTriggerInfo(source).put(key, latencyMs.toString) + } + + def reportTriggerInfo[T](key: String, value: T): Unit = synchronized { + triggerInfo.put(key, value.toString) + } + + def reportSourceTriggerInfo[T](source: Source, key: String, value: T): Unit = synchronized { + sourceTriggerInfo(source).put(key, value.toString) + } + + def reportNumRows(inputRows: Map[Source, Long], outputRows: Option[Long]): Unit = synchronized { + numInputRows ++= inputRows + numOutputRows = outputRows + } + + def reportTriggerFinished(): Unit = synchronized { + require(currentTriggerStartTimestamp >= 0) + val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() + reportTriggerInfo(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) + reportTriggerInfo(STATUS_MESSAGE, "") + reportTriggerInfo(ACTIVE, false) + + // Report number of rows + val totalNumInputRows = numInputRows.values.sum + reportTriggerInfo(NUM_INPUT_ROWS, totalNumInputRows) + reportTriggerInfo(NUM_OUTPUT_ROWS, numOutputRows.getOrElse(0)) + numInputRows.foreach { case (s, r) => + reportSourceTriggerInfo(s, NUM_SOURCE_INPUT_ROWS, r) + } + + val currentTriggerDuration = currentTriggerFinishTimestamp - currentTriggerStartTimestamp + val previousInputIntervalOption = if (previousTriggerStartTimestamp >= 0) { + Some(currentTriggerStartTimestamp - previousTriggerStartTimestamp) + } else None + + // Update input rate = num rows received by each source during the previous trigger interval + // Interval is measures as interval between start times of previous and current trigger. + // + // TODO: Instead of trigger start, we should use time when getOffset was called on each source + // as this may be different for each source if there are many sources in the query plan + // and getOffset is called serially on them. + if (previousInputIntervalOption.nonEmpty) { + numInputRows.foreach { case (s, v) => + inputRates(s).update(v, previousInputIntervalOption.get) + } + } + + // Update processing rate = num rows processed for each source in current trigger duration + numInputRows.foreach { case (s, v) => + processingRates(s).update(v, currentTriggerDuration) + } + + // Update output rate = num rows output to the sink in current trigger duration + outputRate.update(numOutputRows.getOrElse(0), currentTriggerDuration) + logDebug("Output rate updated to " + outputRate.currentRate) + + // Update latency = if data present, 0.5 * previous trigger interval + current trigger duration + if (previousInputIntervalOption.nonEmpty && totalNumInputRows > 0) { + latency = Some((previousInputIntervalOption.get.toDouble / 2) + currentTriggerDuration) + } else { + latency = None + } + + previousTriggerStartTimestamp = currentTriggerStartTimestamp + currentTriggerStartTimestamp = -1 + } + + // =========== Getter methods =========== + + def currentInputRate(): Double = synchronized { + // Since we are calculating source input rates using the same time interval for all sources + // it is fine to calculate total input rate as the sum of per source input rate. + inputRates.map(_._2.currentRate).sum + } + + def currentSourceInputRate(source: Source): Double = synchronized { + inputRates(source).currentRate + } + + def currentProcessingRate(): Double = synchronized { + // Since we are calculating source processing rates using the same time interval for all sources + // it is fine to calculate total processing rate as the sum of per source processing rate. + processingRates.map(_._2.currentRate).sum + } + + def currentSourceProcessingRate(source: Source): Double = synchronized { + processingRates(source).currentRate + } + + def currentOutputRate(): Double = synchronized { outputRate.currentRate } + + def currentLatency(): Option[Double] = synchronized { latency } + + def currentTriggerInfo(): Map[String, String] = synchronized { triggerInfo.toMap } + + def currentSourceTriggerInfo(source: Source): Map[String, String] = synchronized { + sourceTriggerInfo(source).toMap + } + + // =========== Other methods =========== + + private def registerGauge[T](name: String, f: () => T)(implicit num: Numeric[T]): Unit = { + synchronized { + metricRegistry.register(name, new Gauge[T] { + override def getValue: T = f() + }) + } + } + + def stop(): Unit = synchronized { + inputRates.valuesIterator.foreach { _.stop() } + processingRates.valuesIterator.foreach { _.stop() } + outputRate.stop() + latency = None + } +} + +object StreamMetrics extends Logging { + + class RateCalculator { + @volatile private var rate: Option[Double] = None + + def update(numRows: Long, timeGapMs: Long): Unit = { + if (timeGapMs > 0) { + rate = Some(numRows.toDouble * 1000 / timeGapMs) + } else { + rate = None + logDebug(s"Rate updates cannot with zero or negative time gap $timeGapMs") + } + } + + def currentRate: Double = rate.getOrElse(0.0) + + def stop(): Unit = { rate = None } + } + + val TRIGGER_ID = "triggerId" + val ACTIVE = "isActive" + val DATA_AVAILABLE = "isDataAvailable" + val STATUS_MESSAGE = "statusMessage" + + val START_TIMESTAMP = "timestamp.triggerStart" + val GET_OFFSET_TIMESTAMP = "timestamp.afterGetOffset" + val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch" + val FINISH_TIMESTAMP = "timestamp.triggerFinish" + + val SOURCE_GET_OFFSET_LATENCY = "latency.sourceGetOffset" + val GET_OFFSET_LATENCY = "latency.getOffset" + val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" + val GET_BATCH_LATENCY = "latency.getBatch" + val TRIGGER_LATENCY = "latency.fullTrigger" + + val NUM_INPUT_ROWS = "numRows.input" + val NUM_OUTPUT_ROWS = "numRows.output" + val NUM_SOURCE_INPUT_ROWS = "numRows.sourceInput" + def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.$aggId.total" + def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.$aggId.updated" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index e37f0c77795c3..3e05194319fe7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -95,23 +95,27 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) /** * Returns the data that is between the offsets (`start`, `end`]. */ - override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + override def getBatch(start: Option[Offset], end: Offset): DataFrame = synchronized { val startOrdinal = start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1 val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 - val newBlocks = synchronized { batches.slice(startOrdinal, endOrdinal) } - - logDebug( - s"MemoryBatch [$startOrdinal, $endOrdinal]: ${newBlocks.flatMap(_.collect()).mkString(", ")}") - newBlocks - .map(_.toDF()) - .reduceOption(_ union _) - .getOrElse { - sys.error("No data selected!") - } + val batchData = synchronized { batches.slice(startOrdinal, endOrdinal) }.flatMap(_.collect()) + + logInfo( + s"MemoryBatch [$startOrdinal, $endOrdinal]: ${batchData.mkString(", ")}") + if (batchData.isEmpty) { sys.error("No data selected!") } + + // Merge data into a single logical plan node so that StreamExecution can + // match the number of leaf nodes with the number of sources for getting metrics + sqlContext.createDataset(batchData).toDF() } override def stop() {} + + def reset(): Unit = synchronized { + batches.clear() + currentOffset = new LongOffset(-1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 3335755fd3b67..dce53492f6156 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -197,6 +197,8 @@ private[state] class HDFSBackedStateStoreProvider( allUpdates.values().asScala.toIterator } + override def numKeys(): Long = mapToUpdate.size() + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index a67fdceb3cee6..7132e284c28f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -77,6 +77,9 @@ trait StateStore { */ def updates(): Iterator[StoreUpdate] + /** Number of keys in the state store */ + def numKeys(): Long + /** * Whether all updates have been committed */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index de1efe961f8bd..ac09d3f3bfc6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -26,9 +26,11 @@ import org.apache.spark.sql.execution.streaming.Sink * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current offset up to which data has been written by the sink + * @param outputRate Current output rate as rows / second * @since 2.0.0 */ @Experimental -class SinkStatus private[sql]( +case class SinkStatus private[sql]( val description: String, - val offsetDesc: String) + val offsetDesc: String, + val outputRate: Double) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index bd0c8485e4fdd..4322db248e88e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -26,9 +26,13 @@ import org.apache.spark.sql.execution.streaming.Source * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current [[Source]] offset if known + * @param inputRate Current ingestion rate as rows / second * @since 2.0.0 */ @Experimental -class SourceStatus private[sql] ( +case class SourceStatus private[sql]( val description: String, - val offsetDesc: Option[String]) + val offsetDesc: Option[String], + val inputRate: Double, + val processingRate: Double, + val triggerInfo: Map[String, String]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index 1af2668817eae..6a0c2c3cf9833 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -30,8 +30,15 @@ import org.apache.spark.annotation.Experimental * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. */ @Experimental -class StreamingQueryInfo private[sql]( +case class StreamingQueryInfo private[sql]( val name: String, val id: Long, + val timestamp: Long, + val inputRate: Double, + val processingRate: Double, + val outputRate: Double, + val latencyMs: Option[Double], val sourceStatuses: Seq[SourceStatus], - val sinkStatus: SinkStatus) + val sinkStatus: SinkStatus, + val triggerInfo: Map[String, String] +) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala new file mode 100644 index 0000000000000..a42df2c6bcd98 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala @@ -0,0 +1,95 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming + +import org.scalactic.TolerantNumerics + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.ManualClock + +class StreamingMetricsSuite extends SparkFunSuite { + + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + + test("all rates and latency") { + val m = newStreamMetrics() + assert(m.currentInputRate() === 0.0) + assert(m.currentProcessingRate() === 0.0) + assert(m.currentOutputRate() === 0.0) + assert(m.currentSourceInputRate(source) === 0.0) + assert(m.currentSourceProcessingRate(source) === 0.0) + assert(m.currentLatency() === None) + + // When trigger started, the rates should not change + m.reportTriggerStarted(1) + assert(m.currentInputRate() === 0.0) + assert(m.currentProcessingRate() === 0.0) + assert(m.currentOutputRate() === 0.0) + assert(m.currentSourceInputRate(source) === 0.0) + assert(m.currentSourceProcessingRate(source) === 0.0) + assert(m.currentLatency() === None) + + // Finishing the trigger should calculate the rates, except input rate which needs + // to have another trigger interval + m.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows + clock.advance(1000) + m.reportTriggerFinished() + assert(m.currentInputRate() === 0.0) + assert(m.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec + assert(m.currentOutputRate() === 10.0) // 10 output rows generated in 1 sec + assert(m.currentSourceInputRate(source) === 0.0) + assert(m.currentSourceProcessingRate(source) === 100.0) + assert(m.currentLatency() === None) + + // Another trigger should calculate the input rate + clock.advance(1000) + m.reportTriggerStarted(2) + m.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows + clock.advance(500) + m.reportTriggerFinished() + assert(m.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + assert(m.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + assert(m.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec + assert(m.currentSourceInputRate(source) === 100.0) + assert(m.currentSourceProcessingRate(source) === 400.0) + assert(m.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Rates should be set to 0 after stop + m.stop() + assert(m.currentInputRate() === 0.0) + assert(m.currentProcessingRate() === 0.0) + assert(m.currentOutputRate() === 0.0) + assert(m.currentSourceInputRate(source) === 0.0) + assert(m.currentSourceProcessingRate(source) === 0.0) + assert(m.currentLatency() === None) + } + + private def newStreamMetrics(): StreamMetrics = new StreamMetrics( + Set[Source](source), clock, "test") + + private val clock = new ManualClock() + private val source: Source = new Source { + override def schema: StructType = StructType(Array.empty[StructField]) + override def getOffset: Option[Offset] = Some(new LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { null } + override def stop() {} + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 984b84fd13fbd..06f1bd6c3bcc7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -74,6 +74,7 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Verify state after updating put(store, "a", 1) + assert(store.numKeys() === 1) intercept[IllegalStateException] { store.iterator() } @@ -85,7 +86,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth // Make updates, commit and then verify state put(store, "b", 2) put(store, "aa", 3) + assert(store.numKeys() === 3) remove(store, _.startsWith("a")) + assert(store.numKeys() === 1) assert(store.commit() === 1) assert(store.hasCommitted) @@ -107,7 +110,9 @@ class StateStoreSuite extends SparkFunSuite with BeforeAndAfter with PrivateMeth val reloadedProvider = new HDFSBackedStateStoreProvider( store.id, keySchema, valueSchema, StateStoreConf.empty, new Configuration) val reloadedStore = reloadedProvider.getStore(1) + assert(reloadedStore.numKeys() === 1) put(reloadedStore, "c", 4) + assert(reloadedStore.numKeys() === 2) assert(reloadedStore.commit() === 2) assert(rowsToSet(reloadedStore.iterator()) === Set("b" -> 2, "c" -> 4)) assert(getDataFromFiles(provider) === Set("b" -> 2, "c" -> 4)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index af2b58116b2aa..e67fb59f46c75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -297,6 +297,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { try { startedTest.foreach { action => + logInfo(s"Processing test stream action: $action") action match { case StartStream(trigger, triggerClock) => verify(currentStream == null, "stream already running") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 8681199817fe6..e59b5491f90b6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.InternalOutputModes._ +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.StateStore import org.apache.spark.sql.expressions.scalalang.typed @@ -129,6 +130,59 @@ class StreamingAggregationSuite extends StreamTest with BeforeAndAfterAll { ) } + test("state metrics") { + val inputData = MemoryStream[Int] + + val aggregated = + inputData.toDS() + .flatMap(x => Seq(x, x + 1)) + .toDF("value") + .groupBy($"value") + .agg(count("*")) + .as[(Int, Long)] + + implicit class RichStreamExecution(query: StreamExecution) { + def stateNodes: Seq[SparkPlan] = { + query.lastExecution.executedPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + } + } + + // Test with Update mode + testStream(aggregated, Update)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + + // Test with Complete mode + inputData.reset() + testStream(aggregated, Complete)( + AddData(inputData, 1), + CheckLastBatch((1, 1), (2, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AddData(inputData, 2, 3), + CheckLastBatch((1, 1), (2, 2), (3, 2), (4, 1)), + AssertOnQuery { _.stateNodes.size === 1 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 4 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + ) + } + test("multiple keys") { val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 7f4d28cf0598f..9d829ba5c8cf5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,18 +17,19 @@ package org.apache.spark.sql.streaming -import java.util.concurrent.ConcurrentLinkedQueue +import scala.collection.mutable +import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ import org.scalatest.concurrent.AsyncAssertions.Waiter import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.JsonProtocol +import org.apache.spark.util.{JsonProtocol, ManualClock} class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { @@ -36,6 +37,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ import StreamingQueryListener._ + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + after { spark.streams.active.foreach(_.stop()) assert(spark.streams.active.isEmpty) @@ -44,11 +48,18 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { spark.sparkContext.listenerBus.waitUntilEmpty(10000) } - test("single listener") { + test("single listener, check statuses") { val listener = new QueryStatusCollector val input = MemoryStream[Int] + + // This is to make sure that + // - Query takes non-zero time to compute + // - Exec plan ends with a node (filter) that supports the numOutputRows metric + spark.conf.set("spark.sql.codegen.wholeStage", false) + val df = input.toDS.map { x => Thread.sleep(10); x }.toDF("value").where("value != 0") + withListenerAdded(listener) { - testStream(input.toDS)( + testStream(df)( StartStream(), AssertOnQuery("Incorrect query status in onQueryStarted") { query => val status = listener.startStatus @@ -62,28 +73,35 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // batches have started assert(status.sourceStatuses(0).offsetDesc === None) assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString) + assert(status.sinkStatus.outputRate === 0.0) + + // The source and sink rates must be None as this must be called before the batches + // have started + assert(status.sourceStatuses(0).inputRate === 0.0) + assert(status.sourceStatuses(0).processingRate === 0.0) // No progress events or termination events - assert(listener.progressStatuses.isEmpty) assert(listener.terminationStatus === null) }, AddDataMemory(input, Seq(1, 2, 3)), CheckAnswer(1, 2, 3), AssertOnQuery("Incorrect query status in onQueryProgress") { query => eventually(Timeout(streamingTimeout)) { - - // There should be only on progress event as batch has been processed - assert(listener.progressStatuses.size === 1) - val status = listener.progressStatuses.peek() - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - - // No termination events - assert(listener.terminationStatus === null) + assert(listener.lastTriggerStatus.nonEmpty) } + // Check the correctness of data in the latest query info reported by onQueryProgress + val status = listener.lastTriggerStatus.get + assert(status != null) + assert(status.name === query.name) + assert(status.id === query.id) + assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) + assert(status.sourceStatuses(0).inputRate >= 0.0) // flaky if checked for == + assert(status.sourceStatuses(0).processingRate > 0.0) + assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) + assert(status.sinkStatus.outputRate !== 0.0) + + // No termination events + assert(listener.terminationStatus === null) }, StopStream, AssertOnQuery("Incorrect query status in onQueryTerminated") { query => @@ -93,7 +111,10 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.name === query.name) assert(status.id === query.id) assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) + assert(status.sourceStatuses(0).inputRate === 0.0) + assert(status.sourceStatuses(0).processingRate === 0.0) assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) + assert(status.sinkStatus.outputRate === 0.0) assert(listener.terminationStackTrace.isEmpty) assert(listener.terminationException === None) } @@ -103,6 +124,80 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } + test("single listener, trigger infos") { + import StreamingQueryListenerSuite._ + clock = new ManualClock() + + /** Custom MemoryStream that waits for manual clock to reach a time */ + val inputData = new MemoryStream[Int](0, sqlContext) { + // Wait for manual clock to be 100 first time there is data + override def getOffset: Option[Offset] = { + val offset = super.getOffset + if (offset.nonEmpty) { + clock.waitTillTime(100) + } + offset + } + + // Wait for manual clock to be 300 first time there is data + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + clock.waitTillTime(300) + super.getBatch(start, end) + } + } + + // This is to make sure that + // - Query waits for manual clock to be 600 first time there is data + // - Exec plan ends with a node (filter) that supports the numOutputRows metric + spark.conf.set("spark.sql.codegen.wholeStage", false) + val mapped = inputData.toDS().coalesce(1).map { x => + clock.waitTillTime(600) + x + }.where("value != 1") + + val listener = new QueryStatusCollector + withListenerAdded(listener) { + testStream(mapped)( + StartStream(triggerClock = clock), + AddData(inputData, 1, 2), + AdvanceManualClock(100), // unblock getOffset, will block on getBatch + AdvanceManualClock(200), // unblock getBatch, will block on computation + AdvanceManualClock(300), // unblock computation + AssertOnQuery("Incorrect trigger info") { query => + require(clock.getTimeMillis() === 600) + eventually(Timeout(streamingTimeout)) { + assert(listener.lastTriggerStatus.nonEmpty) + } + + // Check the correctness of the trigger info of the first completed batch reported by + // onQueryProgress + val status = listener.lastTriggerStatus.get + assert(status.triggerInfo("triggerId") == "0") + assert(status.triggerInfo("isActive") === "false") + + assert(status.triggerInfo("timestamp.triggerStart") === "0") + assert(status.triggerInfo("timestamp.afterGetOffset") === "100") + assert(status.triggerInfo("timestamp.afterGetBatch") === "300") + assert(status.triggerInfo("timestamp.triggerFinish") === "600") + + assert(status.triggerInfo("latency.getOffset") === "100") + assert(status.triggerInfo("latency.getBatch") === "200") + assert(status.triggerInfo("latency.offsetLogWrite") === "0") + assert(status.triggerInfo("latency.fullTrigger") === "600") + + assert(status.triggerInfo("numRows.input") === "2") + assert(status.triggerInfo("numRows.output") === "1") + + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerInfo("triggerId") === "0") + assert(status.sourceStatuses(0).triggerInfo("latency.sourceGetOffset") === "100") + assert(status.sourceStatuses(0).triggerInfo("numRows.sourceInput") === "2") + }, + CheckAnswer(2) + ) + } + } + test("adding and removing listener") { def isListenerActive(listener: QueryStatusCollector): Boolean = { listener.reset() @@ -147,7 +242,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - test("exception should be reported in QueryTerminated") { + testQuietly("exception should be reported in QueryTerminated") { val listener = new QueryStatusCollector withListenerAdded(listener) { val input = MemoryStream[Int] @@ -167,12 +262,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStartedInfo = new StreamingQueryInfo( - "name", - 1, - Seq(new SourceStatus("source1", None), new SourceStatus("source2", None)), - new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString)) - val queryStarted = new StreamingQueryListener.QueryStarted(queryStartedInfo) + val queryStarted = new StreamingQueryListener.QueryStarted(testQueryInfo) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] @@ -180,14 +270,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryProgress serialization") { - val queryProcessInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) - val queryProcess = new StreamingQueryListener.QueryProgress(queryProcessInfo) + val queryProcess = new StreamingQueryListener.QueryProgress(testQueryInfo) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] @@ -195,16 +278,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryTerminated serialization") { - val queryTerminatedInfo = new StreamingQueryInfo( - "name", - 1, - Seq( - new SourceStatus("source1", Some(LongOffset(0).toString)), - new SourceStatus("source2", Some(LongOffset(1).toString))), - new SinkStatus("sink", new CompositeOffset(Array(None, Some(LongOffset(1)))).toString)) val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( - queryTerminatedInfo, + testQueryInfo, Some(exception.getMessage), exception.getStackTrace) val json = @@ -239,7 +315,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { private def withListenerAdded(listener: StreamingQueryListener)(body: => Unit): Unit = { try { - failAfter(1 minute) { + failAfter(streamingTimeout) { spark.streams.addListener(listener) body } @@ -255,6 +331,16 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } + private val testQueryInfo: StreamingQueryInfo = { + new StreamingQueryInfo( + "name", 1, 123, 1.0, 2.0, 3.0, Some(345), + Seq( + new SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map.empty), + new SourceStatus("source2", Some(LongOffset(1).toString), 1.0, 2.0, Map("a" -> "b"))), + new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString, 2.0), + Map("a" -> "b")) + } + class QueryStatusCollector extends StreamingQueryListener { // to catch errors in the async listener events @volatile private var asyncTestWaiter = new Waiter @@ -264,7 +350,15 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { @volatile var terminationException: Option[String] = null @volatile var terminationStackTrace: Seq[StackTraceElement] = null - val progressStatuses = new ConcurrentLinkedQueue[StreamingQueryInfo] + private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryInfo] + + /** Get the info of the last trigger that processed data */ + def lastTriggerStatus: Option[StreamingQueryInfo] = synchronized { + progressStatuses.filter { i => + i.triggerInfo("isActive").toBoolean == false && + i.triggerInfo("isDataAvailable").toBoolean == true + }.lastOption + } def reset(): Unit = { startStatus = null @@ -287,7 +381,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { override def onQueryProgress(queryProgress: QueryProgress): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryProgress called before onQueryStarted") - progressStatuses.add(queryProgress.queryInfo) + synchronized { progressStatuses += queryProgress.queryInfo } } } @@ -302,3 +396,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } } + +object StreamingQueryListenerSuite { + // Singleton reference to clock that does not get serialized in task closures + @volatile var clock: ManualClock = null +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 9d58315c20031..496d5f60c1d03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.streaming +import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException -import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset, MemoryStream, StreamExecution} -import org.apache.spark.util.Utils +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.util.{ManualClock, Utils} class StreamingQuerySuite extends StreamTest with BeforeAndAfter { @@ -29,6 +30,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { import AwaitTerminationTester._ import testImplicits._ + // To make === between double tolerate inexact values + implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) + after { sqlContext.streams.active.foreach(_.stop()) } @@ -102,26 +106,53 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { testQuietly("source and sink statuses") { val inputData = MemoryStream[Int] - val mapped = inputData.toDS().map(6 / _) + + // This is make the sure the execution plan ends with a node (filter) that supports + // the numOutputRows metric. + spark.conf.set("spark.sql.codegen.wholeStage", false) + val mapped = inputData.toDS().map(6 / _).where("value > 0") testStream(mapped)( AssertOnQuery(_.sourceStatuses.length === 1), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.description.contains("Memory")), AssertOnQuery(_.sinkStatus.offsetDesc === new CompositeOffset(None :: Nil).toString), + AssertOnQuery(_.sinkStatus.outputRate === 0.0), + AddData(inputData, 1, 2), CheckAnswer(6, 3), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.sourceStatuses(0).inputRate > 0.0), // disable if flaky + AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), + AssertOnQuery(_.sinkStatus.outputRate >= 0.0), + AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.sourceStatuses(0).inputRate > 0.0), // disable if flaky + AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sinkStatus.outputRate >= 0.0), + + StopStream, + AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sinkStatus.outputRate === 0.0), + + StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), - AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) + AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.sinkStatus.outputRate === 0.0) ) } From 6106a9dd0133f4ab750acc3a5e0cc24101982147 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 30 Sep 2016 02:08:05 -0700 Subject: [PATCH 02/43] Mima excludes --- project/MimaExcludes.scala | 7 ++++++- .../spark/sql/execution/streaming/StreamExecution.scala | 1 - 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2ffe0ac9bc982..f943b9a18e9d6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -55,7 +55,12 @@ object MimaExcludes { 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.columnExists") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.columnExists"), + + // [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") ) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 46e38f10bce49..ff3c8b0fc1dee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -34,7 +34,6 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} From 3c0c7df1d5350ddb63f22f9eb3e7418e5f4e7683 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 30 Sep 2016 11:02:36 -0700 Subject: [PATCH 03/43] Added queryStatus, and state metric tests in listener --- .../execution/streaming/StreamExecution.scala | 9 ++- .../execution/streaming/StreamMetrics.scala | 8 +-- .../spark/sql/streaming/StreamingQuery.scala | 6 ++ .../StreamingQueryListenerSuite.scala | 15 +++-- .../sql/streaming/StreamingQuerySuite.scala | 60 ++++++++++++++++--- 5 files changed, 78 insertions(+), 20 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ff3c8b0fc1dee..cc813b9ca46e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -139,6 +139,10 @@ class StreamExecution( /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE + override def queryStatus: StreamingQueryInfo = { + this.toInfo + } + /** Returns current status of all the sources. */ override def sourceStatuses: Array[SourceStatus] = { val localAvailableOffsets = availableOffsets @@ -160,7 +164,6 @@ class StreamExecution( streamMetrics.currentOutputRate()) } - /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ override def exception: Option[StreamingQueryException] = Option(streamDeathCause) @@ -570,9 +573,9 @@ class StreamExecution( val stateNodes = executedPlan.collect { case p if p.isInstanceOf[StateStoreSaveExec] => p } stateNodes.zipWithIndex.foreach { case (s, i) => - streamMetrics.reportTriggerInfo(NUM_TOTAL_STATE_ROWS(i), + streamMetrics.reportTriggerInfo(NUM_TOTAL_STATE_ROWS(i + 1), s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) - streamMetrics.reportTriggerInfo(NUM_UPDATED_STATE_ROWS(i), + streamMetrics.reportTriggerInfo(NUM_UPDATED_STATE_ROWS(i + 1), s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index f5fe41586ae41..e3b8f5c25a8b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -242,9 +242,9 @@ object StreamMetrics extends Logging { val GET_BATCH_LATENCY = "latency.getBatch" val TRIGGER_LATENCY = "latency.fullTrigger" - val NUM_INPUT_ROWS = "numRows.input" + val NUM_INPUT_ROWS = "numRows.input.total" val NUM_OUTPUT_ROWS = "numRows.output" - val NUM_SOURCE_INPUT_ROWS = "numRows.sourceInput" - def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.$aggId.total" - def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.$aggId.updated" + val NUM_SOURCE_INPUT_ROWS = "numRows.input.source" + def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.total" + def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.updated" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 91f0a1e3446a1..cf16c4fd26743 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -62,6 +62,12 @@ trait StreamingQuery { */ def exception: Option[StreamingQueryException] + /** + * Returns the current status of the query. + * @since 2.1.0 + */ + def queryStatus: StreamingQueryInfo + /** * Returns current status of all the sources. * @since 2.0.0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 91c3c9595f6c4..fd5013cbf9149 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.functions._ import org.apache.spark.util.{JsonProtocol, ManualClock} @@ -126,7 +127,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - test("single listener, trigger infos") { + test("single listener, check trigger infos") { import StreamingQueryListenerSuite._ clock = new ManualClock() @@ -152,14 +153,14 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // - Query waits for manual clock to be 600 first time there is data // - Exec plan ends with a node (filter) that supports the numOutputRows metric spark.conf.set("spark.sql.codegen.wholeStage", false) - val mapped = inputData.toDS().coalesce(1).map { x => + val mapped = inputData.toDS().agg(count("*")).as[Long].coalesce(1).map { x => clock.waitTillTime(600) x - }.where("value != 1") + }.where("value != 100") val listener = new QueryStatusCollector withListenerAdded(listener) { - testStream(mapped)( + testStream(mapped, OutputMode.Complete)( StartStream(triggerClock = clock), AddData(inputData, 1, 2), AdvanceManualClock(100), // unblock getOffset, will block on getBatch @@ -187,13 +188,15 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.triggerInfo("latency.offsetLogWrite") === "0") assert(status.triggerInfo("latency.fullTrigger") === "600") - assert(status.triggerInfo("numRows.input") === "2") + assert(status.triggerInfo("numRows.input.total") === "2") assert(status.triggerInfo("numRows.output") === "1") + assert(status.triggerInfo("numRows.state.aggregation1.total") === "1") + assert(status.triggerInfo("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.size === 1) assert(status.sourceStatuses(0).triggerInfo("triggerId") === "0") assert(status.sourceStatuses(0).triggerInfo("latency.sourceGetOffset") === "100") - assert(status.sourceStatuses(0).triggerInfo("numRows.sourceInput") === "2") + assert(status.sourceStatuses(0).triggerInfo("numRows.input.source") === "2") true }, CheckAnswer(2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index fa9bd3567744b..515a0dda5c261 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.streaming import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.{ManualClock, Utils} @@ -104,7 +104,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } - testQuietly("source and sink statuses") { + testQuietly("statuses") { val inputData = MemoryStream[Int] // This is make the sure the execution plan ends with a node (filter) that supports @@ -113,7 +113,20 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { val mapped = inputData.toDS().map(6 / _).where("value > 0") testStream(mapped)( - AssertOnQuery(_.sourceStatuses.length === 1), + AssertOnQuery(q => q.queryStatus.name === q.name), + AssertOnQuery(q => q.queryStatus.id === q.id), + AssertOnQuery(_.queryStatus.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.queryStatus.inputRate === 0.0), + AssertOnQuery(_.queryStatus.processingRate === 0.0), + AssertOnQuery(_.queryStatus.outputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), + AssertOnQuery(_.queryStatus.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.queryStatus.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), + AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), @@ -124,21 +137,44 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { AddData(inputData, 1, 2), CheckAnswer(6, 3), + AssertOnQuery(_.queryStatus.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.queryStatus.inputRate >= 0.0), + AssertOnQuery(_.queryStatus.processingRate >= 0.0), + AssertOnQuery(_.queryStatus.outputRate >= 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), + AssertOnQuery(_.queryStatus.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate >= 0.0), + AssertOnQuery(_.queryStatus.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(0)).toString), + AssertOnQuery(_.queryStatus.sinkStatus.outputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), - AssertOnQuery(_.sourceStatuses(0).inputRate > 0.0), // disable if flaky + AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), AssertOnQuery(_.sinkStatus.outputRate >= 0.0), AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), + AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), - AssertOnQuery(_.sourceStatuses(0).inputRate > 0.0), // disable if flaky - AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sinkStatus.outputRate >= 0.0), StopStream, + AssertOnQuery(_.queryStatus.inputRate === 0.0), + AssertOnQuery(_.queryStatus.processingRate === 0.0), + AssertOnQuery(_.queryStatus.outputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), + AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), @@ -148,6 +184,16 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], + AssertOnQuery(_.queryStatus.inputRate === 0.0), + AssertOnQuery(_.queryStatus.processingRate === 0.0), + AssertOnQuery(_.queryStatus.outputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), + AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), From 3b063a2b9f5ff376bc821953f1df117aa786e3b4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 30 Sep 2016 11:22:57 -0700 Subject: [PATCH 04/43] Fixed unit test --- .../apache/spark/sql/execution/streaming/StatefulAggregate.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala index b27ece848c98c..587ea7d02acab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulAggregate.scala @@ -99,6 +99,7 @@ case class StateStoreSaveExec( "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 _ From b31b606855750f77c755a36c09c16e3a9fd0b521 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 30 Sep 2016 15:40:34 -0700 Subject: [PATCH 05/43] More Mima excludes --- project/MimaExcludes.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index f943b9a18e9d6..c0708848cdaa9 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,7 +60,8 @@ object MimaExcludes { // [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[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo.this"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.queryStatus") ) } From e5a5b7e6f57082a298951c3c57f586fd404db82e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 3 Oct 2016 04:15:02 -0700 Subject: [PATCH 06/43] Added tests, and addressed some comments. --- .../execution/streaming/StreamExecution.scala | 2 +- .../execution/streaming/StreamMetrics.scala | 22 ++- .../sql/execution/streaming/memory.scala | 12 +- .../streaming/StreamingMetricsSuite.scala | 185 ++++++++++++++---- 4 files changed, 163 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index cc813b9ca46e5..b2990f4698531 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -111,7 +111,7 @@ class StreamExecution( private val callSite = Utils.getCallSite() private val streamMetrics = new StreamMetrics(uniqueSources.toSet, triggerClock, - "%s.StructuredStreamingMetrics.%s".format(sparkSession.sparkContext.appName, name)) + s"StructuredStreaming.$name") /** * The thread that runs the micro-batches of this stream. Note that this thread must be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index e3b8f5c25a8b3..b2b15c6a86a0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -51,18 +51,20 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // =========== Initialization =========== - registerGauge("inputRate.total", currentInputRate) - registerGauge("processingRate.total", () => currentProcessingRate) - registerGauge("outputRate.total", () => currentOutputRate) - registerGauge("latencyMs", () => currentLatency().getOrElse(-1.0)) + // Metric names should not have . in them, so that all the metrics of a query are identified + // together in Ganglia as a single metric group + registerGauge("inputRate-total", currentInputRate) + registerGauge("processingRate-total", () => currentProcessingRate) + registerGauge("outputRate", () => currentOutputRate) + registerGauge("latency", () => currentLatency().getOrElse(-1.0)) sources.foreach { s => inputRates.put(s, new RateCalculator) processingRates.put(s, new RateCalculator) sourceTriggerInfo.put(s, new mutable.HashMap[String, String]) - registerGauge(s"inputRate.${s.toString}", () => currentSourceInputRate(s)) - registerGauge(s"processingRate.${s.toString}", () => currentSourceProcessingRate(s)) + registerGauge(s"inputRate-${s.toString}", () => currentSourceInputRate(s)) + registerGauge(s"processingRate-${s.toString}", () => currentSourceProcessingRate(s)) } // =========== Setter methods =========== @@ -132,14 +134,14 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // as this may be different for each source if there are many sources in the query plan // and getOffset is called serially on them. if (previousInputIntervalOption.nonEmpty) { - numInputRows.foreach { case (s, v) => - inputRates(s).update(v, previousInputIntervalOption.get) + sources.foreach { s => + inputRates(s).update(numInputRows.getOrElse(s, 0), previousInputIntervalOption.get) } } // Update processing rate = num rows processed for each source in current trigger duration - numInputRows.foreach { case (s, v) => - processingRates(s).update(v, currentTriggerDuration) + sources.foreach { s => + processingRates(s).update(numInputRows.getOrElse(s, 0), currentTriggerDuration) } // Update output rate = num rows output to the sink in current trigger duration diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 4984d7eef88be..ee885dbb9b240 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -82,7 +82,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" + override def toString: String = s"MemorySource[${Utils.truncatedString(output, ",")}]" override def getOffset: Option[Offset] = synchronized { if (batches.isEmpty) { @@ -99,11 +99,11 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) val startOrdinal = start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1 val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 - val batchData = synchronized { batches.slice(startOrdinal, endOrdinal) }.flatMap(_.collect()) + val selectedBatches = synchronized { batches.slice(startOrdinal, endOrdinal) } + if (selectedBatches.isEmpty) { sys.error("No data selected!") } - logInfo( - s"MemoryBatch [$startOrdinal, $endOrdinal]: ${batchData.mkString(", ")}") - if (batchData.isEmpty) { sys.error("No data selected!") } + val batchData = selectedBatches.flatMap(_.collect()) + logInfo(s"MemoryBatch [$startOrdinal, $endOrdinal]: ${batchData.mkString(", ")}") // Merge data into a single logical plan node so that StreamExecution can // match the number of leaf nodes with the number of sources for getting metrics @@ -169,6 +169,8 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi logDebug(s"Skipping already committed batch: $batchId") } } + + override def toString: String = s"MemorySink" } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala index a42df2c6bcd98..3ecbd5b8f082c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala @@ -29,67 +29,168 @@ class StreamingMetricsSuite extends SparkFunSuite { // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) - test("all rates and latency") { - val m = newStreamMetrics() - assert(m.currentInputRate() === 0.0) - assert(m.currentProcessingRate() === 0.0) - assert(m.currentOutputRate() === 0.0) - assert(m.currentSourceInputRate(source) === 0.0) - assert(m.currentSourceProcessingRate(source) === 0.0) - assert(m.currentLatency() === None) + test("rate and latency - basic life cycle") { + val sm = newStreamMetrics(source) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentOutputRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) // When trigger started, the rates should not change - m.reportTriggerStarted(1) - assert(m.currentInputRate() === 0.0) - assert(m.currentProcessingRate() === 0.0) - assert(m.currentOutputRate() === 0.0) - assert(m.currentSourceInputRate(source) === 0.0) - assert(m.currentSourceProcessingRate(source) === 0.0) - assert(m.currentLatency() === None) + sm.reportTriggerStarted(1) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentOutputRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) // Finishing the trigger should calculate the rates, except input rate which needs // to have another trigger interval - m.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows + sm.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows clock.advance(1000) - m.reportTriggerFinished() - assert(m.currentInputRate() === 0.0) - assert(m.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec - assert(m.currentOutputRate() === 10.0) // 10 output rows generated in 1 sec - assert(m.currentSourceInputRate(source) === 0.0) - assert(m.currentSourceProcessingRate(source) === 100.0) - assert(m.currentLatency() === None) + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec + assert(sm.currentOutputRate() === 10.0) // 10 output rows generated in 1 sec + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) // Another trigger should calculate the input rate clock.advance(1000) - m.reportTriggerStarted(2) - m.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows + sm.reportTriggerStarted(2) + sm.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows clock.advance(500) - m.reportTriggerFinished() - assert(m.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts - assert(m.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec - assert(m.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec - assert(m.currentSourceInputRate(source) === 100.0) - assert(m.currentSourceProcessingRate(source) === 400.0) - assert(m.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + sm.reportTriggerFinished() + assert(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + assert(sm.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec + assert(sm.currentSourceInputRate(source) === 100.0) + assert(sm.currentSourceProcessingRate(source) === 400.0) + assert(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms // Rates should be set to 0 after stop - m.stop() - assert(m.currentInputRate() === 0.0) - assert(m.currentProcessingRate() === 0.0) - assert(m.currentOutputRate() === 0.0) - assert(m.currentSourceInputRate(source) === 0.0) - assert(m.currentSourceProcessingRate(source) === 0.0) - assert(m.currentLatency() === None) + sm.stop() + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentOutputRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) } - private def newStreamMetrics(): StreamMetrics = new StreamMetrics( - Set[Source](source), clock, "test") + test("rate and latency - after trigger with no data") { + val sm = newStreamMetrics(source) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + require(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts + require(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec + require(sm.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec + require(sm.currentSourceInputRate(source) === 100.0) + require(sm.currentSourceProcessingRate(source) === 400.0) + require(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 0.0) + assert(sm.currentOutputRate() === 0.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 0.0) + assert(sm.currentLatency() === None) + sm.stop() + } + + test("rates - after trigger with multiple sources, and one source having no info") { + val source1 = TestSource(1) + val source2 = TestSource(2) + val sm = newStreamMetrics(source1, source2) + // Trigger 1 with data + sm.reportTriggerStarted(1) + sm.reportNumRows(Map(source1 -> 100L, source2 -> 100L), Some(10)) + clock.advance(1000) + sm.reportTriggerFinished() + + // Trigger 2 with data + clock.advance(1000) + sm.reportTriggerStarted(2) + sm.reportNumRows(Map(source1 -> 200L, source2 -> 200L), Some(20)) + clock.advance(500) + sm.reportTriggerFinished() + + // Make sure that all rates are set + assert(sm.currentInputRate() === 200.0) // 200*2 input rows generated in 2 seconds b/w starts + assert(sm.currentProcessingRate() === 800.0) // 200*2 output rows processed in 0.5 sec + assert(sm.currentSourceInputRate(source1) === 100.0) + assert(sm.currentSourceInputRate(source2) === 100.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 400.0) + + // Trigger 3 with data + clock.advance(500) + sm.reportTriggerStarted(3) + clock.advance(500) + sm.reportNumRows(Map(source1 -> 200L), Some(20)) + sm.reportTriggerFinished() + + // Rates are set to zero and latency is set to None + assert(sm.currentInputRate() === 200.0) + assert(sm.currentProcessingRate() === 400.0) + assert(sm.currentSourceInputRate(source1) === 200.0) + assert(sm.currentSourceInputRate(source2) === 0.0) + assert(sm.currentSourceProcessingRate(source1) === 400.0) + assert(sm.currentSourceProcessingRate(source2) === 0.0) + sm.stop() + } + + test("registered Codahale metrics") { + import scala.collection.JavaConverters._ + val sm = newStreamMetrics(source) + val gaugeNames = sm.metricRegistry.getGauges().keySet().asScala + + // so that all metrics are considered as a single metric group in Ganglia + assert(!gaugeNames.exists(_.contains("."))) + assert(gaugeNames === Set( + "inputRate-total", + "inputRate-source0", + "processingRate-total", + "processingRate-source0", + "outputRate", + "latency")) + } + + private def newStreamMetrics(sources: Source*): StreamMetrics = { + new StreamMetrics(sources.toSet, clock, "test") + } private val clock = new ManualClock() - private val source: Source = new Source { + private val source = TestSource(0) + + case class TestSource(id: Int) extends Source { override def schema: StructType = StructType(Array.empty[StructField]) override def getOffset: Option[Offset] = Some(new LongOffset(0)) override def getBatch(start: Option[Offset], end: Offset): DataFrame = { null } override def stop() {} + override def toString(): String = s"source$id" } } From 43e1ab1df1406bf3ed7d9084c13bbe392b06b3b4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 3 Oct 2016 19:19:31 -0700 Subject: [PATCH 07/43] Addressed comments, and added more tests --- .../spark/sql/catalyst/trees/TreeNode.scala | 7 ++ .../execution/streaming/StreamExecution.scala | 93 +++++++++++++++---- .../execution/streaming/StreamMetrics.scala | 4 + .../sql/execution/streaming/memory.scala | 25 +++-- .../streaming/StreamingMetricsSuite.scala | 4 +- .../sql/streaming/StreamingQuerySuite.scala | 49 +++++++++- 6 files changed, 145 insertions(+), 37 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 83cb375525832..ea8d8fef7bdf1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -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. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index b2990f4698531..458b612f97efe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand @@ -390,14 +391,14 @@ class StreamExecution( // Rewire the plan to use the new attributes that were returned by the source. val replacementMap = AttributeMap(replacements) - val newPlan = withNewSources transformAllExpressions { + val triggerLogicalPlan = withNewSources transformAllExpressions { case a: Attribute if replacementMap.contains(a) => replacementMap(a) } val optimizerStart = System.nanoTime() lastExecution = new IncrementalExecution( sparkSession, - newPlan, + triggerLogicalPlan, outputMode, checkpointFile("state"), currentBatchId) @@ -409,7 +410,7 @@ class StreamExecution( val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId, nextBatch) - reportMetrics(executedPlan) + reportMetrics(executedPlan, triggerLogicalPlan, newData) awaitBatchLock.lock() try { @@ -555,23 +556,23 @@ class StreamExecution( """.stripMargin } - private def reportMetrics(executedPlan: SparkPlan): Unit = { - val execPlanLeaves = executedPlan.collect { case p if p.children.isEmpty => p } - val sourceToNumInputRows = if (execPlanLeaves.size == sources.size) { - sources.zip(execPlanLeaves).flatMap { case (s, leaf) => - leaf.metrics.get("numOutputRows").map { m => s -> m.value } - }.toMap - } else { - logWarning( - "Could not report metrics as number of sources did not match number of leaves in" + - s" execution plan: sources(${sources.size}) = ${sources.mkString(", ")};" + - s" leaves(${execPlanLeaves.size}) = ${execPlanLeaves.mkString(",")}") - Map.empty[Source, Long] - } - val numOutputRows = executedPlan.metrics.get("numOutputRows").map(_.value) + /** + * Report row metrics of the executed trigger + * @param triggerExecutionPlan Execution plan of the trigger + * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan + * @param sourceToDataframe Source to DataFrame returned by the source.getBatch in this trigger + */ + private def reportMetrics( + triggerExecutionPlan: SparkPlan, + triggerLogicalPlan: LogicalPlan, + sourceToDataframe: Map[Source, DataFrame]): Unit = { + val sourceToNumInputRows = StreamExecution.getNumInputRowsFromTrigger( + triggerExecutionPlan, triggerLogicalPlan, sourceToDataframe) + val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) - val stateNodes = executedPlan.collect { case p if p.isInstanceOf[StateStoreSaveExec] => p } + val stateNodes = triggerExecutionPlan.collect { + case p if p.isInstanceOf[StateStoreSaveExec] => p } stateNodes.zipWithIndex.foreach { case (s, i) => streamMetrics.reportTriggerInfo(NUM_TOTAL_STATE_ROWS(i + 1), s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) @@ -616,8 +617,62 @@ class StreamExecution( case object TERMINATED extends State } -object StreamExecution { +object StreamExecution extends Logging { private val _nextId = new AtomicLong(0) + /** + * Get the number of input rows from the executed plan of the trigger + * @param triggerExecutionPlan Execution plan of the trigger + * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan + * @param sourceToDataframe Source to DataFrame returned by the source.getBatch in this trigger + */ + def getNumInputRowsFromTrigger( + triggerExecutionPlan: SparkPlan, + triggerLogicalPlan: LogicalPlan, + sourceToDataframe: Map[Source, DataFrame]): Map[Source, Long] = { + + // We want to associate execution plan leaves to sources that generate them, so that we match + // the their metrics (e.g. numOutputRows) to the sources. To do this we do the following. + // Consider the translation from the streaming logical plan to the final executed plan. + // + // streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan + // + // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan + // - Each logical plan leaf will be associated with a single streaming source. + // - There can be multiple logical plan leaves associated a streaming source. + // - There can be leaves not associated with any streaming source, because they were + // generated from a batch source (e.g. stream-batch joins) + // + // 2. Assuming that the executed plan has same number of leaves in the same order as that of + // the trigger logical plan, we associate executed plan leaves with corresponding + // streaming sources. + // + // 3. For each source, we sum the metrics of the associated execution plan leaves. + // + val logicalPlanLeafToSource = sourceToDataframe.flatMap { case (source, df) => + df.logicalPlan.collectLeaves().map { leaf => leaf -> source } + } + val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // includes non-streaming sources + val allExecPlanLeaves = triggerExecutionPlan.collectLeaves() + if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) { + val execLeafToSource = allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap { + case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => ep -> source } + } + val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, source) => + val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) + source -> numRows + } + sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source + } else { + def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" + logWarning( + "Could not report metrics as number leaves in trigger logical plan did not match that" + + s" of the execution plan:\n" + + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + + s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + Map.empty + } + } + def nextId: Long = _nextId.getAndIncrement() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index b2b15c6a86a0c..5566b984f194e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -23,6 +23,9 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.{Source => CodahaleSource} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Clock class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceName: String) @@ -228,6 +231,7 @@ object StreamMetrics extends Logging { def stop(): Unit = { rate = None } } + val TRIGGER_ID = "triggerId" val ACTIVE = "isActive" val DATA_AVAILABLE = "isDataAvailable" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index ee885dbb9b240..43d893930b5a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -82,7 +82,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemorySource[${Utils.truncatedString(output, ",")}]" + override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" override def getOffset: Option[Offset] = synchronized { if (batches.isEmpty) { @@ -95,19 +95,20 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) /** * Returns the data that is between the offsets (`start`, `end`]. */ - override def getBatch(start: Option[Offset], end: Offset): DataFrame = synchronized { + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { val startOrdinal = start.map(_.asInstanceOf[LongOffset]).getOrElse(LongOffset(-1)).offset.toInt + 1 val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 - val selectedBatches = synchronized { batches.slice(startOrdinal, endOrdinal) } - if (selectedBatches.isEmpty) { sys.error("No data selected!") } - - val batchData = selectedBatches.flatMap(_.collect()) - logInfo(s"MemoryBatch [$startOrdinal, $endOrdinal]: ${batchData.mkString(", ")}") - - // Merge data into a single logical plan node so that StreamExecution can - // match the number of leaf nodes with the number of sources for getting metrics - sqlContext.createDataset(batchData).toDF() + val newBlocks = synchronized { batches.slice(startOrdinal, endOrdinal) } + + logDebug( + s"MemoryBatch [$startOrdinal, $endOrdinal]: ${newBlocks.flatMap(_.collect()).mkString(", ")}") + newBlocks + .map(_.toDF()) + .reduceOption(_ union _) + .getOrElse { + sys.error("No data selected!") + } } override def stop() {} @@ -169,8 +170,6 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi logDebug(s"Skipping already committed batch: $batchId") } } - - override def toString: String = s"MemorySink" } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala index 3ecbd5b8f082c..e3207400f12f0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala @@ -29,7 +29,7 @@ class StreamingMetricsSuite extends SparkFunSuite { // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) - test("rate and latency - basic life cycle") { + test("rates and latencies - basic life cycle") { val sm = newStreamMetrics(source) assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) @@ -82,7 +82,7 @@ class StreamingMetricsSuite extends SparkFunSuite { assert(sm.currentLatency() === None) } - test("rate and latency - after trigger with no data") { + test("rates and latencies - after trigger with no data") { val sm = newStreamMetrics(source) // Trigger 1 with data sm.reportTriggerStarted(1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 515a0dda5c261..dcb94595dfabf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,12 +17,18 @@ package org.apache.spark.sql.streaming +import java.util.concurrent.atomic.AtomicInteger + import org.scalactic.TolerantNumerics +import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.util.{ManualClock, Utils} +import org.apache.spark.util.Utils class StreamingQuerySuite extends StreamTest with BeforeAndAfter { @@ -104,7 +110,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } - testQuietly("statuses") { + testQuietly("query statuses") { val inputData = MemoryStream[Int] // This is make the sure the execution plan ends with a node (filter) that supports @@ -202,6 +208,43 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } + // This tests whether row stats are correctly associated with streaming sources when + // streaming plan also has batch sources + test("calculating input rows with mixed batch and streaming sources") { + + // A streaming source that returns a pre-determined dataframe for a trigger + // Creates multiple leaves for a streaming source + val streamingTriggerDF = spark.createDataset(1 to 5).toDF.union( + spark.createDataset(6 to 10).toDF) + val streamingSource = new Source() { + override def schema: StructType = StructType(Seq(StructField("value", IntegerType))) + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = streamingTriggerDF + override def stop(): Unit = {} + } + val streamingInputDF = StreamingExecutionRelation(streamingSource).toDF("value") + val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") + + // streaming trigger input has 10 items, static input has 2 items, input row calculation for + // the source should return 10 + spark.conf.set("spark.sql.codegen.wholeStage", false) + testStream(streamingInputDF.join(staticInputDF, "value"))( + StartStream(), + AssertOnQuery { q => + eventually(Timeout(streamingTimeout)) { + assert(q.queryStatus.sinkStatus.offsetDesc + === CompositeOffset.fill(LongOffset(0)).toString) + } + val numInputRows = StreamExecution.getNumInputRowsFromTrigger( + q.lastExecution.executedPlan, + q.lastExecution.logical, + Map(streamingSource -> streamingTriggerDF)) + assert(numInputRows === Map(streamingSource -> 10)) // streaming data has 10 items + true + } + ) + } + testQuietly("StreamExecution metadata garbage collection") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map(6 / _) From 62f5b0bd8462ab5324469ab3612a826c33abe947 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 01:29:08 -0700 Subject: [PATCH 08/43] Fix polling delay --- .../apache/spark/sql/execution/streaming/StreamExecution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 458b612f97efe..547dcdb9f9ce7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -224,7 +224,7 @@ class StreamExecution( } else { streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "No new data") streamMetrics.reportTriggerInfo(DATA_AVAILABLE, false) - Thread.sleep(100) + Thread.sleep(pollingDelayMs) } true } else { From 4074428f0f59b447efb1ab267ef6fbe2d105d0eb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 12:12:09 -0700 Subject: [PATCH 09/43] Added synchronization --- .../execution/streaming/StreamExecution.scala | 118 +++++++++++------- .../execution/streaming/StreamMetrics.scala | 12 -- .../spark/sql/streaming/SinkStatus.scala | 2 +- .../spark/sql/streaming/SourceStatus.scala | 2 +- .../spark/sql/streaming/StreamingQuery.scala | 2 +- .../sql/streaming/StreamingQueryInfo.scala | 2 +- .../StreamingQueryListenerSuite.scala | 8 +- .../sql/streaming/StreamingQuerySuite.scala | 94 +++++++------- 8 files changed, 128 insertions(+), 112 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 547dcdb9f9ce7..c65783b0f0976 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand @@ -70,6 +69,12 @@ class StreamExecution( private val startLatch = new CountDownLatch(1) private val terminationLatch = new CountDownLatch(1) + /** + * Lock object used to synchronized updates to data structures that are used to report + * query status. + */ + private val statusLock = new Object + /** * Tracks how much data we have processed and committed to the sink or state store from each * input source. @@ -140,15 +145,15 @@ class StreamExecution( /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE - override def queryStatus: StreamingQueryInfo = { + override def status: StreamingQueryInfo = statusLock.synchronized { this.toInfo } /** Returns current status of all the sources. */ - override def sourceStatuses: Array[SourceStatus] = { + override def sourceStatuses: Array[SourceStatus] = statusLock.synchronized { val localAvailableOffsets = availableOffsets sources.map(s => - new SourceStatus( + SourceStatus( s.toString, localAvailableOffsets.get(s).map(_.toString), streamMetrics.currentSourceInputRate(s), @@ -158,8 +163,8 @@ class StreamExecution( } /** Returns current status of the sink. */ - override def sinkStatus: SinkStatus = { - new SinkStatus( + override def sinkStatus: SinkStatus = statusLock.synchronized { + SinkStatus( sink.toString, committedOffsets.toCompositeOffset(sources).toString, streamMetrics.currentOutputRate()) @@ -204,9 +209,11 @@ class StreamExecution( SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { - streamMetrics.reportTriggerStarted(currentBatchId) - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Finding new data from sources") - val isTerminated = timeIt(TRIGGER_LATENCY) { + statusLock.synchronized { + streamMetrics.reportTriggerStarted(currentBatchId) + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Finding new data from sources") + } + val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { if (isActive) { if (currentBatchId < 0) { // We'll do this initialization only once @@ -216,14 +223,18 @@ class StreamExecution( constructNextBatch() } if (dataAvailable) { - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Processing new data") - streamMetrics.reportTriggerInfo(DATA_AVAILABLE, true) + statusLock.synchronized { + streamMetrics.reportTriggerInfo(DATA_AVAILABLE, true) + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Processing new data") + } runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 } else { - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "No new data") - streamMetrics.reportTriggerInfo(DATA_AVAILABLE, false) + statusLock.synchronized { + streamMetrics.reportTriggerInfo(DATA_AVAILABLE, false) + streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "No new data") + } Thread.sleep(pollingDelayMs) } true @@ -231,7 +242,7 @@ class StreamExecution( false } } - streamMetrics.reportTriggerFinished() + statusLock.synchronized { streamMetrics.reportTriggerFinished() } postEvent(new QueryProgress(this.toInfo)) isTerminated }) @@ -265,17 +276,18 @@ class StreamExecution( private def populateStartOffsets(): Unit = { offsetLog.getLatest() match { case Some((batchId, nextOffsets)) => - logInfo(s"Resuming streaming query, starting with batch $batchId") - currentBatchId = batchId - availableOffsets = nextOffsets.toStreamProgress(sources) - logDebug(s"Found possibly uncommitted offsets $availableOffsets") - - offsetLog.get(batchId - 1).foreach { - case lastOffsets => - committedOffsets = lastOffsets.toStreamProgress(sources) - logDebug(s"Resuming with committed offsets: $committedOffsets") + statusLock.synchronized { + logInfo(s"Resuming streaming query, starting with batch $batchId") + currentBatchId = batchId + availableOffsets = nextOffsets.toStreamProgress(sources) + logDebug(s"Found possibly uncommitted offsets $availableOffsets") + + offsetLog.get(batchId - 1).foreach { + case lastOffsets => + committedOffsets = lastOffsets.toStreamProgress(sources) + logDebug(s"Resuming with committed offsets: $committedOffsets") + } } - case None => // We are starting this stream for the first time. logInfo(s"Starting new streaming query.") currentBatchId = 0 @@ -305,13 +317,15 @@ class StreamExecution( val hasNewData = { awaitBatchLock.lock() try { - timeIt(GET_OFFSET_LATENCY) { + reportTimeTaken(GET_OFFSET_LATENCY) { val latestOffsets: Map[Source, Option[Offset]] = uniqueSources.map { s => - timeIt(s, SOURCE_GET_OFFSET_LATENCY) { + reportTimeTaken(s, SOURCE_GET_OFFSET_LATENCY) { (s, s.getOffset) } }.toMap - availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) + statusLock.synchronized { + availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) + } } if (dataAvailable) { @@ -325,7 +339,7 @@ class StreamExecution( } } if (hasNewData) { - timeIt(OFFSET_WAL_WRITE_LATENCY) { + reportTimeTaken(OFFSET_WAL_WRITE_LATENCY) { assert( offsetLog.add(currentBatchId, availableOffsets.toCompositeOffset(sources)), s"Concurrent update to the log. Multiple streaming jobs detected for $currentBatchId") @@ -347,7 +361,7 @@ class StreamExecution( awaitBatchLock.unlock() } } - streamMetrics.reportTimestamp(GET_OFFSET_TIMESTAMP) + reportTimestamp(GET_OFFSET_TIMESTAMP) } /** @@ -359,7 +373,7 @@ class StreamExecution( // TODO: Move this to IncrementalExecution. // Request unprocessed data from all sources. - val newData = timeIt(GET_BATCH_LATENCY) { + val newData = reportTimeTaken(GET_BATCH_LATENCY) { availableOffsets.flatMap { case (source, available) if committedOffsets.get(source).map(_ != available).getOrElse(true) => @@ -370,7 +384,7 @@ class StreamExecution( case _ => None } } - streamMetrics.reportTimestamp(GET_BATCH_TIMESTAMP) + reportTimestamp(GET_BATCH_TIMESTAMP) // A list of attributes that will need to be updated. var replacements = new ArrayBuffer[(Attribute, Attribute)] @@ -423,7 +437,9 @@ class StreamExecution( val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. - committedOffsets ++= availableOffsets + statusLock.synchronized { + committedOffsets ++= availableOffsets + } } private def postEvent(event: StreamingQueryListener.Event) { @@ -569,36 +585,48 @@ class StreamExecution( val sourceToNumInputRows = StreamExecution.getNumInputRowsFromTrigger( triggerExecutionPlan, triggerLogicalPlan, sourceToDataframe) val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) - streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) - val stateNodes = triggerExecutionPlan.collect { - case p if p.isInstanceOf[StateStoreSaveExec] => p } - stateNodes.zipWithIndex.foreach { case (s, i) => - streamMetrics.reportTriggerInfo(NUM_TOTAL_STATE_ROWS(i + 1), - s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) - streamMetrics.reportTriggerInfo(NUM_UPDATED_STATE_ROWS(i + 1), - s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) + case p if p.isInstanceOf[StateStoreSaveExec] => p + } + statusLock.synchronized {streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) + stateNodes.zipWithIndex.foreach { case (s, i) => + streamMetrics.reportTriggerInfo( + NUM_TOTAL_STATE_ROWS(i + 1), + s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) + streamMetrics.reportTriggerInfo( + NUM_UPDATED_STATE_ROWS(i + 1), + s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) + } } } - private def timeIt[T](triggerInfoKey: String)(body: => T): T = { + private def reportTimeTaken[T](triggerInfoKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - streamMetrics.reportLatency(triggerInfoKey, math.max(endTime - startTime, 0)) + statusLock.synchronized { + streamMetrics.reportTriggerInfo(triggerInfoKey, math.max(endTime - startTime, 0)) + } result } - private def timeIt[T](source: Source, triggerInfoKey: String)(body: => T): T = { + private def reportTimeTaken[T](source: Source, triggerInfoKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - streamMetrics.reportLatency(source, triggerInfoKey, math.max(endTime - startTime, 0)) + statusLock.synchronized { + streamMetrics.reportSourceTriggerInfo( + source, triggerInfoKey, math.max(endTime - startTime, 0)) + } result } + private def reportTimestamp(string: String): Unit = statusLock.synchronized { + streamMetrics.reportTriggerInfo(string, triggerClock.getTimeMillis) + } + private def toInfo: StreamingQueryInfo = { - new StreamingQueryInfo( + StreamingQueryInfo( this.name, this.id, triggerClock.getTimeMillis(), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 5566b984f194e..0a2c33c4c1417 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -85,18 +85,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam reportTriggerInfo(START_TIMESTAMP, currentTriggerStartTimestamp) } - def reportTimestamp(key: String): Unit = synchronized { - triggerInfo.put(key, triggerClock.getTimeMillis().toString) - } - - def reportLatency(key: String, latencyMs: Long): Unit = synchronized { - triggerInfo.put(key, latencyMs.toString) - } - - def reportLatency(source: Source, key: String, latencyMs: Long): Unit = synchronized { - sourceTriggerInfo(source).put(key, latencyMs.toString) - } - def reportTriggerInfo[T](key: String, value: T): Unit = synchronized { triggerInfo.put(key, value.toString) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index ac09d3f3bfc6f..9513dff33b936 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.streaming.Sink * @since 2.0.0 */ @Experimental -case class SinkStatus private[sql]( +case class SinkStatus private( val description: String, val offsetDesc: String, val outputRate: Double) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 4322db248e88e..51e92bcbe5111 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.streaming.Source * @since 2.0.0 */ @Experimental -case class SourceStatus private[sql]( +case class SourceStatus private( val description: String, val offsetDesc: Option[String], val inputRate: Double, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index cf16c4fd26743..dfdbc93e953c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -66,7 +66,7 @@ trait StreamingQuery { * Returns the current status of the query. * @since 2.1.0 */ - def queryStatus: StreamingQueryInfo + def status: StreamingQueryInfo /** * Returns current status of all the sources. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index 6a0c2c3cf9833..c4cd7390af848 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -30,7 +30,7 @@ import org.apache.spark.annotation.Experimental * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. */ @Experimental -case class StreamingQueryInfo private[sql]( +case class StreamingQueryInfo private( val name: String, val id: Long, val timestamp: Long, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index fd5013cbf9149..62ed1b9a73708 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -340,12 +340,12 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } private val testQueryInfo: StreamingQueryInfo = { - new StreamingQueryInfo( + StreamingQueryInfo( "name", 1, 123, 1.0, 2.0, 3.0, Some(345), Seq( - new SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map.empty), - new SourceStatus("source2", Some(LongOffset(1).toString), 1.0, 2.0, Map("a" -> "b"))), - new SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString, 2.0), + SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map.empty), + SourceStatus("source2", Some(LongOffset(1).toString), 1.0, 2.0, Map("a" -> "b"))), + SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString, 2.0), Map("a" -> "b")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index dcb94595dfabf..4aadfeebc535a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -119,20 +119,20 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { val mapped = inputData.toDS().map(6 / _).where("value > 0") testStream(mapped)( - AssertOnQuery(q => q.queryStatus.name === q.name), - AssertOnQuery(q => q.queryStatus.id === q.id), - AssertOnQuery(_.queryStatus.timestamp <= System.currentTimeMillis), - AssertOnQuery(_.queryStatus.inputRate === 0.0), - AssertOnQuery(_.queryStatus.processingRate === 0.0), - AssertOnQuery(_.queryStatus.outputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), - AssertOnQuery(_.queryStatus.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === None), - AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), - AssertOnQuery(_.queryStatus.sinkStatus.description.contains("Memory")), - AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), - AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), + AssertOnQuery(q => q.status.name === q.name), + AssertOnQuery(q => q.status.id === q.id), + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.outputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), + AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), @@ -143,19 +143,19 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { AddData(inputData, 1, 2), CheckAnswer(6, 3), - AssertOnQuery(_.queryStatus.timestamp <= System.currentTimeMillis), - AssertOnQuery(_.queryStatus.inputRate >= 0.0), - AssertOnQuery(_.queryStatus.processingRate >= 0.0), - AssertOnQuery(_.queryStatus.outputRate >= 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), - AssertOnQuery(_.queryStatus.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), - AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate >= 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate >= 0.0), - AssertOnQuery(_.queryStatus.sinkStatus.description.contains("Memory")), - AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), + AssertOnQuery(_.status.inputRate >= 0.0), + AssertOnQuery(_.status.processingRate >= 0.0), + AssertOnQuery(_.status.outputRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.status.sourceStatuses(0).inputRate >= 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate >= 0.0), + AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), - AssertOnQuery(_.queryStatus.sinkStatus.outputRate >= 0.0), + AssertOnQuery(_.status.sinkStatus.outputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), @@ -164,23 +164,23 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), - AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), - AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), StopStream, - AssertOnQuery(_.queryStatus.inputRate === 0.0), - AssertOnQuery(_.queryStatus.processingRate === 0.0), - AssertOnQuery(_.queryStatus.outputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), - AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), - AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), - AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.outputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), + AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), @@ -190,16 +190,16 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], - AssertOnQuery(_.queryStatus.inputRate === 0.0), - AssertOnQuery(_.queryStatus.processingRate === 0.0), - AssertOnQuery(_.queryStatus.outputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses.length === 1), - AssertOnQuery(_.queryStatus.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), - AssertOnQuery(_.queryStatus.sourceStatuses(0).inputRate === 0.0), - AssertOnQuery(_.queryStatus.sourceStatuses(0).processingRate === 0.0), - AssertOnQuery(_.queryStatus.sinkStatus.offsetDesc === + AssertOnQuery(_.status.inputRate === 0.0), + AssertOnQuery(_.status.processingRate === 0.0), + AssertOnQuery(_.status.outputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses.length === 1), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), + AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), + AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.queryStatus.sinkStatus.outputRate === 0.0), + AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), @@ -232,7 +232,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { StartStream(), AssertOnQuery { q => eventually(Timeout(streamingTimeout)) { - assert(q.queryStatus.sinkStatus.offsetDesc + assert(q.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) } val numInputRows = StreamExecution.getNumInputRowsFromTrigger( From 2bcbcd2961ace97449df880915532fab42d6d876 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 12:24:19 -0700 Subject: [PATCH 10/43] Renamed *info to *status for consistency, except StreamingQueryInfo --- .../execution/streaming/StreamExecution.scala | 30 ++++++------- .../execution/streaming/StreamMetrics.scala | 44 +++++++++---------- .../spark/sql/streaming/SourceStatus.scala | 2 +- .../sql/streaming/StreamingQueryInfo.scala | 2 +- .../StreamingQueryListenerSuite.scala | 38 ++++++++-------- 5 files changed, 58 insertions(+), 58 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index c65783b0f0976..ecff309acbfc2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -158,7 +158,7 @@ class StreamExecution( localAvailableOffsets.get(s).map(_.toString), streamMetrics.currentSourceInputRate(s), streamMetrics.currentSourceProcessingRate(s), - streamMetrics.currentSourceTriggerInfo(s)) + streamMetrics.currentSourceTriggerStatus(s)) ).toArray } @@ -211,7 +211,7 @@ class StreamExecution( triggerExecutor.execute(() => { statusLock.synchronized { streamMetrics.reportTriggerStarted(currentBatchId) - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Finding new data from sources") + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Finding new data from sources") } val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { if (isActive) { @@ -224,16 +224,16 @@ class StreamExecution( } if (dataAvailable) { statusLock.synchronized { - streamMetrics.reportTriggerInfo(DATA_AVAILABLE, true) - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "Processing new data") + streamMetrics.reportTriggerStatus(DATA_AVAILABLE, true) + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Processing new data") } runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 } else { statusLock.synchronized { - streamMetrics.reportTriggerInfo(DATA_AVAILABLE, false) - streamMetrics.reportTriggerInfo(STATUS_MESSAGE, "No new data") + streamMetrics.reportTriggerStatus(DATA_AVAILABLE, false) + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "No new data") } Thread.sleep(pollingDelayMs) } @@ -590,39 +590,39 @@ class StreamExecution( } statusLock.synchronized {streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) stateNodes.zipWithIndex.foreach { case (s, i) => - streamMetrics.reportTriggerInfo( + streamMetrics.reportTriggerStatus( NUM_TOTAL_STATE_ROWS(i + 1), s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) - streamMetrics.reportTriggerInfo( + streamMetrics.reportTriggerStatus( NUM_UPDATED_STATE_ROWS(i + 1), s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) } } } - private def reportTimeTaken[T](triggerInfoKey: String)(body: => T): T = { + private def reportTimeTaken[T](triggerStatusKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() statusLock.synchronized { - streamMetrics.reportTriggerInfo(triggerInfoKey, math.max(endTime - startTime, 0)) + streamMetrics.reportTriggerStatus(triggerStatusKey, math.max(endTime - startTime, 0)) } result } - private def reportTimeTaken[T](source: Source, triggerInfoKey: String)(body: => T): T = { + private def reportTimeTaken[T](source: Source, triggerStatusKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() statusLock.synchronized { - streamMetrics.reportSourceTriggerInfo( - source, triggerInfoKey, math.max(endTime - startTime, 0)) + streamMetrics.reportSourceTriggerStatus( + source, triggerStatusKey, math.max(endTime - startTime, 0)) } result } private def reportTimestamp(string: String): Unit = statusLock.synchronized { - streamMetrics.reportTriggerInfo(string, triggerClock.getTimeMillis) + streamMetrics.reportTriggerStatus(string, triggerClock.getTimeMillis) } private def toInfo: StreamingQueryInfo = { @@ -636,7 +636,7 @@ class StreamExecution( streamMetrics.currentLatency, this.sourceStatuses, this.sinkStatus, - streamMetrics.currentTriggerInfo) + streamMetrics.currentTriggerStatus) } trait State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 0a2c33c4c1417..843c5914004dc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -34,8 +34,8 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam import StreamMetrics._ // Trigger infos - private val triggerInfo = new mutable.HashMap[String, String] - private val sourceTriggerInfo = new mutable.HashMap[Source, mutable.HashMap[String, String]] + private val triggerStatus = new mutable.HashMap[String, String] + private val sourceTriggerStatus = new mutable.HashMap[Source, mutable.HashMap[String, String]] // Rate estimators for sources and sinks private val inputRates = new mutable.HashMap[Source, RateCalculator] @@ -64,7 +64,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam sources.foreach { s => inputRates.put(s, new RateCalculator) processingRates.put(s, new RateCalculator) - sourceTriggerInfo.put(s, new mutable.HashMap[String, String]) + sourceTriggerStatus.put(s, new mutable.HashMap[String, String]) registerGauge(s"inputRate-${s.toString}", () => currentSourceInputRate(s)) registerGauge(s"processingRate-${s.toString}", () => currentSourceProcessingRate(s)) @@ -75,22 +75,22 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def reportTriggerStarted(triggerId: Long): Unit = synchronized { numInputRows.clear() numOutputRows = None - triggerInfo.clear() - sourceTriggerInfo.values.foreach(_.clear()) + triggerStatus.clear() + sourceTriggerStatus.values.foreach(_.clear()) - reportTriggerInfo(TRIGGER_ID, triggerId) - sources.foreach(s => reportSourceTriggerInfo(s, TRIGGER_ID, triggerId)) - reportTriggerInfo(ACTIVE, true) + reportTriggerStatus(TRIGGER_ID, triggerId) + sources.foreach(s => reportSourceTriggerStatus(s, TRIGGER_ID, triggerId)) + reportTriggerStatus(ACTIVE, true) currentTriggerStartTimestamp = triggerClock.getTimeMillis() - reportTriggerInfo(START_TIMESTAMP, currentTriggerStartTimestamp) + reportTriggerStatus(START_TIMESTAMP, currentTriggerStartTimestamp) } - def reportTriggerInfo[T](key: String, value: T): Unit = synchronized { - triggerInfo.put(key, value.toString) + def reportTriggerStatus[T](key: String, value: T): Unit = synchronized { + triggerStatus.put(key, value.toString) } - def reportSourceTriggerInfo[T](source: Source, key: String, value: T): Unit = synchronized { - sourceTriggerInfo(source).put(key, value.toString) + def reportSourceTriggerStatus[T](source: Source, key: String, value: T): Unit = synchronized { + sourceTriggerStatus(source).put(key, value.toString) } def reportNumRows(inputRows: Map[Source, Long], outputRows: Option[Long]): Unit = synchronized { @@ -101,16 +101,16 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def reportTriggerFinished(): Unit = synchronized { require(currentTriggerStartTimestamp >= 0) val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() - reportTriggerInfo(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) - reportTriggerInfo(STATUS_MESSAGE, "") - reportTriggerInfo(ACTIVE, false) + reportTriggerStatus(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) + reportTriggerStatus(STATUS_MESSAGE, "") + reportTriggerStatus(ACTIVE, false) // Report number of rows val totalNumInputRows = numInputRows.values.sum - reportTriggerInfo(NUM_INPUT_ROWS, totalNumInputRows) - reportTriggerInfo(NUM_OUTPUT_ROWS, numOutputRows.getOrElse(0)) + reportTriggerStatus(NUM_INPUT_ROWS, totalNumInputRows) + reportTriggerStatus(NUM_OUTPUT_ROWS, numOutputRows.getOrElse(0)) numInputRows.foreach { case (s, r) => - reportSourceTriggerInfo(s, NUM_SOURCE_INPUT_ROWS, r) + reportSourceTriggerStatus(s, NUM_SOURCE_INPUT_ROWS, r) } val currentTriggerDuration = currentTriggerFinishTimestamp - currentTriggerStartTimestamp @@ -176,10 +176,10 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def currentLatency(): Option[Double] = synchronized { latency } - def currentTriggerInfo(): Map[String, String] = synchronized { triggerInfo.toMap } + def currentTriggerStatus(): Map[String, String] = synchronized { triggerStatus.toMap } - def currentSourceTriggerInfo(source: Source): Map[String, String] = synchronized { - sourceTriggerInfo(source).toMap + def currentSourceTriggerStatus(source: Source): Map[String, String] = synchronized { + sourceTriggerStatus(source).toMap } // =========== Other methods =========== diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 51e92bcbe5111..9b82b92dd37a1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -35,4 +35,4 @@ case class SourceStatus private( val offsetDesc: Option[String], val inputRate: Double, val processingRate: Double, - val triggerInfo: Map[String, String]) + val triggerStatus: Map[String, String]) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index c4cd7390af848..3dcdd510e7091 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -40,5 +40,5 @@ case class StreamingQueryInfo private( val latencyMs: Option[Double], val sourceStatuses: Seq[SourceStatus], val sinkStatus: SinkStatus, - val triggerInfo: Map[String, String] + val triggerStatus: Map[String, String] ) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 62ed1b9a73708..b0ccfed08ed01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -175,28 +175,28 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Check the correctness of the trigger info of the first completed batch reported by // onQueryProgress val status = listener.lastTriggerStatus.get - assert(status.triggerInfo("triggerId") == "0") - assert(status.triggerInfo("isActive") === "false") + assert(status.triggerStatus("triggerId") == "0") + assert(status.triggerStatus("isActive") === "false") - assert(status.triggerInfo("timestamp.triggerStart") === "0") - assert(status.triggerInfo("timestamp.afterGetOffset") === "100") - assert(status.triggerInfo("timestamp.afterGetBatch") === "300") - assert(status.triggerInfo("timestamp.triggerFinish") === "600") + assert(status.triggerStatus("timestamp.triggerStart") === "0") + assert(status.triggerStatus("timestamp.afterGetOffset") === "100") + assert(status.triggerStatus("timestamp.afterGetBatch") === "300") + assert(status.triggerStatus("timestamp.triggerFinish") === "600") - assert(status.triggerInfo("latency.getOffset") === "100") - assert(status.triggerInfo("latency.getBatch") === "200") - assert(status.triggerInfo("latency.offsetLogWrite") === "0") - assert(status.triggerInfo("latency.fullTrigger") === "600") + assert(status.triggerStatus("latency.getOffset") === "100") + assert(status.triggerStatus("latency.getBatch") === "200") + assert(status.triggerStatus("latency.offsetLogWrite") === "0") + assert(status.triggerStatus("latency.fullTrigger") === "600") - assert(status.triggerInfo("numRows.input.total") === "2") - assert(status.triggerInfo("numRows.output") === "1") - assert(status.triggerInfo("numRows.state.aggregation1.total") === "1") - assert(status.triggerInfo("numRows.state.aggregation1.updated") === "1") + assert(status.triggerStatus("numRows.input.total") === "2") + assert(status.triggerStatus("numRows.output") === "1") + assert(status.triggerStatus("numRows.state.aggregation1.total") === "1") + assert(status.triggerStatus("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerInfo("triggerId") === "0") - assert(status.sourceStatuses(0).triggerInfo("latency.sourceGetOffset") === "100") - assert(status.sourceStatuses(0).triggerInfo("numRows.input.source") === "2") + assert(status.sourceStatuses(0).triggerStatus("triggerId") === "0") + assert(status.sourceStatuses(0).triggerStatus("latency.sourceGetOffset") === "100") + assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "2") true }, CheckAnswer(2) @@ -362,8 +362,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { /** Get the info of the last trigger that processed data */ def lastTriggerStatus: Option[StreamingQueryInfo] = synchronized { progressStatuses.filter { i => - i.triggerInfo("isActive").toBoolean == false && - i.triggerInfo("isDataAvailable").toBoolean == true + i.triggerStatus("isActive").toBoolean == false && + i.triggerStatus("isDataAvailable").toBoolean == true }.lastOption } From 2d00f260b5fd2d5c494e57627c39f0abe8aa612c Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 13:40:46 -0700 Subject: [PATCH 11/43] Added exclude --- project/MimaExcludes.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 727e50c0d20e2..b1fc389453e7c 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -60,7 +60,7 @@ object MimaExcludes { 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.queryStatus") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status") ) } From d69935c6943be98839da3609c17937ffbba99020 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 17:16:07 -0700 Subject: [PATCH 12/43] Added latency.sourceGetBatch --- .../spark/sql/execution/streaming/StreamExecution.scala | 4 +++- .../apache/spark/sql/execution/streaming/StreamMetrics.scala | 3 ++- .../spark/sql/streaming/StreamingQueryListenerSuite.scala | 1 + 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ecff309acbfc2..9bfaf1a187e91 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -378,7 +378,9 @@ class StreamExecution( case (source, available) if committedOffsets.get(source).map(_ != available).getOrElse(true) => val current = committedOffsets.get(source) - val batch = source.getBatch(current, available) + val batch = reportTimeTaken(source, SOURCE_GET_BATCH_LATENCY) { + source.getBatch(current, available) + } logDebug(s"Retrieving data from $source: $current -> $available") Some(source -> batch) case _ => None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 843c5914004dc..402c6d5e43fbf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -230,11 +230,12 @@ object StreamMetrics extends Logging { val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch" val FINISH_TIMESTAMP = "timestamp.triggerFinish" - val SOURCE_GET_OFFSET_LATENCY = "latency.sourceGetOffset" val GET_OFFSET_LATENCY = "latency.getOffset" val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" val GET_BATCH_LATENCY = "latency.getBatch" val TRIGGER_LATENCY = "latency.fullTrigger" + val SOURCE_GET_OFFSET_LATENCY = "latency.sourceGetOffset" + val SOURCE_GET_BATCH_LATENCY = "latency.sourceGetBatch" val NUM_INPUT_ROWS = "numRows.input.total" val NUM_OUTPUT_ROWS = "numRows.output" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index b0ccfed08ed01..0943c48a7ec4c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -196,6 +196,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.sourceStatuses.size === 1) assert(status.sourceStatuses(0).triggerStatus("triggerId") === "0") assert(status.sourceStatuses(0).triggerStatus("latency.sourceGetOffset") === "100") + assert(status.sourceStatuses(0).triggerStatus("latency.sourceGetBatch") === "200") assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "2") true }, From 05f22d7974f410289028bfa4df1d2f6036f5023e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 17:57:24 -0700 Subject: [PATCH 13/43] Made sure status objects are private[sql] --- .../main/scala/org/apache/spark/sql/streaming/SinkStatus.scala | 3 +++ .../scala/org/apache/spark/sql/streaming/SourceStatus.scala | 3 +++ .../org/apache/spark/sql/streaming/StreamingQueryInfo.scala | 3 +++ .../spark/sql/streaming/StreamingQueryListenerSuite.scala | 2 +- 4 files changed, 10 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 9513dff33b936..52bb8a60002e7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -34,3 +34,6 @@ case class SinkStatus private( val description: String, val offsetDesc: String, val outputRate: Double) + +/** Companion object, primarily for creating SinkStatus instances internally */ +private[sql] object SinkStatus diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 9b82b92dd37a1..22332cc58067b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -36,3 +36,6 @@ case class SourceStatus private( val inputRate: Double, val processingRate: Double, val triggerStatus: Map[String, String]) + +/** Companion object, primarily for creating SourceStatus instances internally */ +private[sql] object SourceStatus diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index 3dcdd510e7091..ffbb846c1a046 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -42,3 +42,6 @@ case class StreamingQueryInfo private( val sinkStatus: SinkStatus, val triggerStatus: Map[String, String] ) + +/** Companion object, primarily for creating StreamingQueryInfo instances internally */ +private[sql] object StreamingQueryInfo diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 0943c48a7ec4c..a5e736519719c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -127,7 +127,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - test("single listener, check trigger infos") { + test("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ clock = new ManualClock() From bbd0d8bacae529cdb5e43b5165e3c687c5c9ec05 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 18:31:19 -0700 Subject: [PATCH 14/43] Addressed comments --- .../execution/streaming/StreamExecution.scala | 36 +++++++++---------- .../execution/streaming/StreamMetrics.scala | 15 +++++--- .../spark/sql/streaming/SinkStatus.scala | 4 +-- .../spark/sql/streaming/SourceStatus.scala | 6 ++-- .../sql/streaming/StreamingQueryInfo.scala | 21 +++++++---- 5 files changed, 48 insertions(+), 34 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 9bfaf1a187e91..5f792c3f2ba22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -116,8 +116,8 @@ class StreamExecution( /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() - private val streamMetrics = new StreamMetrics(uniqueSources.toSet, triggerClock, - s"StructuredStreaming.$name") + private val streamMetrics = + new StreamMetrics(uniqueSources.toSet, triggerClock, s"StructuredStreaming.$name") /** * The thread that runs the micro-batches of this stream. Note that this thread must be @@ -146,7 +146,17 @@ class StreamExecution( override def isActive: Boolean = state == ACTIVE override def status: StreamingQueryInfo = statusLock.synchronized { - this.toInfo + StreamingQueryInfo( + name = name, + id = id, + timestamp = triggerClock.getTimeMillis(), + inputRate = streamMetrics.currentInputRate, + processingRate = streamMetrics.currentProcessingRate, + outputRate = streamMetrics.currentOutputRate, + latency = streamMetrics.currentLatency, + sourceStatuses = sourceStatuses, + sinkStatus = sinkStatus, + triggerStatus = streamMetrics.currentTriggerStatus) } /** Returns current status of all the sources. */ @@ -200,7 +210,7 @@ class StreamExecution( // so must mark this as ACTIVE first. state = ACTIVE sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) - postEvent(new QueryStarted(this.toInfo)) // Assumption: Does not throw exception. + postEvent(new QueryStarted(status)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -243,7 +253,7 @@ class StreamExecution( } } statusLock.synchronized { streamMetrics.reportTriggerFinished() } - postEvent(new QueryProgress(this.toInfo)) + postEvent(new QueryProgress(status)) isTerminated }) } catch { @@ -260,7 +270,7 @@ class StreamExecution( sparkSession.streams.notifyQueryTermination(StreamExecution.this) streamMetrics.stop() sparkSession.sparkContext.env.metricsSystem.removeSource(streamMetrics) - postEvent(new QueryTerminated(this.toInfo, exception.map(_.cause).map(Utils.exceptionString))) + postEvent(new QueryTerminated(status, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } @@ -627,20 +637,6 @@ class StreamExecution( streamMetrics.reportTriggerStatus(string, triggerClock.getTimeMillis) } - private def toInfo: StreamingQueryInfo = { - StreamingQueryInfo( - this.name, - this.id, - triggerClock.getTimeMillis(), - streamMetrics.currentInputRate, - streamMetrics.currentProcessingRate, - streamMetrics.currentOutputRate, - streamMetrics.currentLatency, - this.sourceStatuses, - this.sinkStatus, - streamMetrics.currentTriggerStatus) - } - trait State case object INITIALIZED extends State case object ACTIVE extends State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 402c6d5e43fbf..3bb635df97c4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -23,11 +23,18 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.{Source => CodahaleSource} -import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.Clock +/** + * Class that manages all the metrics related to a StreamingQuery. It does the following. + * - Calculates metrics (rates, latencies, etc.) based on information reported by StreamExecution. + * - Allows the current metric values to be queried + * - Serves some of the metrics through Codahale/DropWizard metrics + * + * @param sources Unique set of sources in a query + * @param triggerClock Clock used for triggering in StreamExecution + * @param codahaleSourceName Root name for all the Codahale metrics + */ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceName: String) extends CodahaleSource with Logging { @@ -201,7 +208,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam } object StreamMetrics extends Logging { - + /** Simple utility class to calculate rate while avoiding DivideByZero */ class RateCalculator { @volatile private var rate: Option[Double] = None diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 52bb8a60002e7..1c3d52cac523a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -22,11 +22,11 @@ import org.apache.spark.sql.execution.streaming.Sink /** * :: Experimental :: - * Status and metrics of a streaming [[Sink]]. + * Status and metrics of a streaming sink. * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current offset up to which data has been written by the sink - * @param outputRate Current output rate as rows / second + * @param outputRate Current rate (rows/sec) at which the query is writing data to the sink * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 22332cc58067b..35238b5b8a6b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -25,8 +25,10 @@ import org.apache.spark.sql.execution.streaming.Source * Status and metrics of a streaming [[Source]]. * * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current [[Source]] offset if known - * @param inputRate Current ingestion rate as rows / second + * @param offsetDesc Description of the current offset if known + * @param inputRate Rate (rows/sec) at which data is being generated by the source + * @param processingRate Rate (rows/sec) at which the query is processing data from the source + * @param triggerStatus Low-level detailed status of the last completed/currently active trigger * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala index ffbb846c1a046..1986fafa9ee7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala @@ -23,11 +23,20 @@ import org.apache.spark.annotation.Experimental * :: Experimental :: * A class used to report information about the progress of a [[StreamingQuery]]. * - * @param name The [[StreamingQuery]] name. This name is unique across all active queries. - * @param id The [[StreamingQuery]] id. This id is unique across - * all queries that have been started in the current process. - * @param sourceStatuses The current statuses of the [[StreamingQuery]]'s sources. - * @param sinkStatus The current status of the [[StreamingQuery]]'s sink. + * @param name The query name. This name is unique across all active queries. + * @param id The query id. This id is unique across + * all queries that have been started in the current process. + * @param timestamp Timestamp (ms) of when this query was generated + * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * all the sources + * @param outputRate Current rate (rows/sec) at which the query is writing data to the sink + * @param latency Current average latency between the data being available in source and the sink + * writing the corresponding output + * @param sourceStatuses Current statuses of the sources. + * @param sinkStatus Current status of the sink. + * @param triggerStatus Low-level detailed status of the last completed/currently active trigger + * @since 2.0.0 */ @Experimental case class StreamingQueryInfo private( @@ -37,7 +46,7 @@ case class StreamingQueryInfo private( val inputRate: Double, val processingRate: Double, val outputRate: Double, - val latencyMs: Option[Double], + val latency: Option[Double], val sourceStatuses: Seq[SourceStatus], val sinkStatus: SinkStatus, val triggerStatus: Map[String, String] From 02603c7f56c8722d9003d09e40889084122ba40d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 18:51:11 -0700 Subject: [PATCH 15/43] Minor --- .../scala/org/apache/spark/sql/streaming/SourceStatus.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 35238b5b8a6b0..f33f9296ae920 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -26,8 +26,9 @@ import org.apache.spark.sql.execution.streaming.Source * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current offset if known - * @param inputRate Rate (rows/sec) at which data is being generated by the source - * @param processingRate Rate (rows/sec) at which the query is processing data from the source + * @param inputRate Current rate (rows/sec) at which data is being generated by the source + * @param processingRate Current rate (rows/sec) at which the query is processing data from + * the source * @param triggerStatus Low-level detailed status of the last completed/currently active trigger * @since 2.0.0 */ From 9fd681536bf8200af4b448f87e8cdbf17df2c0ba Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 18:54:16 -0700 Subject: [PATCH 16/43] Fixed unnecessary mima change --- project/MimaExcludes.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b1fc389453e7c..a64db3febbe0a 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -54,7 +54,6 @@ object MimaExcludes { 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.columnExists"), // [SPARK-17731] Metrics for structured streaming ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"), From f5732a50da7f0df326f52ad9b85da3876ecfafbc Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 20:17:12 -0700 Subject: [PATCH 17/43] Addressed more comments --- .../execution/streaming/StreamExecution.scala | 105 ++++++++---------- .../StreamingQueryListenerSuite.scala | 73 ++++++++++-- .../sql/streaming/StreamingQuerySuite.scala | 95 ++++++++++------ 3 files changed, 169 insertions(+), 104 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 5f792c3f2ba22..d7487799d0072 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -588,19 +588,60 @@ class StreamExecution( * Report row metrics of the executed trigger * @param triggerExecutionPlan Execution plan of the trigger * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan - * @param sourceToDataframe Source to DataFrame returned by the source.getBatch in this trigger + * @param sourceToDF Source to DataFrame returned by the source.getBatch in this trigger */ private def reportMetrics( triggerExecutionPlan: SparkPlan, triggerLogicalPlan: LogicalPlan, - sourceToDataframe: Map[Source, DataFrame]): Unit = { - val sourceToNumInputRows = StreamExecution.getNumInputRowsFromTrigger( - triggerExecutionPlan, triggerLogicalPlan, sourceToDataframe) + sourceToDF: Map[Source, DataFrame]): Unit = { + // We want to associate execution plan leaves to sources that generate them, so that we match + // the their metrics (e.g. numOutputRows) to the sources. To do this we do the following. + // Consider the translation from the streaming logical plan to the final executed plan. + // + // streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan + // + // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan + // - Each logical plan leaf will be associated with a single streaming source. + // - There can be multiple logical plan leaves associated a streaming source. + // - There can be leaves not associated with any streaming source, because they were + // generated from a batch source (e.g. stream-batch joins) + // + // 2. Assuming that the executed plan has same number of leaves in the same order as that of + // the trigger logical plan, we associate executed plan leaves with corresponding + // streaming sources. + // + // 3. For each source, we sum the metrics of the associated execution plan leaves. + // + val logicalPlanLeafToSource = sourceToDF.flatMap { case (source, df) => + df.logicalPlan.collectLeaves().map { leaf => leaf -> source } + } + val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // includes non-streaming sources + val allExecPlanLeaves = triggerExecutionPlan.collectLeaves() + val sourceToNumInputRows: Map[Source, Long] = + if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) { + val execLeafToSource = allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap { + case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => ep -> source } + } + val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, source) => + val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) + source -> numRows + } + sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source + } else { + def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" + logDebug( + "Could not report metrics as number leaves in trigger logical plan did not match that" + + s" of the execution plan:\n" + + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + + s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + Map.empty + } val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) val stateNodes = triggerExecutionPlan.collect { case p if p.isInstanceOf[StateStoreSaveExec] => p } - statusLock.synchronized {streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) + statusLock.synchronized { + streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) stateNodes.zipWithIndex.foreach { case (s, i) => streamMetrics.reportTriggerStatus( NUM_TOTAL_STATE_ROWS(i + 1), @@ -646,59 +687,5 @@ class StreamExecution( object StreamExecution extends Logging { private val _nextId = new AtomicLong(0) - /** - * Get the number of input rows from the executed plan of the trigger - * @param triggerExecutionPlan Execution plan of the trigger - * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan - * @param sourceToDataframe Source to DataFrame returned by the source.getBatch in this trigger - */ - def getNumInputRowsFromTrigger( - triggerExecutionPlan: SparkPlan, - triggerLogicalPlan: LogicalPlan, - sourceToDataframe: Map[Source, DataFrame]): Map[Source, Long] = { - - // We want to associate execution plan leaves to sources that generate them, so that we match - // the their metrics (e.g. numOutputRows) to the sources. To do this we do the following. - // Consider the translation from the streaming logical plan to the final executed plan. - // - // streaming logical plan (with sources) <==> trigger's logical plan <==> executed plan - // - // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan - // - Each logical plan leaf will be associated with a single streaming source. - // - There can be multiple logical plan leaves associated a streaming source. - // - There can be leaves not associated with any streaming source, because they were - // generated from a batch source (e.g. stream-batch joins) - // - // 2. Assuming that the executed plan has same number of leaves in the same order as that of - // the trigger logical plan, we associate executed plan leaves with corresponding - // streaming sources. - // - // 3. For each source, we sum the metrics of the associated execution plan leaves. - // - val logicalPlanLeafToSource = sourceToDataframe.flatMap { case (source, df) => - df.logicalPlan.collectLeaves().map { leaf => leaf -> source } - } - val allLogicalPlanLeaves = triggerLogicalPlan.collectLeaves() // includes non-streaming sources - val allExecPlanLeaves = triggerExecutionPlan.collectLeaves() - if (allLogicalPlanLeaves.size == allExecPlanLeaves.size) { - val execLeafToSource = allLogicalPlanLeaves.zip(allExecPlanLeaves).flatMap { - case (lp, ep) => logicalPlanLeafToSource.get(lp).map { source => ep -> source } - } - val sourceToNumInputRows = execLeafToSource.map { case (execLeaf, source) => - val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) - source -> numRows - } - sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source - } else { - def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" - logWarning( - "Could not report metrics as number leaves in trigger logical plan did not match that" + - s" of the execution plan:\n" + - s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + - s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") - Map.empty - } - } - def nextId: Long = _nextId.getAndIncrement() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index a5e736519719c..16f52ba25cfe2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.streaming +import java.io.File + import scala.collection.mutable import org.scalactic.TolerantNumerics @@ -28,9 +30,10 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ -import org.apache.spark.util.{JsonProtocol, ManualClock} +import org.apache.spark.util.{JsonProtocol, ManualClock, Utils} class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { @@ -38,6 +41,49 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ import StreamingQueryListener._ + + abstract class AddFileData extends AddData { + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active file stream source") + + val sources = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => + source.asInstanceOf[FileStreamSource] + } + if (sources.isEmpty) { + throw new Exception( + "Could not find file source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the file source in the StreamExecution logical plan as there" + + "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } + val source = sources.head + val newOffset = source.withBatchingLocked { + addData(source) + source.currentOffset + 1 + } + logInfo(s"Added file to $source at offset $newOffset") + (source, newOffset) + } + + protected def addData(source: FileStreamSource): Unit + } + + case class AddTextFileData(content: String, src: File, tmp: File) + extends AddFileData { + + override def addData(source: FileStreamSource): Unit = { + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + src.mkdirs() + require(stringToFile(tempFile, content).renameTo(finalFile)) + logInfo(s"Written text '$content' to file $finalFile") + } + } + // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) @@ -46,18 +92,21 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(spark.streams.active.isEmpty) assert(addedListeners.isEmpty) // Make sure we don't leak any events to the next test - spark.sparkContext.listenerBus.waitUntilEmpty(10000) } - test("single listener, check statuses") { + test("codegen") { withTempDir { case src => + val temp = org.apache.spark.util.Utils.createTempDir().getCanonicalFile + + val input = spark.readStream.format("text").load(src.getCanonicalPath).as[String] + val listener = new QueryStatusCollector - val input = MemoryStream[Int] - // This is to make sure that + // This is to make sure that spark.sparkContext.listenerBus.waitUntilEmpty(10000) + // - Query takes non-zero time to compute // - Exec plan ends with a node (filter) that supports the numOutputRows metric - spark.conf.set("spark.sql.codegen.wholeStage", false) - val df = input.toDS.map { x => Thread.sleep(10); x }.toDF("value").where("value != 0") + // spark.conf.set("spark.sql.codegen.wholeStage", false) + val df = input.map { x => Thread.sleep(10); x.toInt }.toDF("value").where("value != 0") withListenerAdded(listener) { testStream(df)( @@ -68,7 +117,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.name === query.name) assert(status.id === query.id) assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).description.contains("Memory")) + assert(status.sourceStatuses(0).description.contains("File")) // The source and sink offsets must be None as this must be called before the // batches have started @@ -85,8 +134,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(listener.terminationStatus === null) true }, - AddDataMemory(input, Seq(1, 2, 3)), - CheckAnswer(1, 2, 3), + AddTextFileData("1", src, temp), + CheckAnswer(1), AssertOnQuery("Incorrect query status in onQueryProgress") { query => eventually(Timeout(streamingTimeout)) { assert(listener.lastTriggerStatus.nonEmpty) @@ -100,7 +149,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.sourceStatuses(0).inputRate >= 0.0) // flaky if checked for == assert(status.sourceStatuses(0).processingRate > 0.0) assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(status.sinkStatus.outputRate !== 0.0) + assert(status.sinkStatus.outputRate === 0.0) // No termination events assert(listener.terminationStatus === null) @@ -125,7 +174,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } ) } - } + }} test("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 4aadfeebc535a..d945409380292 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming._ @@ -208,41 +209,30 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } - // This tests whether row stats are correctly associated with streaming sources when - // streaming plan also has batch sources - test("calculating input rows with mixed batch and streaming sources") { - - // A streaming source that returns a pre-determined dataframe for a trigger - // Creates multiple leaves for a streaming source - val streamingTriggerDF = spark.createDataset(1 to 5).toDF.union( - spark.createDataset(6 to 10).toDF) - val streamingSource = new Source() { - override def schema: StructType = StructType(Seq(StructField("value", IntegerType))) - override def getOffset: Option[Offset] = Some(LongOffset(0)) - override def getBatch(start: Option[Offset], end: Offset): DataFrame = streamingTriggerDF - override def stop(): Unit = {} - } - val streamingInputDF = StreamingExecutionRelation(streamingSource).toDF("value") + test("input row calculation with mixed batch and streaming sources") { + val streamingTriggerDF = spark.createDataset(1 to 10).toDF + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") - // streaming trigger input has 10 items, static input has 2 items, input row calculation for - // the source should return 10 - spark.conf.set("spark.sql.codegen.wholeStage", false) - testStream(streamingInputDF.join(staticInputDF, "value"))( - StartStream(), - AssertOnQuery { q => - eventually(Timeout(streamingTimeout)) { - assert(q.status.sinkStatus.offsetDesc - === CompositeOffset.fill(LongOffset(0)).toString) - } - val numInputRows = StreamExecution.getNumInputRowsFromTrigger( - q.lastExecution.executedPlan, - q.lastExecution.logical, - Map(streamingSource -> streamingTriggerDF)) - assert(numInputRows === Map(streamingSource -> 10)) // streaming data has 10 items - true - } - ) + // Trigger input has 10 rows, static input has 2 rows, + // therefore after the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF.join(staticInputDF, "value")) + assert(status.triggerStatus("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "10") + } + + test("input row calculation with trigger DF having multiple leaves") { + val streamingTriggerDF = + spark.createDataset(1 to 5).toDF.union(spark.createDataset(6 to 10).toDF) + require(streamingTriggerDF.logicalPlan.collectLeaves().size > 1) + val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF) + + // After the first trigger, the calculated input rows should be 10 + val status = getFirstTriggerStatus(streamingInputDF) + assert(status.triggerStatus("numRows.input.total") === "10") + assert(status.sourceStatuses.size === 1) + assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "10") } testQuietly("StreamExecution metadata garbage collection") { @@ -269,6 +259,45 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter { ) } + /** Create a streaming DF that only execute one batch in which it returns the given static DF */ + private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = { + require(!triggerDF.isStreaming) + // A streaming Source that generate only on trigger and returns the given Dataframe as batch + val source = new Source() { + override def schema: StructType = triggerDF.schema + override def getOffset: Option[Offset] = Some(LongOffset(0)) + override def getBatch(start: Option[Offset], end: Offset): DataFrame = triggerDF + override def stop(): Unit = {} + } + StreamingExecutionRelation(source) + } + + /** Returns the query status at the end of the first trigger of streaming DF */ + private def getFirstTriggerStatus(streamingDF: DataFrame): StreamingQueryInfo = { + // A StreamingQueryListener that gets the query status after the first completed trigger + val listener = new StreamingQueryListener { + @volatile var firstStatus: StreamingQueryInfo = null + override def onQueryStarted(queryStarted: QueryStarted): Unit = { } + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + if (firstStatus == null) firstStatus = queryProgress.queryInfo + } + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } + } + + try { + spark.streams.addListener(listener) + val q = streamingDF.writeStream.format("memory").queryName("test").start() + q.processAllAvailable() + eventually(timeout(streamingTimeout)) { + assert(listener.firstStatus != null) + } + listener.firstStatus + } finally { + spark.streams.active.map(_.stop()) + spark.streams.removeListener(listener) + } + } + /** * A [[StreamAction]] to test the behavior of `StreamingQuery.awaitTermination()`. * From e708b3b86a69833169962713ce8bef88bcbdc2f7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 4 Oct 2016 21:29:43 -0700 Subject: [PATCH 18/43] Added periodic warning --- .../execution/streaming/PeriodicWarning.scala | 47 +++++++++++++++++++ .../execution/streaming/StreamExecution.scala | 5 +- .../sql/streaming/StreamingQuerySuite.scala | 11 +++-- 3 files changed, 56 insertions(+), 7 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala new file mode 100644 index 0000000000000..a10c0b8b1a9c2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.streaming + +import java.util.concurrent.ConcurrentHashMap + +import org.apache.spark.internal.Logging + +/** Trait that adds a method to allow warning to be only printed once in every X seconds */ +trait PeriodicWarning { self: Logging => + case class CallLocation(className: String, lineNum: Int, threadId: Long) + private val lastLogTimes = new ConcurrentHashMap[CallLocation, Long] + + /** + * Log warning only once every `periodSecs`. Use this when you dont want a warning to + * get printed too often. For example, a warning generated from a fast loop. + */ + def logPeriodicWarning(periodSecs: Long, msg: => String): Unit = { + val callLoc = callLocation() + lastLogTimes.putIfAbsent(callLoc, 0) + val now = System.currentTimeMillis + if (now - lastLogTimes.get(callLoc) > periodSecs * 1000) { + logWarning(msg) + lastLogTimes.put(callLoc, now) + } + } + + private def callLocation(): CallLocation = { + val th = Thread.currentThread() + val e = th.getStackTrace.head + CallLocation(e.getClassName, e.getLineNumber, th.getId) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index d7487799d0072..28414e6c3a692 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -53,7 +53,7 @@ class StreamExecution( val trigger: Trigger, val triggerClock: Clock, val outputMode: OutputMode) - extends StreamingQuery with Logging { + extends StreamingQuery with Logging with PeriodicWarning { import org.apache.spark.sql.streaming.StreamingQueryListener._ import StreamMetrics._ @@ -629,7 +629,8 @@ class StreamExecution( sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source } else { def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" - logDebug( + logPeriodicWarning( + periodSecs = 60, "Could not report metrics as number leaves in trigger logical plan did not match that" + s" of the execution plan:\n" + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index d945409380292..bb0e81fc26125 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,22 +17,23 @@ package org.apache.spark.sql.streaming -import java.util.concurrent.atomic.AtomicInteger - import org.scalactic.TolerantNumerics import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame import org.apache.spark.sql.streaming.StreamingQueryListener._ -import org.apache.spark.sql.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.Utils -class StreamingQuerySuite extends StreamTest with BeforeAndAfter { +class StreamingQuerySuite extends StreamTest + with BeforeAndAfter + with Logging + with PeriodicWarning { import AwaitTerminationTester._ import testImplicits._ From a47c73a17160d69d38369c3e56f81b1c7256dd8d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Oct 2016 11:15:05 -0700 Subject: [PATCH 19/43] Remove unnecessary test --- .../StreamingQueryListenerSuite.scala | 125 ------------------ .../sql/streaming/StreamingQuerySuite.scala | 48 +++++++ 2 files changed, 48 insertions(+), 125 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 16f52ba25cfe2..cc4f74d934a0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -41,49 +41,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ import StreamingQueryListener._ - - abstract class AddFileData extends AddData { - override def addData(query: Option[StreamExecution]): (Source, Offset) = { - require( - query.nonEmpty, - "Cannot add data when there is no query for finding the active file stream source") - - val sources = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => - source.asInstanceOf[FileStreamSource] - } - if (sources.isEmpty) { - throw new Exception( - "Could not find file source in the StreamExecution logical plan to add data to") - } else if (sources.size > 1) { - throw new Exception( - "Could not select the file source in the StreamExecution logical plan as there" + - "are multiple file sources:\n\t" + sources.mkString("\n\t")) - } - val source = sources.head - val newOffset = source.withBatchingLocked { - addData(source) - source.currentOffset + 1 - } - logInfo(s"Added file to $source at offset $newOffset") - (source, newOffset) - } - - protected def addData(source: FileStreamSource): Unit - } - - case class AddTextFileData(content: String, src: File, tmp: File) - extends AddFileData { - - override def addData(source: FileStreamSource): Unit = { - val tempFile = Utils.tempFileWith(new File(tmp, "text")) - val finalFile = new File(src, tempFile.getName) - src.mkdirs() - require(stringToFile(tempFile, content).renameTo(finalFile)) - logInfo(s"Written text '$content' to file $finalFile") - } - } - // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) @@ -94,88 +51,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Make sure we don't leak any events to the next test } - test("codegen") { withTempDir { case src => - val temp = org.apache.spark.util.Utils.createTempDir().getCanonicalFile - - val input = spark.readStream.format("text").load(src.getCanonicalPath).as[String] - - val listener = new QueryStatusCollector - - // This is to make sure that spark.sparkContext.listenerBus.waitUntilEmpty(10000) - - // - Query takes non-zero time to compute - // - Exec plan ends with a node (filter) that supports the numOutputRows metric - // spark.conf.set("spark.sql.codegen.wholeStage", false) - val df = input.map { x => Thread.sleep(10); x.toInt }.toDF("value").where("value != 0") - - withListenerAdded(listener) { - testStream(df)( - StartStream(), - AssertOnQuery("Incorrect query status in onQueryStarted") { query => - val status = listener.startStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).description.contains("File")) - - // The source and sink offsets must be None as this must be called before the - // batches have started - assert(status.sourceStatuses(0).offsetDesc === None) - assert(status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString) - assert(status.sinkStatus.outputRate === 0.0) - - // The source and sink rates must be None as this must be called before the batches - // have started - assert(status.sourceStatuses(0).inputRate === 0.0) - assert(status.sourceStatuses(0).processingRate === 0.0) - - // No progress events or termination events - assert(listener.terminationStatus === null) - true - }, - AddTextFileData("1", src, temp), - CheckAnswer(1), - AssertOnQuery("Incorrect query status in onQueryProgress") { query => - eventually(Timeout(streamingTimeout)) { - assert(listener.lastTriggerStatus.nonEmpty) - } - // Check the correctness of data in the latest query info reported by onQueryProgress - val status = listener.lastTriggerStatus.get - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sourceStatuses(0).inputRate >= 0.0) // flaky if checked for == - assert(status.sourceStatuses(0).processingRate > 0.0) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(status.sinkStatus.outputRate === 0.0) - - // No termination events - assert(listener.terminationStatus === null) - true - }, - StopStream, - AssertOnQuery("Incorrect query status in onQueryTerminated") { query => - eventually(Timeout(streamingTimeout)) { - val status = listener.terminationStatus - assert(status != null) - assert(status.name === query.name) - assert(status.id === query.id) - assert(status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)) - assert(status.sourceStatuses(0).inputRate === 0.0) - assert(status.sourceStatuses(0).processingRate === 0.0) - assert(status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString) - assert(status.sinkStatus.outputRate === 0.0) - assert(listener.terminationException === None) - } - listener.checkAsyncErrors() - true - } - ) - } - }} - test("single listener, check trigger statuses") { import StreamingQueryListenerSuite._ clock = new ManualClock() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index bb0e81fc26125..cb799f7427c59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,12 +17,15 @@ package org.apache.spark.sql.streaming +import java.io.File + import org.scalactic.TolerantNumerics import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException @@ -45,6 +48,51 @@ class StreamingQuerySuite extends StreamTest sqlContext.streams.active.foreach(_.stop()) } + abstract class AddFileData extends AddData { + override def addData(query: Option[StreamExecution]): (Source, Offset) = { + require( + query.nonEmpty, + "Cannot add data when there is no query for finding the active file stream source") + + val sources = query.get.logicalPlan.collect { + case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => + source.asInstanceOf[FileStreamSource] + } + if (sources.isEmpty) { + throw new Exception( + "Could not find file source in the StreamExecution logical plan to add data to") + } else if (sources.size > 1) { + throw new Exception( + "Could not select the file source in the StreamExecution logical plan as there" + + "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } + val source = sources.head + val newOffset = source.withBatchingLocked { + addData(source) + source.currentOffset + 1 + } + logInfo(s"Added file to $source at offset $newOffset") + (source, newOffset) + } + + protected def addData(source: FileStreamSource): Unit + } + + case class AddTextFileData(content: String, src: File, tmp: File) + extends AddFileData { + + override def addData(source: FileStreamSource): Unit = { + val tempFile = Utils.tempFileWith(new File(tmp, "text")) + val finalFile = new File(src, tempFile.getName) + src.mkdirs() + require(stringToFile(tempFile, content).renameTo(finalFile)) + logInfo(s"Written text '$content' to file $finalFile") + } + } + + + + test("names unique across active queries, ids unique across all started queries") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map { 6 / _} From f883b48df79bebc2d5c342aee1e56906c217a9c4 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Oct 2016 16:56:13 -0700 Subject: [PATCH 20/43] Removed locks --- .../execution/streaming/PeriodicWarning.scala | 47 ----- .../execution/streaming/StreamExecution.scala | 192 +++++++++--------- .../StreamingQueryListenerSuite.scala | 1 - .../sql/streaming/StreamingQuerySuite.scala | 5 +- 4 files changed, 96 insertions(+), 149 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala deleted file mode 100644 index a10c0b8b1a9c2..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/PeriodicWarning.scala +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.streaming - -import java.util.concurrent.ConcurrentHashMap - -import org.apache.spark.internal.Logging - -/** Trait that adds a method to allow warning to be only printed once in every X seconds */ -trait PeriodicWarning { self: Logging => - case class CallLocation(className: String, lineNum: Int, threadId: Long) - private val lastLogTimes = new ConcurrentHashMap[CallLocation, Long] - - /** - * Log warning only once every `periodSecs`. Use this when you dont want a warning to - * get printed too often. For example, a warning generated from a fast loop. - */ - def logPeriodicWarning(periodSecs: Long, msg: => String): Unit = { - val callLoc = callLocation() - lastLogTimes.putIfAbsent(callLoc, 0) - val now = System.currentTimeMillis - if (now - lastLogTimes.get(callLoc) > periodSecs * 1000) { - logWarning(msg) - lastLogTimes.put(callLoc, now) - } - } - - private def callLocation(): CallLocation = { - val th = Thread.currentThread() - val e = th.getStackTrace.head - CallLocation(e.getClassName, e.getLineNumber, th.getId) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 28414e6c3a692..68dcc6f55b8d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -53,7 +53,7 @@ class StreamExecution( val trigger: Trigger, val triggerClock: Clock, val outputMode: OutputMode) - extends StreamingQuery with Logging with PeriodicWarning { + extends StreamingQuery with Logging { import org.apache.spark.sql.streaming.StreamingQueryListener._ import StreamMetrics._ @@ -69,12 +69,6 @@ class StreamExecution( private val startLatch = new CountDownLatch(1) private val terminationLatch = new CountDownLatch(1) - /** - * Lock object used to synchronized updates to data structures that are used to report - * query status. - */ - private val statusLock = new Object - /** * Tracks how much data we have processed and committed to the sink or state store from each * input source. @@ -116,9 +110,16 @@ class StreamExecution( /* Get the call site in the caller thread; will pass this into the micro batch thread */ private val callSite = Utils.getCallSite() + /** Metrics for this query */ private val streamMetrics = new StreamMetrics(uniqueSources.toSet, triggerClock, s"StructuredStreaming.$name") + @volatile + private var currentStatus: StreamingQueryInfo = null + + @volatile + private var metricWarningLogged: Boolean = false + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to avoid potential deadlocks in using @@ -145,40 +146,14 @@ class StreamExecution( /** Whether the query is currently active or not */ override def isActive: Boolean = state == ACTIVE - override def status: StreamingQueryInfo = statusLock.synchronized { - StreamingQueryInfo( - name = name, - id = id, - timestamp = triggerClock.getTimeMillis(), - inputRate = streamMetrics.currentInputRate, - processingRate = streamMetrics.currentProcessingRate, - outputRate = streamMetrics.currentOutputRate, - latency = streamMetrics.currentLatency, - sourceStatuses = sourceStatuses, - sinkStatus = sinkStatus, - triggerStatus = streamMetrics.currentTriggerStatus) - } + /** Returns the current status of the query. */ + override def status: StreamingQueryInfo = currentStatus /** Returns current status of all the sources. */ - override def sourceStatuses: Array[SourceStatus] = statusLock.synchronized { - val localAvailableOffsets = availableOffsets - sources.map(s => - SourceStatus( - s.toString, - localAvailableOffsets.get(s).map(_.toString), - streamMetrics.currentSourceInputRate(s), - streamMetrics.currentSourceProcessingRate(s), - streamMetrics.currentSourceTriggerStatus(s)) - ).toArray - } + override def sourceStatuses: Array[SourceStatus] = currentStatus.sourceStatuses.toArray /** Returns current status of the sink. */ - override def sinkStatus: SinkStatus = statusLock.synchronized { - SinkStatus( - sink.toString, - committedOffsets.toCompositeOffset(sources).toString, - streamMetrics.currentOutputRate()) - } + override def sinkStatus: SinkStatus = currentStatus.sinkStatus /** Returns the [[StreamingQueryException]] if the query was terminated by an exception. */ override def exception: Option[StreamingQueryException] = Option(streamDeathCause) @@ -210,7 +185,8 @@ class StreamExecution( // so must mark this as ACTIVE first. state = ACTIVE sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) - postEvent(new QueryStarted(status)) // Assumption: Does not throw exception. + updateStatus() + postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. // Unblock starting thread startLatch.countDown() @@ -219,10 +195,9 @@ class StreamExecution( SparkSession.setActiveSession(sparkSession) triggerExecutor.execute(() => { - statusLock.synchronized { - streamMetrics.reportTriggerStarted(currentBatchId) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Finding new data from sources") - } + streamMetrics.reportTriggerStarted(currentBatchId) + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Finding new data from sources") + updateStatus() val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { if (isActive) { if (currentBatchId < 0) { @@ -233,18 +208,16 @@ class StreamExecution( constructNextBatch() } if (dataAvailable) { - statusLock.synchronized { - streamMetrics.reportTriggerStatus(DATA_AVAILABLE, true) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Processing new data") - } + streamMetrics.reportTriggerStatus(DATA_AVAILABLE, true) + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Processing new data") + updateStatus() runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 } else { - statusLock.synchronized { - streamMetrics.reportTriggerStatus(DATA_AVAILABLE, false) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "No new data") - } + streamMetrics.reportTriggerStatus(DATA_AVAILABLE, false) + streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "No new data") + updateStatus() Thread.sleep(pollingDelayMs) } true @@ -252,8 +225,9 @@ class StreamExecution( false } } - statusLock.synchronized { streamMetrics.reportTriggerFinished() } - postEvent(new QueryProgress(status)) + streamMetrics.reportTriggerFinished() + updateStatus() + postEvent(new QueryProgress(currentStatus)) isTerminated }) } catch { @@ -286,17 +260,15 @@ class StreamExecution( private def populateStartOffsets(): Unit = { offsetLog.getLatest() match { case Some((batchId, nextOffsets)) => - statusLock.synchronized { - logInfo(s"Resuming streaming query, starting with batch $batchId") - currentBatchId = batchId - availableOffsets = nextOffsets.toStreamProgress(sources) - logDebug(s"Found possibly uncommitted offsets $availableOffsets") - - offsetLog.get(batchId - 1).foreach { - case lastOffsets => - committedOffsets = lastOffsets.toStreamProgress(sources) - logDebug(s"Resuming with committed offsets: $committedOffsets") - } + logInfo(s"Resuming streaming query, starting with batch $batchId") + currentBatchId = batchId + availableOffsets = nextOffsets.toStreamProgress(sources) + logDebug(s"Found possibly uncommitted offsets $availableOffsets") + + offsetLog.get(batchId - 1).foreach { + case lastOffsets => + committedOffsets = lastOffsets.toStreamProgress(sources) + logDebug(s"Resuming with committed offsets: $committedOffsets") } case None => // We are starting this stream for the first time. logInfo(s"Starting new streaming query.") @@ -333,9 +305,7 @@ class StreamExecution( (s, s.getOffset) } }.toMap - statusLock.synchronized { - availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) - } + availableOffsets ++= latestOffsets.filter { case (s, o) => o.nonEmpty }.mapValues(_.get) } if (dataAvailable) { @@ -436,7 +406,7 @@ class StreamExecution( val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) sink.addBatch(currentBatchId, nextBatch) - reportMetrics(executedPlan, triggerLogicalPlan, newData) + reportNumRows(executedPlan, triggerLogicalPlan, newData) awaitBatchLock.lock() try { @@ -449,9 +419,7 @@ class StreamExecution( val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. - statusLock.synchronized { - committedOffsets ++= availableOffsets - } + committedOffsets ++= availableOffsets } private def postEvent(event: StreamingQueryListener.Event) { @@ -590,7 +558,7 @@ class StreamExecution( * @param triggerLogicalPlan Logical plan of the trigger, generated from the query logical plan * @param sourceToDF Source to DataFrame returned by the source.getBatch in this trigger */ - private def reportMetrics( + private def reportNumRows( triggerExecutionPlan: SparkPlan, triggerLogicalPlan: LogicalPlan, sourceToDF: Map[Source, DataFrame]): Unit = { @@ -628,39 +596,40 @@ class StreamExecution( } sourceToNumInputRows.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source } else { - def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" - logPeriodicWarning( - periodSecs = 60, - "Could not report metrics as number leaves in trigger logical plan did not match that" + - s" of the execution plan:\n" + - s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + - s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + if (!metricWarningLogged) { + def toString[T](seq: Seq[T]): String = s"(size = ${seq.size}), ${seq.mkString(", ")}" + logWarning( + "Could not report metrics as number leaves in trigger logical plan did not match that" + + s" of the execution plan:\n" + + s"logical plan leaves: ${toString(allLogicalPlanLeaves)}\n" + + s"execution plan leaves: ${toString(allExecPlanLeaves)}\n") + metricWarningLogged = true + } Map.empty } val numOutputRows = triggerExecutionPlan.metrics.get("numOutputRows").map(_.value) val stateNodes = triggerExecutionPlan.collect { case p if p.isInstanceOf[StateStoreSaveExec] => p } - statusLock.synchronized { - streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) - stateNodes.zipWithIndex.foreach { case (s, i) => - streamMetrics.reportTriggerStatus( - NUM_TOTAL_STATE_ROWS(i + 1), - s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) - streamMetrics.reportTriggerStatus( - NUM_UPDATED_STATE_ROWS(i + 1), - s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) - } + + streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) + stateNodes.zipWithIndex.foreach { case (s, i) => + streamMetrics.reportTriggerStatus( + NUM_TOTAL_STATE_ROWS(i + 1), + s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) + streamMetrics.reportTriggerStatus( + NUM_UPDATED_STATE_ROWS(i + 1), + s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) } + updateStatus() } private def reportTimeTaken[T](triggerStatusKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - statusLock.synchronized { - streamMetrics.reportTriggerStatus(triggerStatusKey, math.max(endTime - startTime, 0)) - } + streamMetrics.reportTriggerStatus(triggerStatusKey, math.max(endTime - startTime, 0)) + updateStatus() result } @@ -668,15 +637,44 @@ class StreamExecution( val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - statusLock.synchronized { - streamMetrics.reportSourceTriggerStatus( - source, triggerStatusKey, math.max(endTime - startTime, 0)) - } + streamMetrics.reportSourceTriggerStatus( + source, triggerStatusKey, math.max(endTime - startTime, 0)) + updateStatus() result } - private def reportTimestamp(string: String): Unit = statusLock.synchronized { - streamMetrics.reportTriggerStatus(string, triggerClock.getTimeMillis) + private def reportTimestamp(triggerStatusKey: String): Unit = { + streamMetrics.reportTriggerStatus(triggerStatusKey, triggerClock.getTimeMillis) + updateStatus() + } + + private def updateStatus(): Unit = { + val localAvailableOffsets = availableOffsets + val sourceStatuses = sources.map { s => + SourceStatus( + s.toString, + localAvailableOffsets.get(s).map(_.toString), + streamMetrics.currentSourceInputRate(s), + streamMetrics.currentSourceProcessingRate(s), + streamMetrics.currentSourceTriggerStatus(s)) + } + val sinkStatus = SinkStatus( + sink.toString, + committedOffsets.toCompositeOffset(sources).toString, + streamMetrics.currentOutputRate()) + + currentStatus = + StreamingQueryInfo( + name = name, + id = id, + timestamp = triggerClock.getTimeMillis(), + inputRate = streamMetrics.currentInputRate(), + processingRate = streamMetrics.currentProcessingRate(), + outputRate = streamMetrics.currentOutputRate(), + latency = streamMetrics.currentLatency(), + sourceStatuses = sourceStatuses, + sinkStatus = sinkStatus, + triggerStatus = streamMetrics.currentTriggerStatus()) } trait State diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index cc4f74d934a0d..fd46fa2cfd023 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -30,7 +30,6 @@ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ import org.apache.spark.util.{JsonProtocol, ManualClock, Utils} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index cb799f7427c59..384066afa5639 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -33,10 +33,7 @@ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.util.Utils -class StreamingQuerySuite extends StreamTest - with BeforeAndAfter - with Logging - with PeriodicWarning { +class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { import AwaitTerminationTester._ import testImplicits._ From 713f77349a596d94177db8a237faaa53bb176714 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Oct 2016 17:37:06 -0700 Subject: [PATCH 21/43] Removed output rate, renamed StreamingQueryInfo -> StreamingQueryStatus, and made API java-friendly --- .../execution/streaming/StreamExecution.scala | 14 ++-- .../execution/streaming/StreamMetrics.scala | 21 ++---- .../spark/sql/streaming/SinkStatus.scala | 4 +- .../spark/sql/streaming/SourceStatus.scala | 6 +- .../spark/sql/streaming/StreamingQuery.scala | 2 +- .../streaming/StreamingQueryListener.scala | 6 +- ...yInfo.scala => StreamingQueryStatus.scala} | 12 ++-- .../streaming/StreamingMetricsSuite.scala | 21 ++---- .../StreamingQueryListenerSuite.scala | 69 +++++++++---------- .../sql/streaming/StreamingQuerySuite.scala | 26 ++----- 10 files changed, 75 insertions(+), 106 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/streaming/{StreamingQueryInfo.scala => StreamingQueryStatus.scala} (88%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 68dcc6f55b8d9..ffc14a4e47169 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -115,7 +115,7 @@ class StreamExecution( new StreamMetrics(uniqueSources.toSet, triggerClock, s"StructuredStreaming.$name") @volatile - private var currentStatus: StreamingQueryInfo = null + private var currentStatus: StreamingQueryStatus = null @volatile private var metricWarningLogged: Boolean = false @@ -147,7 +147,7 @@ class StreamExecution( override def isActive: Boolean = state == ACTIVE /** Returns the current status of the query. */ - override def status: StreamingQueryInfo = currentStatus + override def status: StreamingQueryStatus = currentStatus /** Returns current status of all the sources. */ override def sourceStatuses: Array[SourceStatus] = currentStatus.sourceStatuses.toArray @@ -612,7 +612,7 @@ class StreamExecution( case p if p.isInstanceOf[StateStoreSaveExec] => p } - streamMetrics.reportNumRows(sourceToNumInputRows, numOutputRows) + streamMetrics.reportNumInputRows(sourceToNumInputRows) stateNodes.zipWithIndex.foreach { case (s, i) => streamMetrics.reportTriggerStatus( NUM_TOTAL_STATE_ROWS(i + 1), @@ -657,20 +657,18 @@ class StreamExecution( streamMetrics.currentSourceInputRate(s), streamMetrics.currentSourceProcessingRate(s), streamMetrics.currentSourceTriggerStatus(s)) - } + }.toArray val sinkStatus = SinkStatus( sink.toString, - committedOffsets.toCompositeOffset(sources).toString, - streamMetrics.currentOutputRate()) + committedOffsets.toCompositeOffset(sources).toString) currentStatus = - StreamingQueryInfo( + StreamingQueryStatus( name = name, id = id, timestamp = triggerClock.getTimeMillis(), inputRate = streamMetrics.currentInputRate(), processingRate = streamMetrics.currentProcessingRate(), - outputRate = streamMetrics.currentOutputRate(), latency = streamMetrics.currentLatency(), sourceStatuses = sourceStatuses, sinkStatus = sinkStatus, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 3bb635df97c4f..7080dbdf6639a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -17,7 +17,10 @@ package org.apache.spark.sql.execution.streaming +import java.{util => ju} + import scala.collection.mutable +import scala.collection.JavaConverters._ import com.codahale.metrics.{Gauge, MetricRegistry} @@ -47,7 +50,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // Rate estimators for sources and sinks private val inputRates = new mutable.HashMap[Source, RateCalculator] private val processingRates = new mutable.HashMap[Source, RateCalculator] - private val outputRate = new RateCalculator // Number of input rows in the current trigger private val numInputRows = new mutable.HashMap[Source, Long] @@ -65,7 +67,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // together in Ganglia as a single metric group registerGauge("inputRate-total", currentInputRate) registerGauge("processingRate-total", () => currentProcessingRate) - registerGauge("outputRate", () => currentOutputRate) registerGauge("latency", () => currentLatency().getOrElse(-1.0)) sources.foreach { s => @@ -100,9 +101,8 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam sourceTriggerStatus(source).put(key, value.toString) } - def reportNumRows(inputRows: Map[Source, Long], outputRows: Option[Long]): Unit = synchronized { + def reportNumInputRows(inputRows: Map[Source, Long]): Unit = synchronized { numInputRows ++= inputRows - numOutputRows = outputRows } def reportTriggerFinished(): Unit = synchronized { @@ -142,10 +142,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam processingRates(s).update(numInputRows.getOrElse(s, 0), currentTriggerDuration) } - // Update output rate = num rows output to the sink in current trigger duration - outputRate.update(numOutputRows.getOrElse(0), currentTriggerDuration) - logDebug("Output rate updated to " + outputRate.currentRate) - // Update latency = if data present, 0.5 * previous trigger interval + current trigger duration if (previousInputIntervalOption.nonEmpty && totalNumInputRows > 0) { latency = Some((previousInputIntervalOption.get.toDouble / 2) + currentTriggerDuration) @@ -179,14 +175,12 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam processingRates(source).currentRate } - def currentOutputRate(): Double = synchronized { outputRate.currentRate } - def currentLatency(): Option[Double] = synchronized { latency } - def currentTriggerStatus(): Map[String, String] = synchronized { triggerStatus.toMap } + def currentTriggerStatus(): ju.Map[String, String] = synchronized { triggerStatus.toMap.asJava } - def currentSourceTriggerStatus(source: Source): Map[String, String] = synchronized { - sourceTriggerStatus(source).toMap + def currentSourceTriggerStatus(source: Source): ju.Map[String, String] = synchronized { + sourceTriggerStatus(source).toMap.asJava } // =========== Other methods =========== @@ -202,7 +196,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def stop(): Unit = synchronized { inputRates.valuesIterator.foreach { _.stop() } processingRates.valuesIterator.foreach { _.stop() } - outputRate.stop() latency = None } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 1c3d52cac523a..ac65ac9a971fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -26,14 +26,12 @@ import org.apache.spark.sql.execution.streaming.Sink * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current offset up to which data has been written by the sink - * @param outputRate Current rate (rows/sec) at which the query is writing data to the sink * @since 2.0.0 */ @Experimental case class SinkStatus private( val description: String, - val offsetDesc: String, - val outputRate: Double) + val offsetDesc: String) /** Companion object, primarily for creating SinkStatus instances internally */ private[sql] object SinkStatus diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index f33f9296ae920..549c109c730b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.streaming +import java.{util => ju} + import org.apache.spark.annotation.Experimental import org.apache.spark.sql.execution.streaming.Source /** * :: Experimental :: - * Status and metrics of a streaming [[Source]]. + * Status and metrics of a streaming Source. * * @param description Description of the source corresponding to this status * @param offsetDesc Description of the current offset if known @@ -38,7 +40,7 @@ case class SourceStatus private( val offsetDesc: Option[String], val inputRate: Double, val processingRate: Double, - val triggerStatus: Map[String, String]) + val triggerStatus: ju.Map[String, String]) /** Companion object, primarily for creating SourceStatus instances internally */ private[sql] object SourceStatus diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index dfdbc93e953c3..757cd516a34c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -66,7 +66,7 @@ trait StreamingQuery { * Returns the current status of the query. * @since 2.1.0 */ - def status: StreamingQueryInfo + def status: StreamingQueryStatus /** * Returns current status of all the sources. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index db606abb8ce43..c2399f47f7f69 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryStarted private[sql](val queryInfo: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,7 +92,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryInfo: StreamingQueryInfo) extends Event + class QueryProgress private[sql](val queryInfo: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -107,6 +107,6 @@ object StreamingQueryListener { */ @Experimental class QueryTerminated private[sql]( - val queryInfo: StreamingQueryInfo, + val queryInfo: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala similarity index 88% rename from sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala rename to sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 1986fafa9ee7f..1df700af35378 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.streaming +import java.{util => ju} + import org.apache.spark.annotation.Experimental /** @@ -30,7 +32,6 @@ import org.apache.spark.annotation.Experimental * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources * @param processingRate Current rate (rows/sec) at which the query is processing data from * all the sources - * @param outputRate Current rate (rows/sec) at which the query is writing data to the sink * @param latency Current average latency between the data being available in source and the sink * writing the corresponding output * @param sourceStatuses Current statuses of the sources. @@ -39,18 +40,17 @@ import org.apache.spark.annotation.Experimental * @since 2.0.0 */ @Experimental -case class StreamingQueryInfo private( +case class StreamingQueryStatus private( val name: String, val id: Long, val timestamp: Long, val inputRate: Double, val processingRate: Double, - val outputRate: Double, val latency: Option[Double], - val sourceStatuses: Seq[SourceStatus], + val sourceStatuses: Array[SourceStatus], val sinkStatus: SinkStatus, - val triggerStatus: Map[String, String] + val triggerStatus: ju.Map[String, String] ) /** Companion object, primarily for creating StreamingQueryInfo instances internally */ -private[sql] object StreamingQueryInfo +private[sql] object StreamingQueryStatus diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala index e3207400f12f0..c629534db9338 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala @@ -33,7 +33,6 @@ class StreamingMetricsSuite extends SparkFunSuite { val sm = newStreamMetrics(source) assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) - assert(sm.currentOutputRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) @@ -42,19 +41,17 @@ class StreamingMetricsSuite extends SparkFunSuite { sm.reportTriggerStarted(1) assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) - assert(sm.currentOutputRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) // Finishing the trigger should calculate the rates, except input rate which needs // to have another trigger interval - sm.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows, 10 output rows clock.advance(1000) sm.reportTriggerFinished() assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 100.0) // 100 input rows processed in 1 sec - assert(sm.currentOutputRate() === 10.0) // 10 output rows generated in 1 sec assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 100.0) assert(sm.currentLatency() === None) @@ -62,12 +59,11 @@ class StreamingMetricsSuite extends SparkFunSuite { // Another trigger should calculate the input rate clock.advance(1000) sm.reportTriggerStarted(2) - sm.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows clock.advance(500) sm.reportTriggerFinished() assert(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts assert(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec - assert(sm.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec assert(sm.currentSourceInputRate(source) === 100.0) assert(sm.currentSourceProcessingRate(source) === 400.0) assert(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms @@ -76,7 +72,6 @@ class StreamingMetricsSuite extends SparkFunSuite { sm.stop() assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) - assert(sm.currentOutputRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) @@ -86,21 +81,20 @@ class StreamingMetricsSuite extends SparkFunSuite { val sm = newStreamMetrics(source) // Trigger 1 with data sm.reportTriggerStarted(1) - sm.reportNumRows(Map(source -> 100L), Some(10)) // 100 input rows, 10 output rows + sm.reportNumInputRows(Map(source -> 100L)) // 100 input rows clock.advance(1000) sm.reportTriggerFinished() // Trigger 2 with data clock.advance(1000) sm.reportTriggerStarted(2) - sm.reportNumRows(Map(source -> 200L), Some(20)) // 200 input rows, 20 output rows + sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows clock.advance(500) sm.reportTriggerFinished() // Make sure that all rates are set require(sm.currentInputRate() === 100.0) // 200 input rows generated in 2 seconds b/w starts require(sm.currentProcessingRate() === 400.0) // 200 output rows processed in 0.5 sec - require(sm.currentOutputRate() === 40.0) // 20 output rows generated in 0.5 sec require(sm.currentSourceInputRate(source) === 100.0) require(sm.currentSourceProcessingRate(source) === 400.0) require(sm.currentLatency().get === 1500.0) // 2000 ms / 2 + 500 ms @@ -114,7 +108,6 @@ class StreamingMetricsSuite extends SparkFunSuite { // Rates are set to zero and latency is set to None assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) - assert(sm.currentOutputRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) @@ -127,14 +120,14 @@ class StreamingMetricsSuite extends SparkFunSuite { val sm = newStreamMetrics(source1, source2) // Trigger 1 with data sm.reportTriggerStarted(1) - sm.reportNumRows(Map(source1 -> 100L, source2 -> 100L), Some(10)) + sm.reportNumInputRows(Map(source1 -> 100L, source2 -> 100L)) clock.advance(1000) sm.reportTriggerFinished() // Trigger 2 with data clock.advance(1000) sm.reportTriggerStarted(2) - sm.reportNumRows(Map(source1 -> 200L, source2 -> 200L), Some(20)) + sm.reportNumInputRows(Map(source1 -> 200L, source2 -> 200L)) clock.advance(500) sm.reportTriggerFinished() @@ -150,7 +143,7 @@ class StreamingMetricsSuite extends SparkFunSuite { clock.advance(500) sm.reportTriggerStarted(3) clock.advance(500) - sm.reportNumRows(Map(source1 -> 200L), Some(20)) + sm.reportNumInputRows(Map(source1 -> 200L)) sm.reportTriggerFinished() // Rates are set to zero and latency is set to None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index fd46fa2cfd023..3a78ae6657c3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,9 +17,8 @@ package org.apache.spark.sql.streaming -import java.io.File - import scala.collection.mutable +import scala.collection.JavaConverters._ import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter @@ -98,29 +97,28 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Check the correctness of the trigger info of the first completed batch reported by // onQueryProgress val status = listener.lastTriggerStatus.get - assert(status.triggerStatus("triggerId") == "0") - assert(status.triggerStatus("isActive") === "false") + assert(status.triggerStatus.get("triggerId") == "0") + assert(status.triggerStatus.get("isActive") === "false") - assert(status.triggerStatus("timestamp.triggerStart") === "0") - assert(status.triggerStatus("timestamp.afterGetOffset") === "100") - assert(status.triggerStatus("timestamp.afterGetBatch") === "300") - assert(status.triggerStatus("timestamp.triggerFinish") === "600") + assert(status.triggerStatus.get("timestamp.triggerStart") === "0") + assert(status.triggerStatus.get("timestamp.afterGetOffset") === "100") + assert(status.triggerStatus.get("timestamp.afterGetBatch") === "300") + assert(status.triggerStatus.get("timestamp.triggerFinish") === "600") - assert(status.triggerStatus("latency.getOffset") === "100") - assert(status.triggerStatus("latency.getBatch") === "200") - assert(status.triggerStatus("latency.offsetLogWrite") === "0") - assert(status.triggerStatus("latency.fullTrigger") === "600") + assert(status.triggerStatus.get("latency.getOffset") === "100") + assert(status.triggerStatus.get("latency.getBatch") === "200") + assert(status.triggerStatus.get("latency.offsetLogWrite") === "0") + assert(status.triggerStatus.get("latency.fullTrigger") === "600") - assert(status.triggerStatus("numRows.input.total") === "2") - assert(status.triggerStatus("numRows.output") === "1") - assert(status.triggerStatus("numRows.state.aggregation1.total") === "1") - assert(status.triggerStatus("numRows.state.aggregation1.updated") === "1") + assert(status.triggerStatus.get("numRows.input.total") === "2") + assert(status.triggerStatus.get("numRows.state.aggregation1.total") === "1") + assert(status.triggerStatus.get("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus("triggerId") === "0") - assert(status.sourceStatuses(0).triggerStatus("latency.sourceGetOffset") === "100") - assert(status.sourceStatuses(0).triggerStatus("latency.sourceGetBatch") === "200") - assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "2") + assert(status.sourceStatuses(0).triggerStatus.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerStatus.get("latency.sourceGetOffset") === "100") + assert(status.sourceStatuses(0).triggerStatus.get("latency.sourceGetBatch") === "200") + assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "2") true }, CheckAnswer(2) @@ -224,8 +222,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } private def assertStreamingQueryInfoEquals( - expected: StreamingQueryInfo, - actual: StreamingQueryInfo): Unit = { + expected: StreamingQueryStatus, + actual: StreamingQueryStatus): Unit = { assert(expected.name === actual.name) assert(expected.sourceStatuses.size === actual.sourceStatuses.size) expected.sourceStatuses.zip(actual.sourceStatuses).foreach { @@ -263,31 +261,30 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } - private val testQueryInfo: StreamingQueryInfo = { - StreamingQueryInfo( - "name", 1, 123, 1.0, 2.0, 3.0, Some(345), - Seq( - SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map.empty), - SourceStatus("source2", Some(LongOffset(1).toString), 1.0, 2.0, Map("a" -> "b"))), - SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString, 2.0), - Map("a" -> "b")) + private val testQueryInfo: StreamingQueryStatus = { + StreamingQueryStatus( + "name", 1, 123, 1.0, 2.0, Some(345), + Array( + SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map("a" -> "b").asJava)), + SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString), + Map("a" -> "b").asJava) } class QueryStatusCollector extends StreamingQueryListener { // to catch errors in the async listener events @volatile private var asyncTestWaiter = new Waiter - @volatile var startStatus: StreamingQueryInfo = null - @volatile var terminationStatus: StreamingQueryInfo = null + @volatile var startStatus: StreamingQueryStatus = null + @volatile var terminationStatus: StreamingQueryStatus = null @volatile var terminationException: Option[String] = null - private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryInfo] + private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryStatus] /** Get the info of the last trigger that processed data */ - def lastTriggerStatus: Option[StreamingQueryInfo] = synchronized { + def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { progressStatuses.filter { i => - i.triggerStatus("isActive").toBoolean == false && - i.triggerStatus("isDataAvailable").toBoolean == true + i.triggerStatus.get("isActive").toBoolean == false && + i.triggerStatus.get("isDataAvailable").toBoolean == true }.lastOption } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 384066afa5639..89d4db11a7d3e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -171,7 +171,6 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), AssertOnQuery(_.status.inputRate === 0.0), AssertOnQuery(_.status.processingRate === 0.0), - AssertOnQuery(_.status.outputRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === None), @@ -179,21 +178,18 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), - AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.description.contains("Memory")), AssertOnQuery(_.sinkStatus.offsetDesc === new CompositeOffset(None :: Nil).toString), - AssertOnQuery(_.sinkStatus.outputRate === 0.0), AddData(inputData, 1, 2), CheckAnswer(6, 3), AssertOnQuery(_.status.timestamp <= System.currentTimeMillis), AssertOnQuery(_.status.inputRate >= 0.0), AssertOnQuery(_.status.processingRate >= 0.0), - AssertOnQuery(_.status.outputRate >= 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), @@ -202,12 +198,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), - AssertOnQuery(_.status.sinkStatus.outputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), - AssertOnQuery(_.sinkStatus.outputRate >= 0.0), AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), @@ -220,38 +214,32 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { StopStream, AssertOnQuery(_.status.inputRate === 0.0), AssertOnQuery(_.status.processingRate === 0.0), - AssertOnQuery(_.status.outputRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sinkStatus.outputRate === 0.0), StartStream(), AddData(inputData, 0), ExpectFailure[SparkException], AssertOnQuery(_.status.inputRate === 0.0), AssertOnQuery(_.status.processingRate === 0.0), - AssertOnQuery(_.status.outputRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.status.sinkStatus.outputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), - AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sinkStatus.outputRate === 0.0) + AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) ) } @@ -263,9 +251,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // Trigger input has 10 rows, static input has 2 rows, // therefore after the first trigger, the calculated input rows should be 10 val status = getFirstTriggerStatus(streamingInputDF.join(staticInputDF, "value")) - assert(status.triggerStatus("numRows.input.total") === "10") + assert(status.triggerStatus.get("numRows.input.total") === "10") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "10") + assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "10") } test("input row calculation with trigger DF having multiple leaves") { @@ -276,9 +264,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // After the first trigger, the calculated input rows should be 10 val status = getFirstTriggerStatus(streamingInputDF) - assert(status.triggerStatus("numRows.input.total") === "10") + assert(status.triggerStatus.get("numRows.input.total") === "10") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus("numRows.input.source") === "10") + assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "10") } testQuietly("StreamExecution metadata garbage collection") { @@ -319,10 +307,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { } /** Returns the query status at the end of the first trigger of streaming DF */ - private def getFirstTriggerStatus(streamingDF: DataFrame): StreamingQueryInfo = { + private def getFirstTriggerStatus(streamingDF: DataFrame): StreamingQueryStatus = { // A StreamingQueryListener that gets the query status after the first completed trigger val listener = new StreamingQueryListener { - @volatile var firstStatus: StreamingQueryInfo = null + @volatile var firstStatus: StreamingQueryStatus = null override def onQueryStarted(queryStarted: QueryStarted): Unit = { } override def onQueryProgress(queryProgress: QueryProgress): Unit = { if (firstStatus == null) firstStatus = queryProgress.queryInfo From bad5644f0b3c1036e315a47eeabdf15cee6e00fe Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Oct 2016 18:17:57 -0700 Subject: [PATCH 22/43] Added Mima excludes --- project/MimaExcludes.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index a64db3febbe0a..4553fcf015a21 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -58,8 +58,13 @@ object MimaExcludes { // [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") + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), + ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo") ) } From 8bb554b3d21ed97b032473ce9a969bdabcf74d3c Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 5 Oct 2016 19:17:13 -0700 Subject: [PATCH 23/43] More refactoring, and file num rows test --- .../spark/sql/streaming/StreamingQuery.scala | 9 ++++-- .../streaming/StreamingQueryListener.scala | 10 +++---- .../sql/streaming/FileStreamSourceSuite.scala | 27 +++++++++++++++++ .../StreamingQueryListenerSuite.scala | 30 ++++++++++--------- .../sql/streaming/StreamingQuerySuite.scala | 2 +- 5 files changed, 55 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index 757cd516a34c5..0a85414451981 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -64,7 +64,7 @@ trait StreamingQuery { /** * Returns the current status of the query. - * @since 2.1.0 + * @since 2.0.2 */ def status: StreamingQueryStatus @@ -72,9 +72,14 @@ trait StreamingQuery { * Returns current status of all the sources. * @since 2.0.0 */ + @deprecated("use status.sourceStatuses", "2.0.2") def sourceStatuses: Array[SourceStatus] - /** Returns current status of the sink. */ + /** + * Returns current status of the sink. + * @since 2.0.0 + */ + @deprecated("use status.sinkStatus", "2.0.2") def sinkStatus: SinkStatus /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index c2399f47f7f69..ec86207a9b821 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -84,7 +84,7 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryStarted private[sql](val queryInfo: StreamingQueryStatus) extends Event + class QueryStarted private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: @@ -92,21 +92,19 @@ object StreamingQueryListener { * @since 2.0.0 */ @Experimental - class QueryProgress private[sql](val queryInfo: StreamingQueryStatus) extends Event + class QueryProgress private[sql](val queryStatus: StreamingQueryStatus) extends Event /** * :: Experimental :: * Event representing that termination of a query * - * @param queryInfo Information about the status of the query. + * @param queryStatus Information about the status of the query. * @param exception The exception message of the [[StreamingQuery]] if the query was terminated * with an exception. Otherwise, it will be `None`. - * @param stackTrace The stack trace of the exception if the query was terminated with an - * exception. It will be empty if there was no error. * @since 2.0.0 */ @Experimental class QueryTerminated private[sql]( - val queryInfo: StreamingQueryStatus, + val queryStatus: StreamingQueryStatus, val exception: Option[String]) extends Event } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 3157afe5a56c0..5cb41fd7f836e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.streaming import java.io.File import org.scalatest.PrivateMethodTester +import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.StreamingQueryListenerSuite.QueryStatusCollector import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -980,6 +982,31 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } } + + test("can calculate num rows read in a trigger") { + withTempDirs { case (src, tmp) => + val input = spark.readStream.format("text").load(src.getCanonicalPath).as[String] + val listener = new QueryStatusCollector + spark.streams.addListener(listener) + try { + testStream(input)( + AddTextFileData("100", src, tmp), + CheckAnswer("100"), + AssertOnQuery { query => + eventually(timeout(streamingTimeout)) { + assert(listener.lastTriggerStatus.nonEmpty) + } + val status = listener.lastTriggerStatus.get + assert(status.triggerStatus.get("numRows.input.total") === "1") + assert(status.sourceStatuses(0).processingRate > 0.0) + true + } + ) + } finally { + spark.streams.removeListener(listener) + } + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 3a78ae6657c3f..daf3c9772d009 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -26,18 +26,20 @@ import org.scalatest.PrivateMethodTester._ import org.scalatest.concurrent.AsyncAssertions.Waiter import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout +import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ -import org.apache.spark.util.{JsonProtocol, ManualClock, Utils} +import org.apache.spark.sql.streaming.StreamingQueryListener._ +import org.apache.spark.util.{JsonProtocol, ManualClock} class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - import StreamingQueryListener._ + import StreamingQueryListenerSuite._ // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) @@ -197,7 +199,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] - assertStreamingQueryInfoEquals(queryStarted.queryInfo, newQueryStarted.queryInfo) + assertStreamingQueryInfoEquals(queryStarted.queryStatus, newQueryStarted.queryStatus) } test("QueryProgress serialization") { @@ -205,7 +207,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] - assertStreamingQueryInfoEquals(queryProcess.queryInfo, newQueryProcess.queryInfo) + assertStreamingQueryInfoEquals(queryProcess.queryStatus, newQueryProcess.queryStatus) } test("QueryTerminated serialization") { @@ -217,7 +219,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { JsonProtocol.sparkEventToJson(queryQueryTerminated) val newQueryTerminated = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryTerminated] - assertStreamingQueryInfoEquals(queryQueryTerminated.queryInfo, newQueryTerminated.queryInfo) + assertStreamingQueryInfoEquals(queryQueryTerminated.queryStatus, newQueryTerminated.queryStatus) assert(queryQueryTerminated.exception === newQueryTerminated.exception) } @@ -269,6 +271,11 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString), Map("a" -> "b").asJava) } +} + +object StreamingQueryListenerSuite { + // Singleton reference to clock that does not get serialized in task closures + @volatile var clock: ManualClock = null class QueryStatusCollector extends StreamingQueryListener { // to catch errors in the async listener events @@ -296,35 +303,30 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } def checkAsyncErrors(): Unit = { - asyncTestWaiter.await(timeout(streamingTimeout)) + asyncTestWaiter.await(timeout(10 seconds)) } override def onQueryStarted(queryStarted: QueryStarted): Unit = { asyncTestWaiter { - startStatus = queryStarted.queryInfo + startStatus = queryStarted.queryStatus } } override def onQueryProgress(queryProgress: QueryProgress): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryProgress called before onQueryStarted") - synchronized { progressStatuses += queryProgress.queryInfo } + synchronized { progressStatuses += queryProgress.queryStatus } } } override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { asyncTestWaiter { assert(startStatus != null, "onQueryTerminated called before onQueryStarted") - terminationStatus = queryTerminated.queryInfo + terminationStatus = queryTerminated.queryStatus terminationException = queryTerminated.exception } asyncTestWaiter.dismiss() } } } - -object StreamingQueryListenerSuite { - // Singleton reference to clock that does not get serialized in task closures - @volatile var clock: ManualClock = null -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 89d4db11a7d3e..a76efcd5c56de 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -313,7 +313,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { @volatile var firstStatus: StreamingQueryStatus = null override def onQueryStarted(queryStarted: QueryStarted): Unit = { } override def onQueryProgress(queryProgress: QueryProgress): Unit = { - if (firstStatus == null) firstStatus = queryProgress.queryInfo + if (firstStatus == null) firstStatus = queryProgress.queryStatus } override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { } } From 29185254d325834c40bd63a543317950b2794b30 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 6 Oct 2016 17:32:36 -0700 Subject: [PATCH 24/43] Added tests for sources --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 40 +++++++++++++++++++ .../sql/execution/LocalTableScanExec.scala | 5 ++- .../execution/streaming/StreamExecution.scala | 8 +++- .../execution/metric/SQLMetricsSuite.scala | 17 ++++++++ ...csSuite.scala => StreamMetricsSuite.scala} | 3 +- .../streaming/TextSocketStreamSuite.scala | 24 +++++++++++ .../sql/streaming/FileStreamSourceSuite.scala | 2 +- 7 files changed, 94 insertions(+), 5 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/{StreamingMetricsSuite.scala => StreamMetricsSuite.scala} (99%) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 6c03070398fca..ae86ae7fde6b1 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -23,9 +23,11 @@ import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamingQueryListenerSuite.QueryStatusCollector import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSQLContext @@ -265,6 +267,44 @@ class KafkaSourceSuite extends KafkaSourceTest { testUnsupportedConfig("kafka.auto.offset.reset", "latest") } + test("input row metrics") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 5) + testUtils.sendMessages(topic, Array("-1")) + require(testUtils.getLatestOffsets(Set(topic)).size === 5) + + val kafka = spark + .readStream + .format("kafka") + .option("subscribe", topic) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val mapped = kafka.map(kv => kv._2.toInt + 1) + val listener = new QueryStatusCollector + spark.streams.addListener(listener) + try { + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnQuery { query => + eventually(timeout(streamingTimeout)) { + assert(listener.lastTriggerStatus.nonEmpty) + } + val status = listener.lastTriggerStatus.get + assert(status.triggerStatus.get("numRows.input.total").toInt > 0) + assert(status.sourceStatuses(0).processingRate > 0.0) + true + } + ) + } finally { + spark.streams.removeListener(listener) + } + } + private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" private def testFromLatestOffsets(topic: String, options: (String, String)*): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index 6598fa381aa3d..e366b9af35c62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -64,10 +64,13 @@ case class LocalTableScanExec( } override def executeCollect(): Array[InternalRow] = { + longMetric("numOutputRows").add(unsafeRows.size) unsafeRows } override def executeTake(limit: Int): Array[InternalRow] = { - unsafeRows.take(limit) + val taken = unsafeRows.take(limit) + longMetric("numOutputRows").add(taken.size) + taken } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index ef02a4ca3c5d5..9ea4547c57497 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -225,6 +225,7 @@ class StreamExecution( false } } + // Update metrics and notify others streamMetrics.reportTriggerFinished() updateStatus() postEvent(new QueryProgress(currentStatus)) @@ -241,9 +242,14 @@ class StreamExecution( logError(s"Query $name terminated with error", e) } finally { state = TERMINATED - sparkSession.streams.notifyQueryTermination(StreamExecution.this) + + // Update metrics and status streamMetrics.stop() sparkSession.sparkContext.env.metricsSystem.removeSource(streamMetrics) + updateStatus() + + // Notify others + sparkSession.streams.notifyQueryTermination(StreamExecution.this) postEvent(new QueryTerminated(status, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index bba40c6510cfb..229d8814e0143 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.metric import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.SparkPlanGraph import org.apache.spark.sql.functions._ @@ -85,6 +86,22 @@ class SQLMetricsSuite extends SparkFunSuite with SharedSQLContext { } } + test("LocalTableScanExec computes metrics in collect and take") { + val df1 = spark.createDataset(Seq(1, 2, 3)) + val logical = df1.queryExecution.logical + require(logical.isInstanceOf[LocalRelation]) + df1.collect() + val metrics1 = df1.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics1.contains("numOutputRows")) + assert(metrics1("numOutputRows").value === 3) + + val df2 = spark.createDataset(Seq(1, 2, 3)).limit(2) + df2.collect() + val metrics2 = df2.queryExecution.executedPlan.collectLeaves().head.metrics + assert(metrics2.contains("numOutputRows")) + assert(metrics2("numOutputRows").value === 2) + } + test("Filter metrics") { // Assume the execution plan is // PhysicalRDD(nodeId = 1) -> Filter(nodeId = 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala index c629534db9338..c00043689186a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamingMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.ManualClock -class StreamingMetricsSuite extends SparkFunSuite { +class StreamMetricsSuite extends SparkFunSuite { // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) @@ -168,7 +168,6 @@ class StreamingMetricsSuite extends SparkFunSuite { "inputRate-source0", "processingRate-total", "processingRate-source0", - "outputRate", "latency")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala index 6b0ba7acb4804..5174a0415304c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/TextSocketStreamSuite.scala @@ -156,6 +156,30 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } } + test("input row metrics") { + serverThread = new ServerThread() + serverThread.start() + + val provider = new TextSocketSourceProvider + val parameters = Map("host" -> "localhost", "port" -> serverThread.port.toString) + source = provider.createSource(sqlContext, "", None, "", parameters) + + failAfter(streamingTimeout) { + serverThread.enqueue("hello") + while (source.getOffset.isEmpty) { + Thread.sleep(10) + } + val batch = source.getBatch(None, source.getOffset.get).as[String] + batch.collect() + val numRowsMetric = + batch.queryExecution.executedPlan.collectLeaves().head.metrics.get("numOutputRows") + assert(numRowsMetric.nonEmpty) + assert(numRowsMetric.get.value === 1) + source.stop() + source = null + } + } + private class ServerThread extends Thread with Logging { private val serverSocket = new ServerSocket(0) private val messageQueue = new LinkedBlockingQueue[String]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 5cb41fd7f836e..b24fbb4d10a58 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -983,7 +983,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } - test("can calculate num rows read in a trigger") { + test("input row metrics") { withTempDirs { case (src, tmp) => val input = spark.readStream.format("text").load(src.getCanonicalPath).as[String] val listener = new QueryStatusCollector From 10d1c243a71d464ada33db269a30ad0e4dff3ced Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 6 Oct 2016 18:15:38 -0700 Subject: [PATCH 25/43] Fixed mima --- project/MimaExcludes.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b3fda07eaeafc..7484c43e6456a 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -62,11 +62,11 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SourceStatus.this"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.this"), - ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo") + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryProgress.queryInfo"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.queryInfo") ) } From 8537783abc495156d3f356e378d260c9222f2c46 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 6 Oct 2016 20:26:12 -0700 Subject: [PATCH 26/43] Pretty strings from statuses --- .../execution/streaming/StreamExecution.scala | 2 +- .../execution/streaming/StreamMetrics.scala | 7 ++- .../sql/execution/streaming/memory.scala | 2 + .../spark/sql/streaming/SinkStatus.scala | 19 +++++-- .../spark/sql/streaming/SourceStatus.scala | 35 ++++++++++-- .../sql/streaming/StreamingQueryStatus.scala | 53 +++++++++++++++++-- .../StreamingQueryListenerSuite.scala | 12 ++--- .../sql/streaming/StreamingQuerySuite.scala | 2 - 8 files changed, 106 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 9ea4547c57497..f27b8198fdf56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -659,7 +659,7 @@ class StreamExecution( val sourceStatuses = sources.map { s => SourceStatus( s.toString, - localAvailableOffsets.get(s).map(_.toString), + localAvailableOffsets.get(s).map(_.toString).getOrElse("-"), streamMetrics.currentSourceInputRate(s), streamMetrics.currentSourceProcessingRate(s), streamMetrics.currentSourceTriggerStatus(s)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 7080dbdf6639a..307f120a48db2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution.streaming import java.{util => ju} import scala.collection.mutable -import scala.collection.JavaConverters._ import com.codahale.metrics.{Gauge, MetricRegistry} @@ -177,10 +176,10 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def currentLatency(): Option[Double] = synchronized { latency } - def currentTriggerStatus(): ju.Map[String, String] = synchronized { triggerStatus.toMap.asJava } + def currentTriggerStatus(): Map[String, String] = synchronized { triggerStatus.toMap } - def currentSourceTriggerStatus(source: Source): ju.Map[String, String] = synchronized { - sourceTriggerStatus(source).toMap.asJava + def currentSourceTriggerStatus(source: Source): Map[String, String] = synchronized { + sourceTriggerStatus(source).toMap } // =========== Other methods =========== diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index 43d893930b5a6..788fcd0361bee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -170,6 +170,8 @@ class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink wi logDebug(s"Skipping already committed batch: $batchId") } } + + override def toString(): String = "MemorySink" } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index ac65ac9a971fb..377ff20af8705 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Sink /** * :: Experimental :: @@ -29,9 +28,21 @@ import org.apache.spark.sql.execution.streaming.Sink * @since 2.0.0 */ @Experimental -case class SinkStatus private( +class SinkStatus private( val description: String, - val offsetDesc: String) + val offsetDesc: String) { + + override def toString: String = + "SinkStatus:\n" + prettyStrings.map(" " + _).mkString("\n") + + private[sql] def prettyStrings: Array[String] = { + s"""Description: $description + |Committed offsets: $offsetDesc + |""".stripMargin.split("\n") + } +} /** Companion object, primarily for creating SinkStatus instances internally */ -private[sql] object SinkStatus +private[sql] object SinkStatus { + def apply(desc: String, offsetDesc: String): SinkStatus = new SinkStatus(desc, offsetDesc) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 549c109c730b9..44f76cf9ac581 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.streaming import java.{util => ju} +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental -import org.apache.spark.sql.execution.streaming.Source /** * :: Experimental :: @@ -35,12 +36,36 @@ import org.apache.spark.sql.execution.streaming.Source * @since 2.0.0 */ @Experimental -case class SourceStatus private( +class SourceStatus private( val description: String, - val offsetDesc: Option[String], + val offsetDesc: String, val inputRate: Double, val processingRate: Double, - val triggerStatus: ju.Map[String, String]) + val triggerStatus: ju.Map[String, String]) { + + override def toString: String = + "SourceStatus:\n" + prettyStrings.map(" " + _).mkString("\n") + + private[sql] def prettyStrings: Array[String] = { + val triggerStatusStrings = triggerStatus.asScala.map { case (k, v) => s" $k: $v" } + s"""Description: $description + |Available offset: $offsetDesc + |Input rate: $inputRate rows/sec + |Processing rate: $processingRate rows/sec + |Trigger status: + |""".stripMargin.split("\n") ++ triggerStatusStrings + + } +} /** Companion object, primarily for creating SourceStatus instances internally */ -private[sql] object SourceStatus +private[sql] object SourceStatus { + def apply( + desc: String, + offsetDesc: String, + inputRate: Double, + processingRate: Double, + triggerStatus: Map[String, String]): SourceStatus = { + new SourceStatus(desc, offsetDesc, inputRate, processingRate, triggerStatus.asJava) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 1df700af35378..154db6589f5ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.streaming import java.{util => ju} +import scala.collection.JavaConverters._ + import org.apache.spark.annotation.Experimental /** @@ -40,7 +42,7 @@ import org.apache.spark.annotation.Experimental * @since 2.0.0 */ @Experimental -case class StreamingQueryStatus private( +class StreamingQueryStatus private( val name: String, val id: Long, val timestamp: Long, @@ -49,8 +51,51 @@ case class StreamingQueryStatus private( val latency: Option[Double], val sourceStatuses: Array[SourceStatus], val sinkStatus: SinkStatus, - val triggerStatus: ju.Map[String, String] -) + val triggerStatus: ju.Map[String, String]) { + + override def toString: String = { + val indent = " " + + val sourceStatusStrings = + sourceStatuses.zipWithIndex.flatMap { case (s, i) => + Seq(s"Source $i:") ++ s.prettyStrings.map(indent + _) + }.map(s"|$indent" + _).mkString("\n") + val sinkStatusString = + sinkStatus.prettyStrings.map(s"|$indent" + _).mkString("\n") + val triggerStatusString = + triggerStatus.asScala.map { case (k, v) => s"|$indent$k: $v" }.mkString("\n") + + val allString = s""" + |Name: $name + |Id: $id + |Timestamp: $timestamp + |Input rate: $inputRate rows/sec + |Processing rate $processingRate rows/sec + |Latency: ${latency.getOrElse("-")} ms + |Trigger status: + $triggerStatusString + |Source statuses: + $sourceStatusStrings + |Sink status: + $sinkStatusString""".stripMargin.split("\n").map(indent + _).mkString("\n") + + s"StreamingQueryStatus:$allString" + } +} /** Companion object, primarily for creating StreamingQueryInfo instances internally */ -private[sql] object StreamingQueryStatus +private[sql] object StreamingQueryStatus { + def apply( + name: String, + id: Long, + timestamp: Long, + inputRate: Double, + processingRate: Double, + latency: Option[Double], + sourceStatuses: Array[SourceStatus], + sinkStatus: SinkStatus, + triggerStatus: Map[String, String]): StreamingQueryStatus = { + new StreamingQueryStatus(name, id, timestamp, inputRate, processingRate, + latency, sourceStatuses, sinkStatus, triggerStatus.asJava) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index daf3c9772d009..b9c614bc8d1fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -195,7 +195,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStarted = new StreamingQueryListener.QueryStarted(testQueryInfo) + val queryStarted = new StreamingQueryListener.QueryStarted(testQueryStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] @@ -203,7 +203,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryProgress serialization") { - val queryProcess = new StreamingQueryListener.QueryProgress(testQueryInfo) + val queryProcess = new StreamingQueryListener.QueryProgress(testQueryStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] @@ -213,7 +213,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { test("QueryTerminated serialization") { val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( - testQueryInfo, + testQueryStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) @@ -263,13 +263,13 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } - private val testQueryInfo: StreamingQueryStatus = { + private val testQueryStatus: StreamingQueryStatus = { StreamingQueryStatus( "name", 1, 123, 1.0, 2.0, Some(345), Array( - SourceStatus("source1", Some(LongOffset(0).toString), 0.0, 0.0, Map("a" -> "b").asJava)), + SourceStatus("source1", LongOffset(0).toString, 0.0, 0.0, Map("a" -> "b"))), SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString), - Map("a" -> "b").asJava) + Map("a" -> "b")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index a76efcd5c56de..586e9947c1566 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -88,8 +88,6 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { } - - test("names unique across active queries, ids unique across all started queries") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map { 6 / _} From 1655c967a661f000586011822453f86acce0e06d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 7 Oct 2016 01:31:18 -0700 Subject: [PATCH 27/43] Fixed mima and improved status desc --- project/MimaExcludes.scala | 3 +- .../spark/sql/streaming/SinkStatus.scala | 9 ++-- .../spark/sql/streaming/SourceStatus.scala | 12 +++--- .../sql/streaming/StreamingQueryStatus.scala | 41 ++++++++++--------- 4 files changed, 35 insertions(+), 30 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7484c43e6456a..07669509e2995 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -58,9 +58,10 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.catalog.Catalog.functionExists"), // [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[IncompatibleResultTypeProblem]("org.apache.spark.sql.streaming.SourceStatus.offsetDesc"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQuery.status"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.SinkStatus.this"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.streaming.StreamingQueryInfo"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryStarted.queryInfo"), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 377ff20af8705..40cb4aa42dce8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: @@ -33,12 +34,12 @@ class SinkStatus private( val offsetDesc: String) { override def toString: String = - "SinkStatus:\n" + prettyStrings.map(" " + _).mkString("\n") + "SinkStatus:" + indent(prettyString) - private[sql] def prettyStrings: Array[String] = { - s"""Description: $description + private[sql] def prettyString: String = { + s"""$description |Committed offsets: $offsetDesc - |""".stripMargin.split("\n") + |""".stripMargin } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 44f76cf9ac581..ef30cf9553a94 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -22,6 +22,7 @@ import java.{util => ju} import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.streaming.StreamingQueryStatus.indent /** * :: Experimental :: @@ -44,16 +45,17 @@ class SourceStatus private( val triggerStatus: ju.Map[String, String]) { override def toString: String = - "SourceStatus:\n" + prettyStrings.map(" " + _).mkString("\n") + "SourceStatus:" + indent(prettyString) - private[sql] def prettyStrings: Array[String] = { - val triggerStatusStrings = triggerStatus.asScala.map { case (k, v) => s" $k: $v" } - s"""Description: $description + private[sql] def prettyString: String = { + val triggerStatusStrings = + triggerStatus.asScala.map { case (k, v) => s"$k: $v" } + s"""$description |Available offset: $offsetDesc |Input rate: $inputRate rows/sec |Processing rate: $processingRate rows/sec |Trigger status: - |""".stripMargin.split("\n") ++ triggerStatusStrings + |""".stripMargin + indent(triggerStatusStrings) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 154db6589f5ad..a5fe8bc8c53d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -53,33 +53,31 @@ class StreamingQueryStatus private( val sinkStatus: SinkStatus, val triggerStatus: ju.Map[String, String]) { - override def toString: String = { - val indent = " " + import StreamingQueryStatus._ - val sourceStatusStrings = - sourceStatuses.zipWithIndex.flatMap { case (s, i) => - Seq(s"Source $i:") ++ s.prettyStrings.map(indent + _) - }.map(s"|$indent" + _).mkString("\n") - val sinkStatusString = - sinkStatus.prettyStrings.map(s"|$indent" + _).mkString("\n") - val triggerStatusString = - triggerStatus.asScala.map { case (k, v) => s"|$indent$k: $v" }.mkString("\n") + override def toString: String = { + val sourceStatusLines = sourceStatuses.zipWithIndex.map { case (s, i) => + s"Source ${i + 1}:" + indent(s.prettyString) + } + val sinkStatusLines = sinkStatus.prettyString + val triggerStatusLines = triggerStatus.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted + val numSources = sourceStatuses.length + val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } - val allString = s""" - |Name: $name - |Id: $id - |Timestamp: $timestamp + val allLines = s""" + |Query name: $name + |Query id: $id + |Status timestamp: $timestamp |Input rate: $inputRate rows/sec |Processing rate $processingRate rows/sec |Latency: ${latency.getOrElse("-")} ms |Trigger status: - $triggerStatusString - |Source statuses: - $sourceStatusStrings - |Sink status: - $sinkStatusString""".stripMargin.split("\n").map(indent + _).mkString("\n") + |${indent(triggerStatusLines)} + |Source statuses [$numSourcesString]: + |${indent(sourceStatusLines)} + |Sink status: ${indent(sinkStatusLines)}""".stripMargin - s"StreamingQueryStatus:$allString" + s"StreamingQueryStatus:${indent(allLines)}" } } @@ -98,4 +96,7 @@ private[sql] object StreamingQueryStatus { new StreamingQueryStatus(name, id, timestamp, inputRate, processingRate, latency, sourceStatuses, sinkStatus, triggerStatus.asJava) } + + def indent(strings: Iterable[String]): String = strings.map(indent).mkString("\n") + def indent(string: String): String = string.split("\n").map(" " + _).mkString("\n") } From 3984d5420a72642cff3f3e41acbbd49f47f0691e Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 7 Oct 2016 03:40:48 -0700 Subject: [PATCH 28/43] Added python APIs for StreamingQueryStatus, SourceStatus, SinkStatus --- python/pyspark/sql/streaming.py | 261 ++++++++++++++++++ .../spark/sql/streaming/SinkStatus.scala | 5 +- .../spark/sql/streaming/SourceStatus.scala | 10 +- .../sql/streaming/StreamingQueryStatus.scala | 25 +- .../StreamingQueryListenerSuite.scala | 2 +- 5 files changed, 288 insertions(+), 15 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 4e438fd5bee22..d427f68fbc24a 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -189,6 +189,264 @@ def resetTerminated(self): self._jsqm.resetTerminated() +class StreamingQueryStatus(object): + """A class used to report information about the progress of a StreamingQuery. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jsqs): + self._jsqs = jsqs + + @since(2.1) + def __str__(self): + """ + Pretty string of this query status. + + >>> print(sqs) + StreamingQueryStatus: + Query name: query + Query id: 1 + Status timestamp: 123 + Input rate: 1.0 rows/sec + Processing rate 2.0 rows/sec + Latency: 345.0 ms + Trigger status: + key: value + Source statuses [1 source]: + Source 1: MySource1 + Available offset: #0 + Input rate: 4.0 rows/sec + Processing rate: 5.0 rows/sec + Trigger status: + key: value + Sink status: MySink + Committed offsets: [#1, -] + """ + return self._jsqs.toString() + + @since(2.1) + def name(self): + """ + Name of the query. This name is unique across all active queries. + + >>> sqs.name() + u'query' + """ + return self._jsqs.name() + + @since(2.1) + def id(self): + """ + Id of the query. This id is unique across all queries that have been started in + the current process. + + >>> sqs.id() + 1L + """ + return self._jsqs.id() + + @since(2.1) + def timestamp(self): + """ + Timestamp (ms) of when this query was generated. + + >>> sqs.timestamp() + 123L + """ + return self._jsqs.timestamp() + + @since(2.1) + def inputRate(self): + """ + Current rate (rows/sec) at which data is being generated by all the sources. + + >>> sqs.inputRate() + 1.0 + """ + return self._jsqs.inputRate() + + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from all the sources. + + >>> sqs.processingRate() + 2.0 + """ + return self._jsqs.processingRate() + + @since(2.1) + def latency(self): + """ + Current average latency between the data being available in source and the sink + writing the corresponding output. + + >>> sqs.latency() + 345.0 + """ + if (self._jsqs.latency().nonEmpty()): + return self._jsqs.latency().get() + else: + return None + + @since(2.1) + def sourceStatuses(self): + """ + Current statuses of the sources. + + >>> len(sqs.sourceStatuses()) + 1 + >>> sqs.sourceStatuses()[0].description() + u'MySource1' + """ + return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()] + + @since(2.1) + def sinkStatus(self): + """ + Current status of the sink. + + >>> sqs.sinkStatus().description() + u'MySink' + """ + return SinkStatus(self._jsqs.sinkStatus()) + + @since(2.1) + def triggerStatus(self): + """ + Low-level detailed status of the last completed/currently active trigger. + + >>> sqs.triggerStatus() + {u'key': u'value'} + """ + return self._jsqs.triggerStatus() + + +class SourceStatus(object): + """ + Status and metrics of a streaming Source. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + @since(2.1) + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sourceStatuses()[0]) + SourceStatus: MySource1 + Available offset: #0 + Input rate: 4.0 rows/sec + Processing rate: 5.0 rows/sec + Trigger status: + key: value + """ + return self._jss.toString() + + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sourceStatuses()[0].description() + u'MySource1' + """ + return self._jss.description() + + @since(2.1) + def offsetDesc(self): + """ + Description of the current offset if known. + + >>> sqs.sourceStatuses()[0].offsetDesc() + u'#0' + """ + return self._jss.offsetDesc() + + @since(2.1) + def inputRate(self): + """ + Current rate (rows/sec) at which data is being generated by the source. + + >>> sqs.sourceStatuses()[0].inputRate() + 4.0 + """ + return self._jss.inputRate() + + @since(2.1) + def processingRate(self): + """ + Current rate (rows/sec) at which the query is processing data from the source. + + >>> sqs.sourceStatuses()[0].processingRate() + 5.0 + """ + return self._jss.processingRate() + + @since(2.1) + def triggerStatus(self): + """ + Low-level detailed status of the last completed/currently active trigger. + + >>> sqs.sourceStatuses()[0].triggerStatus() + {u'key': u'value'} + """ + return self._jss.triggerStatus() + + +class SinkStatus(object): + """ + Status and metrics of a streaming Sink. + + .. note:: Experimental + + .. versionadded:: 2.1 + """ + + def __init__(self, jss): + self._jss = jss + + @since(2.1) + def __str__(self): + """ + Pretty string of this source status. + + >>> print(sqs.sinkStatus()) + SinkStatus: MySink + Committed offsets: [#1, -] + """ + return self._jss.toString() + + @since(2.1) + def description(self): + """ + Description of the source corresponding to this status. + + >>> sqs.sinkStatus().description() + u'MySink' + """ + return self._jss.description() + + @since(2.1) + def offsetDesc(self): + """ + Description of the current offsets up to which data has been written by the sink. + + >>> sqs.sinkStatus().offsetDesc() + u'[#1, -]' + """ + return self._jss.offsetDesc() + + class Trigger(object): """Used to indicate how often results should be produced by a :class:`StreamingQuery`. @@ -753,11 +1011,14 @@ def _test(): globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') + globs['sqs'] = StreamingQueryStatus(\ + spark.sparkContext._jvm.org.apache.spark.sql.streaming.StreamingQueryStatus.testStatus()) (failure_count, test_count) = doctest.testmod( pyspark.sql.streaming, globs=globs, optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF) globs['spark'].stop() + if failure_count: exit(-1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala index 40cb4aa42dce8..c9911665f7d72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SinkStatus.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.streaming.StreamingQueryStatus.indent * :: Experimental :: * Status and metrics of a streaming sink. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current offset up to which data has been written by the sink + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offsets up to which data has been written + * by the sink. * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index ef30cf9553a94..68b08bf4fbdf3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -28,12 +28,12 @@ import org.apache.spark.sql.streaming.StreamingQueryStatus.indent * :: Experimental :: * Status and metrics of a streaming Source. * - * @param description Description of the source corresponding to this status - * @param offsetDesc Description of the current offset if known - * @param inputRate Current rate (rows/sec) at which data is being generated by the source + * @param description Description of the source corresponding to this status. + * @param offsetDesc Description of the current offset if known. + * @param inputRate Current rate (rows/sec) at which data is being generated by the source. * @param processingRate Current rate (rows/sec) at which the query is processing data from - * the source - * @param triggerStatus Low-level detailed status of the last completed/currently active trigger + * the source. + * @param triggerStatus Low-level detailed status of the last completed/currently active trigger. * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index a5fe8bc8c53d6..ea4e7f90881ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -22,23 +22,24 @@ import java.{util => ju} import scala.collection.JavaConverters._ import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} /** * :: Experimental :: * A class used to report information about the progress of a [[StreamingQuery]]. * - * @param name The query name. This name is unique across all active queries. - * @param id The query id. This id is unique across + * @param name Name of the query. This name is unique across all active queries. + * @param id Id of the query. This id is unique across * all queries that have been started in the current process. - * @param timestamp Timestamp (ms) of when this query was generated - * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources + * @param timestamp Timestamp (ms) of when this query was generated. + * @param inputRate Current rate (rows/sec) at which data is being generated by all the sources. * @param processingRate Current rate (rows/sec) at which the query is processing data from - * all the sources + * all the sources. * @param latency Current average latency between the data being available in source and the sink - * writing the corresponding output + * writing the corresponding output. * @param sourceStatuses Current statuses of the sources. * @param sinkStatus Current status of the sink. - * @param triggerStatus Low-level detailed status of the last completed/currently active trigger + * @param triggerStatus Low-level detailed status of the last completed/currently active trigger. * @since 2.0.0 */ @Experimental @@ -99,4 +100,14 @@ private[sql] object StreamingQueryStatus { def indent(strings: Iterable[String]): String = strings.map(indent).mkString("\n") def indent(string: String): String = string.split("\n").map(" " + _).mkString("\n") + + /** Create an instance of status for python testing */ + def testStatus(): StreamingQueryStatus = { + StreamingQueryStatus( + "query", 1, 123, 1.0, 2.0, Some(345), + Array( + SourceStatus("MySource1", LongOffset(0).toString, 4.0, 5.0, Map("key" -> "value"))), + SinkStatus("MySink", CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), + Map("key" -> "value")) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index b9c614bc8d1fe..b149c77003a00 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -268,7 +268,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { "name", 1, 123, 1.0, 2.0, Some(345), Array( SourceStatus("source1", LongOffset(0).toString, 0.0, 0.0, Map("a" -> "b"))), - SinkStatus("sink", CompositeOffset(None :: None :: Nil).toString), + SinkStatus("sink", CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), Map("a" -> "b")) } } From 5e0aa570e119b7ec51479646c511354af35b1db0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 7 Oct 2016 03:46:43 -0700 Subject: [PATCH 29/43] Fixed StreamingQuerySuite --- python/pyspark/sql/streaming.py | 2 +- .../sql/streaming/StreamingQuerySuite.scala | 20 +++++++++---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index d427f68fbc24a..583e9a15e0dd4 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -1011,7 +1011,7 @@ def _test(): globs['sdf_schema'] = StructType([StructField("data", StringType(), False)]) globs['df'] = \ globs['spark'].readStream.format('text').load('python/test_support/sql/streaming') - globs['sqs'] = StreamingQueryStatus(\ + globs['sqs'] = StreamingQueryStatus( spark.sparkContext._jvm.org.apache.spark.sql.streaming.StreamingQueryStatus.testStatus()) (failure_count, test_count) = doctest.testmod( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 586e9947c1566..b54038e81f041 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -171,13 +171,13 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.processingRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === "-"), AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset(None :: Nil).toString), AssertOnQuery(_.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === None), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === "-"), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.description.contains("Memory")), @@ -190,35 +190,35 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.processingRate >= 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), AssertOnQuery(_.status.sourceStatuses(0).description.contains("Memory")), - AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(0).toString), AssertOnQuery(_.status.sourceStatuses(0).inputRate >= 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.status.sinkStatus.description.contains("Memory")), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(0).toString)), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(0).toString), AssertOnQuery(_.sourceStatuses(0).inputRate >= 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate >= 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(0)).toString), AddData(inputData, 1, 2), CheckAnswer(6, 3, 6, 3), - AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), StopStream, AssertOnQuery(_.status.inputRate === 0.0), AssertOnQuery(_.status.processingRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), - AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(1).toString)), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(1).toString), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), @@ -229,12 +229,12 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.status.inputRate === 0.0), AssertOnQuery(_.status.processingRate === 0.0), AssertOnQuery(_.status.sourceStatuses.length === 1), - AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.status.sourceStatuses(0).offsetDesc === LongOffset(2).toString), AssertOnQuery(_.status.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.status.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.status.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), - AssertOnQuery(_.sourceStatuses(0).offsetDesc === Some(LongOffset(2).toString)), + AssertOnQuery(_.sourceStatuses(0).offsetDesc === LongOffset(2).toString), AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString) From b747b23f303f1d9913482699b6141f94b5a6e8e7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 10 Oct 2016 17:11:09 -0700 Subject: [PATCH 30/43] Fixed python 3.4 tests --- python/pyspark/sql/streaming.py | 66 +++++++++++++++++++++------------ 1 file changed, 42 insertions(+), 24 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 583e9a15e0dd4..0d0d2650deab3 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -200,7 +200,6 @@ class StreamingQueryStatus(object): def __init__(self, jsqs): self._jsqs = jsqs - @since(2.1) def __str__(self): """ Pretty string of this query status. @@ -227,64 +226,71 @@ def __str__(self): """ return self._jsqs.toString() + @property + @ignore_unicode_prefix @since(2.1) def name(self): """ Name of the query. This name is unique across all active queries. - >>> sqs.name() + >>> sqs.name u'query' """ return self._jsqs.name() + @property @since(2.1) def id(self): """ Id of the query. This id is unique across all queries that have been started in the current process. - >>> sqs.id() - 1L + >>> int(sqs.id) + 1 """ return self._jsqs.id() + @property @since(2.1) def timestamp(self): """ Timestamp (ms) of when this query was generated. - >>> sqs.timestamp() - 123L + >>> int(sqs.timestamp) + 123 """ return self._jsqs.timestamp() + @property @since(2.1) def inputRate(self): """ Current rate (rows/sec) at which data is being generated by all the sources. - >>> sqs.inputRate() + >>> sqs.inputRate 1.0 """ return self._jsqs.inputRate() + @property @since(2.1) def processingRate(self): """ Current rate (rows/sec) at which the query is processing data from all the sources. - >>> sqs.processingRate() + >>> sqs.processingRate 2.0 """ return self._jsqs.processingRate() + @property @since(2.1) def latency(self): """ Current average latency between the data being available in source and the sink writing the corresponding output. - >>> sqs.latency() + >>> sqs.latency 345.0 """ if (self._jsqs.latency().nonEmpty()): @@ -292,34 +298,37 @@ def latency(self): else: return None + @property @since(2.1) def sourceStatuses(self): """ Current statuses of the sources. - >>> len(sqs.sourceStatuses()) + >>> len(sqs.sourceStatuses) 1 - >>> sqs.sourceStatuses()[0].description() + >>> sqs.sourceStatuses[0].description u'MySource1' """ return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()] + @property @since(2.1) def sinkStatus(self): """ Current status of the sink. - >>> sqs.sinkStatus().description() + >>> sqs.sinkStatus.description u'MySink' """ return SinkStatus(self._jsqs.sinkStatus()) + @property @since(2.1) def triggerStatus(self): """ Low-level detailed status of the last completed/currently active trigger. - >>> sqs.triggerStatus() + >>> sqs.triggerStatus {u'key': u'value'} """ return self._jsqs.triggerStatus() @@ -337,12 +346,11 @@ class SourceStatus(object): def __init__(self, jss): self._jss = jss - @since(2.1) def __str__(self): """ Pretty string of this source status. - >>> print(sqs.sourceStatuses()[0]) + >>> print(sqs.sourceStatuses[0]) SourceStatus: MySource1 Available offset: #0 Input rate: 4.0 rows/sec @@ -352,52 +360,59 @@ def __str__(self): """ return self._jss.toString() + @property + @ignore_unicode_prefix @since(2.1) def description(self): """ Description of the source corresponding to this status. - >>> sqs.sourceStatuses()[0].description() + >>> sqs.sourceStatuses[0].description u'MySource1' """ return self._jss.description() + @property + @ignore_unicode_prefix @since(2.1) def offsetDesc(self): """ Description of the current offset if known. - >>> sqs.sourceStatuses()[0].offsetDesc() + >>> sqs.sourceStatuses[0].offsetDesc u'#0' """ return self._jss.offsetDesc() + @property @since(2.1) def inputRate(self): """ Current rate (rows/sec) at which data is being generated by the source. - >>> sqs.sourceStatuses()[0].inputRate() + >>> sqs.sourceStatuses[0].inputRate 4.0 """ return self._jss.inputRate() + @property @since(2.1) def processingRate(self): """ Current rate (rows/sec) at which the query is processing data from the source. - >>> sqs.sourceStatuses()[0].processingRate() + >>> sqs.sourceStatuses[0].processingRate 5.0 """ return self._jss.processingRate() + @property @since(2.1) def triggerStatus(self): """ Low-level detailed status of the last completed/currently active trigger. - >>> sqs.sourceStatuses()[0].triggerStatus() + >>> sqs.sourceStatuses[0].triggerStatus {u'key': u'value'} """ return self._jss.triggerStatus() @@ -415,33 +430,36 @@ class SinkStatus(object): def __init__(self, jss): self._jss = jss - @since(2.1) def __str__(self): """ Pretty string of this source status. - >>> print(sqs.sinkStatus()) + >>> print(sqs.sinkStatus) SinkStatus: MySink Committed offsets: [#1, -] """ return self._jss.toString() + @property + @ignore_unicode_prefix @since(2.1) def description(self): """ Description of the source corresponding to this status. - >>> sqs.sinkStatus().description() + >>> sqs.sinkStatus.description u'MySink' """ return self._jss.description() + @property + @ignore_unicode_prefix @since(2.1) def offsetDesc(self): """ Description of the current offsets up to which data has been written by the sink. - >>> sqs.sinkStatus().offsetDesc() + >>> sqs.sinkStatus.offsetDesc u'[#1, -]' """ return self._jss.offsetDesc() From dca9939487e77bd739c08149df495b89e5898656 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 12:59:38 -0700 Subject: [PATCH 31/43] Addressed comments --- python/pyspark/sql/streaming.py | 51 ++++++++++++------- .../execution/streaming/StreamExecution.scala | 8 +-- .../execution/streaming/StreamMetrics.scala | 4 -- .../spark/sql/streaming/SourceStatus.scala | 5 +- .../sql/streaming/StreamingQueryStatus.scala | 38 +++++++++++--- 5 files changed, 74 insertions(+), 32 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 0d0d2650deab3..6cad847d0a464 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -209,18 +209,25 @@ def __str__(self): Query name: query Query id: 1 Status timestamp: 123 - Input rate: 1.0 rows/sec - Processing rate 2.0 rows/sec + Input rate: 15.5 rows/sec + Processing rate 23.5 rows/sec Latency: 345.0 ms Trigger status: - key: value + isActive: true + isDataAvailable: true + latency.getBatch: 20 + latency.getOffset: 10 + numRows.input.total: 100 + triggerId: 5 Source statuses [1 source]: Source 1: MySource1 Available offset: #0 - Input rate: 4.0 rows/sec - Processing rate: 5.0 rows/sec + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec Trigger status: - key: value + numRows.input.source: 100 + latency.sourceGetOffset: 10 + latency.sourceGetBatch: 20 Sink status: MySink Committed offsets: [#1, -] """ @@ -268,7 +275,7 @@ def inputRate(self): Current rate (rows/sec) at which data is being generated by all the sources. >>> sqs.inputRate - 1.0 + 15.5 """ return self._jsqs.inputRate() @@ -279,7 +286,7 @@ def processingRate(self): Current rate (rows/sec) at which the query is processing data from all the sources. >>> sqs.processingRate - 2.0 + 23.5 """ return self._jsqs.processingRate() @@ -302,7 +309,7 @@ def latency(self): @since(2.1) def sourceStatuses(self): """ - Current statuses of the sources. + Current statuses of the sources as a list. >>> len(sqs.sourceStatuses) 1 @@ -326,10 +333,13 @@ def sinkStatus(self): @since(2.1) def triggerStatus(self): """ - Low-level detailed status of the last completed/currently active trigger. + Low-level detailed status of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. >>> sqs.triggerStatus - {u'key': u'value'} + {u'latency.getOffset': u'10', u'triggerId': u'5', u'isDataAvailable': u'true', u'numRows.input.total': u'100', u'latency.getBatch': u'20', u'isActive': u'true'} """ return self._jsqs.triggerStatus() @@ -353,10 +363,12 @@ def __str__(self): >>> print(sqs.sourceStatuses[0]) SourceStatus: MySource1 Available offset: #0 - Input rate: 4.0 rows/sec - Processing rate: 5.0 rows/sec + Input rate: 15.5 rows/sec + Processing rate: 23.5 rows/sec Trigger status: - key: value + numRows.input.source: 100 + latency.sourceGetOffset: 10 + latency.sourceGetBatch: 20 """ return self._jss.toString() @@ -391,7 +403,7 @@ def inputRate(self): Current rate (rows/sec) at which data is being generated by the source. >>> sqs.sourceStatuses[0].inputRate - 4.0 + 15.5 """ return self._jss.inputRate() @@ -402,7 +414,7 @@ def processingRate(self): Current rate (rows/sec) at which the query is processing data from the source. >>> sqs.sourceStatuses[0].processingRate - 5.0 + 23.5 """ return self._jss.processingRate() @@ -410,10 +422,13 @@ def processingRate(self): @since(2.1) def triggerStatus(self): """ - Low-level detailed status of the last completed/currently active trigger. + Low-level detailed status of the currently active trigger (e.g. number of rows processed + in trigger, latency of intermediate steps, etc.). + + If no trigger is currently active, then it will have details of the last completed trigger. >>> sqs.sourceStatuses[0].triggerStatus - {u'key': u'value'} + {u'numRows.input.source': u'100', u'latency.sourceGetOffset': u'10', u'latency.sourceGetBatch': u'20'} """ return self._jss.triggerStatus() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 06c12fdb213aa..01cd3d357a2fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -117,6 +117,7 @@ class StreamExecution( @volatile private var currentStatus: StreamingQueryStatus = null + /** Flag that signals whether any error with input metrics have already been logged */ @volatile private var metricWarningLogged: Boolean = false @@ -255,7 +256,8 @@ class StreamExecution( // Notify others sparkSession.streams.notifyQueryTermination(StreamExecution.this) - postEvent(new QueryTerminated(status, exception.map(_.cause).map(Utils.exceptionString))) + postEvent( + new QueryTerminated(currentStatus, exception.map(_.cause).map(Utils.exceptionString))) terminationLatch.countDown() } } @@ -581,7 +583,7 @@ class StreamExecution( // // 1. We keep track of streaming sources associated with each leaf in the trigger's logical plan // - Each logical plan leaf will be associated with a single streaming source. - // - There can be multiple logical plan leaves associated a streaming source. + // - There can be multiple logical plan leaves associated with a streaming source. // - There can be leaves not associated with any streaming source, because they were // generated from a batch source (e.g. stream-batch joins) // @@ -692,7 +694,7 @@ class StreamExecution( case object TERMINATED extends State } -object StreamExecution extends Logging { +object StreamExecution { private val _nextId = new AtomicLong(0) def nextId: Long = _nextId.getAndIncrement() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 307f120a48db2..7d5b485335968 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -52,7 +52,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // Number of input rows in the current trigger private val numInputRows = new mutable.HashMap[Source, Long] - private var numOutputRows: Option[Long] = None private var currentTriggerStartTimestamp: Long = -1 private var previousTriggerStartTimestamp: Long = -1 private var latency: Option[Double] = None @@ -81,7 +80,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def reportTriggerStarted(triggerId: Long): Unit = synchronized { numInputRows.clear() - numOutputRows = None triggerStatus.clear() sourceTriggerStatus.values.foreach(_.clear()) @@ -114,7 +112,6 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam // Report number of rows val totalNumInputRows = numInputRows.values.sum reportTriggerStatus(NUM_INPUT_ROWS, totalNumInputRows) - reportTriggerStatus(NUM_OUTPUT_ROWS, numOutputRows.getOrElse(0)) numInputRows.foreach { case (s, r) => reportSourceTriggerStatus(s, NUM_SOURCE_INPUT_ROWS, r) } @@ -237,7 +234,6 @@ object StreamMetrics extends Logging { val SOURCE_GET_BATCH_LATENCY = "latency.sourceGetBatch" val NUM_INPUT_ROWS = "numRows.input.total" - val NUM_OUTPUT_ROWS = "numRows.output" val NUM_SOURCE_INPUT_ROWS = "numRows.input.source" def NUM_TOTAL_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.total" def NUM_UPDATED_STATE_ROWS(aggId: Int): String = s"numRows.state.aggregation$aggId.updated" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 68b08bf4fbdf3..84dc8a11a2cfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -33,7 +33,10 @@ import org.apache.spark.sql.streaming.StreamingQueryStatus.indent * @param inputRate Current rate (rows/sec) at which data is being generated by the source. * @param processingRate Current rate (rows/sec) at which the query is processing data from * the source. - * @param triggerStatus Low-level detailed status of the last completed/currently active trigger. + * @param triggerStatus Low-level detailed status of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. * @since 2.0.0 */ @Experimental diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index ea4e7f90881ed..5ba07fa3b7ad6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -39,7 +39,10 @@ import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} * writing the corresponding output. * @param sourceStatuses Current statuses of the sources. * @param sinkStatus Current status of the sink. - * @param triggerStatus Low-level detailed status of the last completed/currently active trigger. + * @param triggerStatus Low-level detailed status of the currently active trigger (e.g. number of + * rows processed in trigger, latency of intermediate steps, etc.). + * If no trigger is active, then it will have details of the last completed + * trigger. * @since 2.0.0 */ @Experimental @@ -103,11 +106,34 @@ private[sql] object StreamingQueryStatus { /** Create an instance of status for python testing */ def testStatus(): StreamingQueryStatus = { + import org.apache.spark.sql.execution.streaming.StreamMetrics._ StreamingQueryStatus( - "query", 1, 123, 1.0, 2.0, Some(345), - Array( - SourceStatus("MySource1", LongOffset(0).toString, 4.0, 5.0, Map("key" -> "value"))), - SinkStatus("MySink", CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), - Map("key" -> "value")) + name = "query", + id = 1, + timestamp = 123, + inputRate = 15.5, + processingRate = 23.5, + latency = Some(345), + sourceStatuses = Array( + SourceStatus( + desc = "MySource1", + offsetDesc = LongOffset(0).toString, + inputRate = 15.5, + processingRate = 23.5, + triggerStatus = Map( + NUM_SOURCE_INPUT_ROWS -> "100", + SOURCE_GET_OFFSET_LATENCY -> "10", + SOURCE_GET_BATCH_LATENCY -> "20"))), + sinkStatus = SinkStatus( + desc = "MySink", + offsetDesc = CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), + triggerStatus = Map( + TRIGGER_ID -> "5", + ACTIVE -> "true", + DATA_AVAILABLE -> "true", + GET_OFFSET_LATENCY -> "10", + GET_BATCH_LATENCY -> "20", + NUM_INPUT_ROWS -> "100" + )) } } From af5854caf2ae8cdd3dcad180657040fc2538c1f6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 14:57:07 -0700 Subject: [PATCH 32/43] Fixed style --- python/pyspark/sql/streaming.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 6cad847d0a464..b7e54d0e6135f 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -339,7 +339,8 @@ def triggerStatus(self): If no trigger is currently active, then it will have details of the last completed trigger. >>> sqs.triggerStatus - {u'latency.getOffset': u'10', u'triggerId': u'5', u'isDataAvailable': u'true', u'numRows.input.total': u'100', u'latency.getBatch': u'20', u'isActive': u'true'} + {u'latency.getOffset': u'10', u'triggerId': u'5', u'isDataAvailable': u'true', + u'numRows.input.total': u'100', u'latency.getBatch': u'20', u'isActive': u'true'} """ return self._jsqs.triggerStatus() @@ -428,7 +429,8 @@ def triggerStatus(self): If no trigger is currently active, then it will have details of the last completed trigger. >>> sqs.sourceStatuses[0].triggerStatus - {u'numRows.input.source': u'100', u'latency.sourceGetOffset': u'10', u'latency.sourceGetBatch': u'20'} + {u'numRows.input.source': u'100', u'latency.sourceGetOffset': u'10', + u'latency.sourceGetBatch': u'20'} """ return self._jss.triggerStatus() From 3d7c71a24b3fbfe86fee074b9034db4b89eca2bb Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 16:19:20 -0700 Subject: [PATCH 33/43] Fixed more issues --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 2 - .../execution/streaming/StreamExecution.scala | 5 ++- .../apache/spark/sql/internal/SQLConf.scala | 8 ++++ .../StreamingQueryListenerSuite.scala | 7 +--- .../sql/streaming/StreamingQuerySuite.scala | 39 ++++++++++++++++--- 5 files changed, 47 insertions(+), 14 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 5f5dc78abda5a..6001c1f266963 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,8 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata - -import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 01cd3d357a2fb..787aa703df411 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.{QueryExecution, SparkPlan} import org.apache.spark.sql.execution.command.ExplainCommand +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} @@ -185,7 +186,9 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE - sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) + if (sparkSession.sqlContext.conf.streamingMetricsEnabled) { + sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) + } updateStatus() postEvent(new QueryStarted(currentStatus)) // Assumption: Does not throw exception. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index fecdf792fd14a..1b168481ea510 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -581,6 +581,12 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(10L) + val STREAMING_METRICS_ENABLED = + SQLConfigBuilder("spark.sql.streaming.metricsEnabled") + .doc("Whether Dropwizard/Codahale metrics will be reported for active streaming queries.") + .booleanConf + .createWithDefault(false) + val NDV_MAX_ERROR = SQLConfigBuilder("spark.sql.statistics.ndv.maxError") .internal() @@ -641,6 +647,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY) + def streamingMetricsEnabled: Boolean = getConf(STREAMING_METRICS_ENABLED) + def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES) def filesOpenCostInBytes: Long = getConf(FILES_OPEN_COST_IN_BYTES) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index b149c77003a00..c55d222f82fad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -73,14 +73,11 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } } - // This is to make sure that - // - Query waits for manual clock to be 600 first time there is data - // - Exec plan ends with a node (filter) that supports the numOutputRows metric - spark.conf.set("spark.sql.codegen.wholeStage", false) + // This is to make sure thatquery waits for manual clock to be 600 first time there is data val mapped = inputData.toDS().agg(count("*")).as[Long].coalesce(1).map { x => clock.waitTillTime(600) x - }.where("value != 100") + } val listener = new QueryStatusCollector withListenerAdded(listener) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index b54038e81f041..60ccc19c9527c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -26,6 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException @@ -157,12 +158,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { testQuietly("query statuses") { val inputData = MemoryStream[Int] - - // This is make the sure the execution plan ends with a node (filter) that supports - // the numOutputRows metric. - spark.conf.set("spark.sql.codegen.wholeStage", false) - val mapped = inputData.toDS().map(6 / _).where("value > 0") - + val mapped = inputData.toDS().map(6 / _) testStream(mapped)( AssertOnQuery(q => q.status.name === q.name), AssertOnQuery(q => q.status.id === q.id), @@ -241,6 +237,37 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { ) } + test("codahale metrics") { + val inputData = MemoryStream[Int] + + /** Whether metrics of a query is registered for reporting */ + def isMetricsRegistered(query: StreamingQuery): Boolean = { + val sourceName = s"StructuredStreaming.${query.name}" + val sources = spark.sparkContext.env.metricsSystem.getSourcesByName(sourceName) + require(sources.size <= 1) + sources.nonEmpty + } + // Disabled by default + assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + + withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + testStream(inputData.toDF)( + AssertOnQuery { q => !isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + + // Registered when enabled + withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + testStream(inputData.toDF)( + AssertOnQuery { q => isMetricsRegistered(q) }, + StopStream, + AssertOnQuery { q => !isMetricsRegistered(q) } + ) + } + } + test("input row calculation with mixed batch and streaming sources") { val streamingTriggerDF = spark.createDataset(1 to 10).toDF val streamingInputDF = createSingleTriggerStreamingDF(streamingTriggerDF).toDF("value") From 8b4bce8ff338aeb982beb6f93e79f09b718c46b6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 16:27:34 -0700 Subject: [PATCH 34/43] One more comment --- python/pyspark/sql/streaming.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index b7e54d0e6135f..43d771337eb0f 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -272,7 +272,7 @@ def timestamp(self): @since(2.1) def inputRate(self): """ - Current rate (rows/sec) at which data is being generated by all the sources. + Current total rate (rows/sec) at which data is being generated by all the sources. >>> sqs.inputRate 15.5 From 35bf5089f0d79ba0ba007ca9983a75616f1a553d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 19:11:01 -0700 Subject: [PATCH 35/43] Fixed python 3 tests --- python/pyspark/sql/streaming.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 43d771337eb0f..91ed93e37053e 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -306,6 +306,7 @@ def latency(self): return None @property + @ignore_unicode_prefix @since(2.1) def sourceStatuses(self): """ @@ -319,6 +320,7 @@ def sourceStatuses(self): return [SourceStatus(ss) for ss in self._jsqs.sourceStatuses()] @property + @ignore_unicode_prefix @since(2.1) def sinkStatus(self): """ @@ -330,6 +332,7 @@ def sinkStatus(self): return SinkStatus(self._jsqs.sinkStatus()) @property + @ignore_unicode_prefix @since(2.1) def triggerStatus(self): """ @@ -420,6 +423,7 @@ def processingRate(self): return self._jss.processingRate() @property + @ignore_unicode_prefix @since(2.1) def triggerStatus(self): """ From 4c08d569f7817e222550ef7578c6e01f90bc4ee0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 11 Oct 2016 23:00:06 -0700 Subject: [PATCH 36/43] Used conf correctly --- .../apache/spark/sql/execution/streaming/StreamExecution.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 787aa703df411..22e161f5bac58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -186,7 +186,7 @@ class StreamExecution( // Mark ACTIVE and then post the event. QueryStarted event is synchronously sent to listeners, // so must mark this as ACTIVE first. state = ACTIVE - if (sparkSession.sqlContext.conf.streamingMetricsEnabled) { + if (sparkSession.sessionState.conf.streamingMetricsEnabled) { sparkSession.sparkContext.env.metricsSystem.registerSource(streamMetrics) } updateStatus() From 38ac35eed2b2e634a44f664e3cd84ce86f867330 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 15:21:32 -0700 Subject: [PATCH 37/43] Addressed some comments --- .../execution/streaming/StreamExecution.scala | 34 +++++----- .../execution/streaming/StreamMetrics.scala | 13 ++-- .../sql/streaming/StreamingQueryStatus.scala | 4 +- .../sql/streaming/FileStreamSourceSuite.scala | 1 - .../spark/sql/streaming/StreamTest.scala | 57 ++++++++++++++++ .../StreamingQueryListenerSuite.scala | 67 ++----------------- .../sql/streaming/StreamingQuerySuite.scala | 1 - 7 files changed, 87 insertions(+), 90 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 22e161f5bac58..8846df4355022 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -212,14 +212,14 @@ class StreamExecution( constructNextBatch() } if (dataAvailable) { - streamMetrics.reportTriggerStatus(DATA_AVAILABLE, true) + streamMetrics.reportTriggerStatus(IS_DATA_PRESENT_IN_TRIGGER, true) streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Processing new data") updateStatus() runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 } else { - streamMetrics.reportTriggerStatus(DATA_AVAILABLE, false) + streamMetrics.reportTriggerStatus(IS_DATA_PRESENT_IN_TRIGGER, false) streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "No new data") updateStatus() Thread.sleep(pollingDelayMs) @@ -364,8 +364,6 @@ class StreamExecution( * Processes any data available between `availableOffsets` and `committedOffsets`. */ private def runBatch(): Unit = { - val startTime = System.nanoTime() - // TODO: Move this to IncrementalExecution. // Request unprocessed data from all sources. @@ -407,17 +405,15 @@ class StreamExecution( case a: Attribute if replacementMap.contains(a) => replacementMap(a) } - val optimizerStart = System.nanoTime() - lastExecution = new IncrementalExecution( - sparkSession, - triggerLogicalPlan, - outputMode, - checkpointFile("state"), - currentBatchId) - - val executedPlan = lastExecution.executedPlan // Force the lazy generation of execution plan - val optimizerTime = (System.nanoTime() - optimizerStart).toDouble / 1000000 - logDebug(s"Optimized batch in ${optimizerTime}ms") + val executedPlan = reportTimeTaken(OPTIMIZER_LATENCY) { + lastExecution = new IncrementalExecution( + sparkSession, + triggerLogicalPlan, + outputMode, + checkpointFile("state"), + currentBatchId) + lastExecution.executedPlan // Force the lazy generation of execution plan + } val nextBatch = new Dataset(sparkSession, lastExecution, RowEncoder(lastExecution.analyzed.schema)) @@ -432,8 +428,6 @@ class StreamExecution( awaitBatchLock.unlock() } - val batchTime = (System.nanoTime() - startTime).toDouble / 1000000 - logInfo(s"Completed up to $availableOffsets in ${batchTime}ms") // Update committed offsets. committedOffsets ++= availableOffsets } @@ -644,8 +638,12 @@ class StreamExecution( val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - streamMetrics.reportTriggerStatus(triggerStatusKey, math.max(endTime - startTime, 0)) + val timeTaken = math.max(endTime - startTime, 0) + streamMetrics.reportTriggerStatus(triggerStatusKey, timeTaken) updateStatus() + if (triggerStatusKey == TRIGGER_LATENCY) { + logInfo(s"Completed up to $availableOffsets in $timeTaken ms") + } result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 7d5b485335968..5aba193284146 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -85,7 +85,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam reportTriggerStatus(TRIGGER_ID, triggerId) sources.foreach(s => reportSourceTriggerStatus(s, TRIGGER_ID, triggerId)) - reportTriggerStatus(ACTIVE, true) + reportTriggerStatus(IS_TRIGGER_ACTIVE, true) currentTriggerStartTimestamp = triggerClock.getTimeMillis() reportTriggerStatus(START_TIMESTAMP, currentTriggerStartTimestamp) } @@ -107,7 +107,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() reportTriggerStatus(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) reportTriggerStatus(STATUS_MESSAGE, "") - reportTriggerStatus(ACTIVE, false) + reportTriggerStatus(IS_TRIGGER_ACTIVE, false) // Report number of rows val totalNumInputRows = numInputRows.values.sum @@ -217,8 +217,8 @@ object StreamMetrics extends Logging { val TRIGGER_ID = "triggerId" - val ACTIVE = "isActive" - val DATA_AVAILABLE = "isDataAvailable" + val IS_TRIGGER_ACTIVE = "isTriggerActive" + val IS_DATA_PRESENT_IN_TRIGGER = "isDataPresentInTrigger" val STATUS_MESSAGE = "statusMessage" val START_TIMESTAMP = "timestamp.triggerStart" @@ -229,9 +229,10 @@ object StreamMetrics extends Logging { val GET_OFFSET_LATENCY = "latency.getOffset" val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" val GET_BATCH_LATENCY = "latency.getBatch" + val OPTIMIZER_LATENCY = "latency.optimizer" val TRIGGER_LATENCY = "latency.fullTrigger" - val SOURCE_GET_OFFSET_LATENCY = "latency.sourceGetOffset" - val SOURCE_GET_BATCH_LATENCY = "latency.sourceGetBatch" + val SOURCE_GET_OFFSET_LATENCY = "latency.getOffset.source" + val SOURCE_GET_BATCH_LATENCY = "latency.getBatch.source" val NUM_INPUT_ROWS = "numRows.input.total" val NUM_SOURCE_INPUT_ROWS = "numRows.input.source" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index 5ba07fa3b7ad6..eeb449d99b77e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -129,8 +129,8 @@ private[sql] object StreamingQueryStatus { offsetDesc = CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), triggerStatus = Map( TRIGGER_ID -> "5", - ACTIVE -> "true", - DATA_AVAILABLE -> "true", + IS_TRIGGER_ACTIVE -> "true", + IS_DATA_PRESENT_IN_TRIGGER -> "true", GET_OFFSET_LATENCY -> "10", GET_BATCH_LATENCY -> "20", NUM_INPUT_ROWS -> "100" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index ac19f693083c8..6bb2f58d7c89e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -27,7 +27,6 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.streaming.StreamingQueryListenerSuite.QueryStatusCollector import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ import org.apache.spark.util.Utils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 31dd15072b82b..515b857c0c914 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -28,6 +28,8 @@ import scala.util.control.NonFatal import org.scalatest.Assertions import org.scalatest.concurrent.{Eventually, Timeouts} +import org.scalatest.concurrent.AsyncAssertions.Waiter +import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.Span @@ -38,6 +40,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, ManualClock, SystemClock, Utils} @@ -607,4 +610,58 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } } + + + class QueryStatusCollector extends StreamingQueryListener { + // to catch errors in the async listener events + @volatile private var asyncTestWaiter = new Waiter + + @volatile var startStatus: StreamingQueryStatus = null + @volatile var terminationStatus: StreamingQueryStatus = null + @volatile var terminationException: Option[String] = null + + private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryStatus] + + /** Get the info of the last trigger that processed data */ + def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { + progressStatuses.filter { i => + i.triggerStatus.get("isTriggerActive").toBoolean == false && + i.triggerStatus.get("isDataPresentInTrigger").toBoolean == true + }.lastOption + } + + def reset(): Unit = { + startStatus = null + terminationStatus = null + progressStatuses.clear() + asyncTestWaiter = new Waiter + } + + def checkAsyncErrors(): Unit = { + asyncTestWaiter.await(timeout(10 seconds)) + } + + + override def onQueryStarted(queryStarted: QueryStarted): Unit = { + asyncTestWaiter { + startStatus = queryStarted.queryStatus + } + } + + override def onQueryProgress(queryProgress: QueryProgress): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryProgress called before onQueryStarted") + synchronized { progressStatuses += queryProgress.queryStatus } + } + } + + override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { + asyncTestWaiter { + assert(startStatus != null, "onQueryTerminated called before onQueryStarted") + terminationStatus = queryTerminated.queryStatus + terminationException = queryTerminated.exception + } + asyncTestWaiter.dismiss() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index c55d222f82fad..4146dad28e85d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -17,22 +17,16 @@ package org.apache.spark.sql.streaming -import scala.collection.mutable -import scala.collection.JavaConverters._ - import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ -import org.scalatest.concurrent.AsyncAssertions.Waiter import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout -import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.util.{JsonProtocol, ManualClock} @@ -97,7 +91,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // onQueryProgress val status = listener.lastTriggerStatus.get assert(status.triggerStatus.get("triggerId") == "0") - assert(status.triggerStatus.get("isActive") === "false") + assert(status.triggerStatus.get("isTriggerActive") === "false") + assert(status.triggerStatus.get("isDataPresentInTrigger") === "true") assert(status.triggerStatus.get("timestamp.triggerStart") === "0") assert(status.triggerStatus.get("timestamp.afterGetOffset") === "100") @@ -106,6 +101,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.triggerStatus.get("latency.getOffset") === "100") assert(status.triggerStatus.get("latency.getBatch") === "200") + assert(status.triggerStatus.get("latency.optimizer") === "0") assert(status.triggerStatus.get("latency.offsetLogWrite") === "0") assert(status.triggerStatus.get("latency.fullTrigger") === "600") @@ -115,8 +111,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(status.sourceStatuses.size === 1) assert(status.sourceStatuses(0).triggerStatus.get("triggerId") === "0") - assert(status.sourceStatuses(0).triggerStatus.get("latency.sourceGetOffset") === "100") - assert(status.sourceStatuses(0).triggerStatus.get("latency.sourceGetBatch") === "200") + assert(status.sourceStatuses(0).triggerStatus.get("latency.getOffset.source") === "100") + assert(status.sourceStatuses(0).triggerStatus.get("latency.getBatch.source") === "200") assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "2") true }, @@ -273,57 +269,4 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { object StreamingQueryListenerSuite { // Singleton reference to clock that does not get serialized in task closures @volatile var clock: ManualClock = null - - class QueryStatusCollector extends StreamingQueryListener { - // to catch errors in the async listener events - @volatile private var asyncTestWaiter = new Waiter - - @volatile var startStatus: StreamingQueryStatus = null - @volatile var terminationStatus: StreamingQueryStatus = null - @volatile var terminationException: Option[String] = null - - private val progressStatuses = new mutable.ArrayBuffer[StreamingQueryStatus] - - /** Get the info of the last trigger that processed data */ - def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { - progressStatuses.filter { i => - i.triggerStatus.get("isActive").toBoolean == false && - i.triggerStatus.get("isDataAvailable").toBoolean == true - }.lastOption - } - - def reset(): Unit = { - startStatus = null - terminationStatus = null - progressStatuses.clear() - asyncTestWaiter = new Waiter - } - - def checkAsyncErrors(): Unit = { - asyncTestWaiter.await(timeout(10 seconds)) - } - - - override def onQueryStarted(queryStarted: QueryStarted): Unit = { - asyncTestWaiter { - startStatus = queryStarted.queryStatus - } - } - - override def onQueryProgress(queryProgress: QueryProgress): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryProgress called before onQueryStarted") - synchronized { progressStatuses += queryProgress.queryStatus } - } - } - - override def onQueryTerminated(queryTerminated: QueryTerminated): Unit = { - asyncTestWaiter { - assert(startStatus != null, "onQueryTerminated called before onQueryStarted") - terminationStatus = queryTerminated.queryStatus - terminationException = queryTerminated.exception - } - asyncTestWaiter.dismiss() - } - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 60ccc19c9527c..16ffd877a116e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -26,7 +26,6 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException From 42bc7bf3d83381fd0fd1679a04a6f8c18089f5fa Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 16:15:49 -0700 Subject: [PATCH 38/43] Addressed comments --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 20 +++---- python/pyspark/sql/streaming.py | 43 +++++++-------- .../execution/streaming/StreamExecution.scala | 36 ++++++------- .../execution/streaming/StreamMetrics.scala | 46 ++++++++-------- .../spark/sql/streaming/SourceStatus.scala | 16 +++--- .../sql/streaming/StreamingQueryStatus.scala | 18 +++---- .../sql/streaming/FileStreamSourceSuite.scala | 2 +- .../spark/sql/streaming/StreamTest.scala | 4 +- .../StreamingQueryListenerSuite.scala | 53 ++++++++----------- .../sql/streaming/StreamingQuerySuite.scala | 51 ++---------------- 10 files changed, 119 insertions(+), 170 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 6001c1f266963..c969c8d4bc3b6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -285,20 +285,20 @@ class KafkaSourceSuite extends KafkaSourceTest { val listener = new QueryStatusCollector spark.streams.addListener(listener) try { - testStream(mapped)( + testStream(mapped)( makeSureGetOffsetCalled, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), - AssertOnQuery { query => - eventually(timeout(streamingTimeout)) { - assert(listener.lastTriggerStatus.nonEmpty) - } - val status = listener.lastTriggerStatus.get - assert(status.triggerStatus.get("numRows.input.total").toInt > 0) - assert(status.sourceStatuses(0).processingRate > 0.0) - true + AssertOnQuery { query => + eventually(timeout(streamingTimeout)) { + assert(listener.lastTriggerStatus.nonEmpty) } - ) + val status = listener.lastTriggerStatus.get + assert(status.triggerDetails.get("numRows.input.total").toInt > 0) + assert(status.sourceStatuses(0).processingRate > 0.0) + true + } + ) } finally { spark.streams.removeListener(listener) } diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 91ed93e37053e..e17ba1ba9f274 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -212,11 +212,11 @@ def __str__(self): Input rate: 15.5 rows/sec Processing rate 23.5 rows/sec Latency: 345.0 ms - Trigger status: - isActive: true - isDataAvailable: true - latency.getBatch: 20 - latency.getOffset: 10 + Trigger details: + isDataPresentInTrigger: true + isTriggerActive: true + latency.getBatch.total: 20 + latency.getOffset.total: 10 numRows.input.total: 100 triggerId: 5 Source statuses [1 source]: @@ -224,10 +224,10 @@ def __str__(self): Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec - Trigger status: + Trigger details: numRows.input.source: 100 - latency.sourceGetOffset: 10 - latency.sourceGetBatch: 20 + latency.getOffset.source: 10 + latency.getBatch.source: 20 Sink status: MySink Committed offsets: [#1, -] """ @@ -334,18 +334,19 @@ def sinkStatus(self): @property @ignore_unicode_prefix @since(2.1) - def triggerStatus(self): + def triggerDetails(self): """ Low-level detailed status of the currently active trigger (e.g. number of rows processed in trigger, latency of intermediate steps, etc.). If no trigger is currently active, then it will have details of the last completed trigger. - >>> sqs.triggerStatus - {u'latency.getOffset': u'10', u'triggerId': u'5', u'isDataAvailable': u'true', - u'numRows.input.total': u'100', u'latency.getBatch': u'20', u'isActive': u'true'} + >>> sqs.triggerDetails + {u'triggerId': u'5', u'latency.getBatch.total': u'20', u'numRows.input.total': u'100', + u'isTriggerActive': u'true', u'latency.getOffset.total': u'10', + u'isDataPresentInTrigger': u'true'} """ - return self._jsqs.triggerStatus() + return self._jsqs.triggerDetails() class SourceStatus(object): @@ -369,10 +370,10 @@ def __str__(self): Available offset: #0 Input rate: 15.5 rows/sec Processing rate: 23.5 rows/sec - Trigger status: + Trigger details: numRows.input.source: 100 - latency.sourceGetOffset: 10 - latency.sourceGetBatch: 20 + latency.getOffset.source: 10 + latency.getBatch.source: 20 """ return self._jss.toString() @@ -425,18 +426,18 @@ def processingRate(self): @property @ignore_unicode_prefix @since(2.1) - def triggerStatus(self): + def triggerDetails(self): """ Low-level detailed status of the currently active trigger (e.g. number of rows processed in trigger, latency of intermediate steps, etc.). If no trigger is currently active, then it will have details of the last completed trigger. - >>> sqs.sourceStatuses[0].triggerStatus - {u'numRows.input.source': u'100', u'latency.sourceGetOffset': u'10', - u'latency.sourceGetBatch': u'20'} + >>> sqs.sourceStatuses[0].triggerDetails + {u'numRows.input.source': u'100', u'latency.getOffset.source': u'10', + u'latency.getBatch.source': u'20'} """ - return self._jss.triggerStatus() + return self._jss.triggerDetails() class SinkStatus(object): diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 8846df4355022..9144736c940f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -200,7 +200,7 @@ class StreamExecution( triggerExecutor.execute(() => { streamMetrics.reportTriggerStarted(currentBatchId) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Finding new data from sources") + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Finding new data from sources") updateStatus() val isTerminated = reportTimeTaken(TRIGGER_LATENCY) { if (isActive) { @@ -212,15 +212,15 @@ class StreamExecution( constructNextBatch() } if (dataAvailable) { - streamMetrics.reportTriggerStatus(IS_DATA_PRESENT_IN_TRIGGER, true) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "Processing new data") + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, true) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "Processing new data") updateStatus() runBatch() // We'll increase currentBatchId after we complete processing current batch's data currentBatchId += 1 } else { - streamMetrics.reportTriggerStatus(IS_DATA_PRESENT_IN_TRIGGER, false) - streamMetrics.reportTriggerStatus(STATUS_MESSAGE, "No new data") + streamMetrics.reportTriggerDetail(IS_DATA_PRESENT_IN_TRIGGER, false) + streamMetrics.reportTriggerDetail(STATUS_MESSAGE, "No new data") updateStatus() Thread.sleep(pollingDelayMs) } @@ -624,41 +624,41 @@ class StreamExecution( streamMetrics.reportNumInputRows(sourceToNumInputRows) stateNodes.zipWithIndex.foreach { case (s, i) => - streamMetrics.reportTriggerStatus( + streamMetrics.reportTriggerDetail( NUM_TOTAL_STATE_ROWS(i + 1), s.metrics.get("numTotalStateRows").map(_.value).getOrElse(0L)) - streamMetrics.reportTriggerStatus( + streamMetrics.reportTriggerDetail( NUM_UPDATED_STATE_ROWS(i + 1), s.metrics.get("numUpdatedStateRows").map(_.value).getOrElse(0L)) } updateStatus() } - private def reportTimeTaken[T](triggerStatusKey: String)(body: => T): T = { + private def reportTimeTaken[T](triggerDetailKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() val timeTaken = math.max(endTime - startTime, 0) - streamMetrics.reportTriggerStatus(triggerStatusKey, timeTaken) + streamMetrics.reportTriggerDetail(triggerDetailKey, timeTaken) updateStatus() - if (triggerStatusKey == TRIGGER_LATENCY) { + if (triggerDetailKey == TRIGGER_LATENCY) { logInfo(s"Completed up to $availableOffsets in $timeTaken ms") } result } - private def reportTimeTaken[T](source: Source, triggerStatusKey: String)(body: => T): T = { + private def reportTimeTaken[T](source: Source, triggerDetailKey: String)(body: => T): T = { val startTime = triggerClock.getTimeMillis() val result = body val endTime = triggerClock.getTimeMillis() - streamMetrics.reportSourceTriggerStatus( - source, triggerStatusKey, math.max(endTime - startTime, 0)) + streamMetrics.reportSourceTriggerDetail( + source, triggerDetailKey, math.max(endTime - startTime, 0)) updateStatus() result } - private def reportTimestamp(triggerStatusKey: String): Unit = { - streamMetrics.reportTriggerStatus(triggerStatusKey, triggerClock.getTimeMillis) + private def reportTimestamp(triggerDetailKey: String): Unit = { + streamMetrics.reportTriggerDetail(triggerDetailKey, triggerClock.getTimeMillis) updateStatus() } @@ -667,10 +667,10 @@ class StreamExecution( val sourceStatuses = sources.map { s => SourceStatus( s.toString, - localAvailableOffsets.get(s).map(_.toString).getOrElse("-"), + localAvailableOffsets.get(s).map(_.toString).getOrElse("-"), // TODO: use json if available streamMetrics.currentSourceInputRate(s), streamMetrics.currentSourceProcessingRate(s), - streamMetrics.currentSourceTriggerStatus(s)) + streamMetrics.currentSourceTriggerDetails(s)) }.toArray val sinkStatus = SinkStatus( sink.toString, @@ -686,7 +686,7 @@ class StreamExecution( latency = streamMetrics.currentLatency(), sourceStatuses = sourceStatuses, sinkStatus = sinkStatus, - triggerStatus = streamMetrics.currentTriggerStatus()) + triggerDetails = streamMetrics.currentTriggerDetails()) } trait State diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 5aba193284146..4d679851a7573 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -43,8 +43,8 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam import StreamMetrics._ // Trigger infos - private val triggerStatus = new mutable.HashMap[String, String] - private val sourceTriggerStatus = new mutable.HashMap[Source, mutable.HashMap[String, String]] + private val triggerDetails = new mutable.HashMap[String, String] + private val sourceTriggerDetails = new mutable.HashMap[Source, mutable.HashMap[String, String]] // Rate estimators for sources and sinks private val inputRates = new mutable.HashMap[Source, RateCalculator] @@ -70,7 +70,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam sources.foreach { s => inputRates.put(s, new RateCalculator) processingRates.put(s, new RateCalculator) - sourceTriggerStatus.put(s, new mutable.HashMap[String, String]) + sourceTriggerDetails.put(s, new mutable.HashMap[String, String]) registerGauge(s"inputRate-${s.toString}", () => currentSourceInputRate(s)) registerGauge(s"processingRate-${s.toString}", () => currentSourceProcessingRate(s)) @@ -80,22 +80,22 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def reportTriggerStarted(triggerId: Long): Unit = synchronized { numInputRows.clear() - triggerStatus.clear() - sourceTriggerStatus.values.foreach(_.clear()) + triggerDetails.clear() + sourceTriggerDetails.values.foreach(_.clear()) - reportTriggerStatus(TRIGGER_ID, triggerId) - sources.foreach(s => reportSourceTriggerStatus(s, TRIGGER_ID, triggerId)) - reportTriggerStatus(IS_TRIGGER_ACTIVE, true) + reportTriggerDetail(TRIGGER_ID, triggerId) + sources.foreach(s => reportSourceTriggerDetail(s, TRIGGER_ID, triggerId)) + reportTriggerDetail(IS_TRIGGER_ACTIVE, true) currentTriggerStartTimestamp = triggerClock.getTimeMillis() - reportTriggerStatus(START_TIMESTAMP, currentTriggerStartTimestamp) + reportTriggerDetail(START_TIMESTAMP, currentTriggerStartTimestamp) } - def reportTriggerStatus[T](key: String, value: T): Unit = synchronized { - triggerStatus.put(key, value.toString) + def reportTriggerDetail[T](key: String, value: T): Unit = synchronized { + triggerDetails.put(key, value.toString) } - def reportSourceTriggerStatus[T](source: Source, key: String, value: T): Unit = synchronized { - sourceTriggerStatus(source).put(key, value.toString) + def reportSourceTriggerDetail[T](source: Source, key: String, value: T): Unit = synchronized { + sourceTriggerDetails(source).put(key, value.toString) } def reportNumInputRows(inputRows: Map[Source, Long]): Unit = synchronized { @@ -105,15 +105,15 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def reportTriggerFinished(): Unit = synchronized { require(currentTriggerStartTimestamp >= 0) val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() - reportTriggerStatus(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) - reportTriggerStatus(STATUS_MESSAGE, "") - reportTriggerStatus(IS_TRIGGER_ACTIVE, false) + reportTriggerDetail(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) + reportTriggerDetail(STATUS_MESSAGE, "") + reportTriggerDetail(IS_TRIGGER_ACTIVE, false) // Report number of rows val totalNumInputRows = numInputRows.values.sum - reportTriggerStatus(NUM_INPUT_ROWS, totalNumInputRows) + reportTriggerDetail(NUM_INPUT_ROWS, totalNumInputRows) numInputRows.foreach { case (s, r) => - reportSourceTriggerStatus(s, NUM_SOURCE_INPUT_ROWS, r) + reportSourceTriggerDetail(s, NUM_SOURCE_INPUT_ROWS, r) } val currentTriggerDuration = currentTriggerFinishTimestamp - currentTriggerStartTimestamp @@ -173,10 +173,10 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam def currentLatency(): Option[Double] = synchronized { latency } - def currentTriggerStatus(): Map[String, String] = synchronized { triggerStatus.toMap } + def currentTriggerDetails(): Map[String, String] = synchronized { triggerDetails.toMap } - def currentSourceTriggerStatus(source: Source): Map[String, String] = synchronized { - sourceTriggerStatus(source).toMap + def currentSourceTriggerDetails(source: Source): Map[String, String] = synchronized { + sourceTriggerDetails(source).toMap } // =========== Other methods =========== @@ -226,9 +226,9 @@ object StreamMetrics extends Logging { val GET_BATCH_TIMESTAMP = "timestamp.afterGetBatch" val FINISH_TIMESTAMP = "timestamp.triggerFinish" - val GET_OFFSET_LATENCY = "latency.getOffset" + val GET_OFFSET_LATENCY = "latency.getOffset.total" + val GET_BATCH_LATENCY = "latency.getBatch.total" val OFFSET_WAL_WRITE_LATENCY = "latency.offsetLogWrite" - val GET_BATCH_LATENCY = "latency.getBatch" val OPTIMIZER_LATENCY = "latency.optimizer" val TRIGGER_LATENCY = "latency.fullTrigger" val SOURCE_GET_OFFSET_LATENCY = "latency.getOffset.source" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala index 84dc8a11a2cfb..6ace4833be22f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/SourceStatus.scala @@ -33,7 +33,7 @@ import org.apache.spark.sql.streaming.StreamingQueryStatus.indent * @param inputRate Current rate (rows/sec) at which data is being generated by the source. * @param processingRate Current rate (rows/sec) at which the query is processing data from * the source. - * @param triggerStatus Low-level detailed status of the currently active trigger (e.g. number of + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of * rows processed in trigger, latency of intermediate steps, etc.). * If no trigger is active, then it will have details of the last completed * trigger. @@ -45,20 +45,20 @@ class SourceStatus private( val offsetDesc: String, val inputRate: Double, val processingRate: Double, - val triggerStatus: ju.Map[String, String]) { + val triggerDetails: ju.Map[String, String]) { override def toString: String = "SourceStatus:" + indent(prettyString) private[sql] def prettyString: String = { - val triggerStatusStrings = - triggerStatus.asScala.map { case (k, v) => s"$k: $v" } + val triggerDetailsLines = + triggerDetails.asScala.map { case (k, v) => s"$k: $v" } s"""$description |Available offset: $offsetDesc |Input rate: $inputRate rows/sec |Processing rate: $processingRate rows/sec - |Trigger status: - |""".stripMargin + indent(triggerStatusStrings) + |Trigger details: + |""".stripMargin + indent(triggerDetailsLines) } } @@ -70,7 +70,7 @@ private[sql] object SourceStatus { offsetDesc: String, inputRate: Double, processingRate: Double, - triggerStatus: Map[String, String]): SourceStatus = { - new SourceStatus(desc, offsetDesc, inputRate, processingRate, triggerStatus.asJava) + triggerDetails: Map[String, String]): SourceStatus = { + new SourceStatus(desc, offsetDesc, inputRate, processingRate, triggerDetails.asJava) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index eeb449d99b77e..47689928730d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -39,7 +39,7 @@ import org.apache.spark.sql.execution.streaming.{CompositeOffset, LongOffset} * writing the corresponding output. * @param sourceStatuses Current statuses of the sources. * @param sinkStatus Current status of the sink. - * @param triggerStatus Low-level detailed status of the currently active trigger (e.g. number of + * @param triggerDetails Low-level details of the currently active trigger (e.g. number of * rows processed in trigger, latency of intermediate steps, etc.). * If no trigger is active, then it will have details of the last completed * trigger. @@ -55,7 +55,7 @@ class StreamingQueryStatus private( val latency: Option[Double], val sourceStatuses: Array[SourceStatus], val sinkStatus: SinkStatus, - val triggerStatus: ju.Map[String, String]) { + val triggerDetails: ju.Map[String, String]) { import StreamingQueryStatus._ @@ -64,7 +64,7 @@ class StreamingQueryStatus private( s"Source ${i + 1}:" + indent(s.prettyString) } val sinkStatusLines = sinkStatus.prettyString - val triggerStatusLines = triggerStatus.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted + val triggerDetailsLines = triggerDetails.asScala.map { case (k, v) => s"$k: $v" }.toSeq.sorted val numSources = sourceStatuses.length val numSourcesString = s"$numSources source" + { if (numSources > 1) "s" else "" } @@ -75,8 +75,8 @@ class StreamingQueryStatus private( |Input rate: $inputRate rows/sec |Processing rate $processingRate rows/sec |Latency: ${latency.getOrElse("-")} ms - |Trigger status: - |${indent(triggerStatusLines)} + |Trigger details: + |${indent(triggerDetailsLines)} |Source statuses [$numSourcesString]: |${indent(sourceStatusLines)} |Sink status: ${indent(sinkStatusLines)}""".stripMargin @@ -96,9 +96,9 @@ private[sql] object StreamingQueryStatus { latency: Option[Double], sourceStatuses: Array[SourceStatus], sinkStatus: SinkStatus, - triggerStatus: Map[String, String]): StreamingQueryStatus = { + triggerDetails: Map[String, String]): StreamingQueryStatus = { new StreamingQueryStatus(name, id, timestamp, inputRate, processingRate, - latency, sourceStatuses, sinkStatus, triggerStatus.asJava) + latency, sourceStatuses, sinkStatus, triggerDetails.asJava) } def indent(strings: Iterable[String]): String = strings.map(indent).mkString("\n") @@ -120,14 +120,14 @@ private[sql] object StreamingQueryStatus { offsetDesc = LongOffset(0).toString, inputRate = 15.5, processingRate = 23.5, - triggerStatus = Map( + triggerDetails = Map( NUM_SOURCE_INPUT_ROWS -> "100", SOURCE_GET_OFFSET_LATENCY -> "10", SOURCE_GET_BATCH_LATENCY -> "20"))), sinkStatus = SinkStatus( desc = "MySink", offsetDesc = CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), - triggerStatus = Map( + triggerDetails = Map( TRIGGER_ID -> "5", IS_TRIGGER_ACTIVE -> "true", IS_DATA_PRESENT_IN_TRIGGER -> "true", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 6bb2f58d7c89e..917342d506f87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1014,7 +1014,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { assert(listener.lastTriggerStatus.nonEmpty) } val status = listener.lastTriggerStatus.get - assert(status.triggerStatus.get("numRows.input.total") === "1") + assert(status.triggerDetails.get("numRows.input.total") === "1") assert(status.sourceStatuses(0).processingRate > 0.0) true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 515b857c0c914..747e78e76fd8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -625,8 +625,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { /** Get the info of the last trigger that processed data */ def lastTriggerStatus: Option[StreamingQueryStatus] = synchronized { progressStatuses.filter { i => - i.triggerStatus.get("isTriggerActive").toBoolean == false && - i.triggerStatus.get("isDataPresentInTrigger").toBoolean == true + i.triggerDetails.get("isTriggerActive").toBoolean == false && + i.triggerDetails.get("isDataPresentInTrigger").toBoolean == true }.lastOption } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 4146dad28e85d..1ceafe530f365 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -90,30 +90,30 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // Check the correctness of the trigger info of the first completed batch reported by // onQueryProgress val status = listener.lastTriggerStatus.get - assert(status.triggerStatus.get("triggerId") == "0") - assert(status.triggerStatus.get("isTriggerActive") === "false") - assert(status.triggerStatus.get("isDataPresentInTrigger") === "true") + assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.get("isTriggerActive") === "false") + assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") - assert(status.triggerStatus.get("timestamp.triggerStart") === "0") - assert(status.triggerStatus.get("timestamp.afterGetOffset") === "100") - assert(status.triggerStatus.get("timestamp.afterGetBatch") === "300") - assert(status.triggerStatus.get("timestamp.triggerFinish") === "600") + assert(status.triggerDetails.get("timestamp.triggerStart") === "0") + assert(status.triggerDetails.get("timestamp.afterGetOffset") === "100") + assert(status.triggerDetails.get("timestamp.afterGetBatch") === "300") + assert(status.triggerDetails.get("timestamp.triggerFinish") === "600") - assert(status.triggerStatus.get("latency.getOffset") === "100") - assert(status.triggerStatus.get("latency.getBatch") === "200") - assert(status.triggerStatus.get("latency.optimizer") === "0") - assert(status.triggerStatus.get("latency.offsetLogWrite") === "0") - assert(status.triggerStatus.get("latency.fullTrigger") === "600") + assert(status.triggerDetails.get("latency.getOffset.total") === "100") + assert(status.triggerDetails.get("latency.getBatch.total") === "200") + assert(status.triggerDetails.get("latency.optimizer") === "0") + assert(status.triggerDetails.get("latency.offsetLogWrite") === "0") + assert(status.triggerDetails.get("latency.fullTrigger") === "600") - assert(status.triggerStatus.get("numRows.input.total") === "2") - assert(status.triggerStatus.get("numRows.state.aggregation1.total") === "1") - assert(status.triggerStatus.get("numRows.state.aggregation1.updated") === "1") + assert(status.triggerDetails.get("numRows.input.total") === "2") + assert(status.triggerDetails.get("numRows.state.aggregation1.total") === "1") + assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus.get("triggerId") === "0") - assert(status.sourceStatuses(0).triggerStatus.get("latency.getOffset.source") === "100") - assert(status.sourceStatuses(0).triggerStatus.get("latency.getBatch.source") === "200") - assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "2") + assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") true }, CheckAnswer(2) @@ -188,7 +188,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryStarted serialization") { - val queryStarted = new StreamingQueryListener.QueryStarted(testQueryStatus) + val queryStarted = new StreamingQueryListener.QueryStarted(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryStarted) val newQueryStarted = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryStarted] @@ -196,7 +196,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } test("QueryProgress serialization") { - val queryProcess = new StreamingQueryListener.QueryProgress(testQueryStatus) + val queryProcess = new StreamingQueryListener.QueryProgress(StreamingQueryStatus.testStatus) val json = JsonProtocol.sparkEventToJson(queryProcess) val newQueryProcess = JsonProtocol.sparkEventFromJson(json) .asInstanceOf[StreamingQueryListener.QueryProgress] @@ -206,7 +206,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { test("QueryTerminated serialization") { val exception = new RuntimeException("exception") val queryQueryTerminated = new StreamingQueryListener.QueryTerminated( - testQueryStatus, + StreamingQueryStatus.testStatus, Some(exception.getMessage)) val json = JsonProtocol.sparkEventToJson(queryQueryTerminated) @@ -255,15 +255,6 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val listenerBus = spark.streams invokePrivate listenerBusMethod() listenerBus.listeners.toArray.map(_.asInstanceOf[StreamingQueryListener]) } - - private val testQueryStatus: StreamingQueryStatus = { - StreamingQueryStatus( - "name", 1, 123, 1.0, 2.0, Some(345), - Array( - SourceStatus("source1", LongOffset(0).toString, 0.0, 0.0, Map("a" -> "b"))), - SinkStatus("sink", CompositeOffset(Some(LongOffset(1)) :: None :: Nil).toString), - Map("a" -> "b")) - } } object StreamingQueryListenerSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 16ffd877a116e..54d2e2030c452 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -45,49 +45,6 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { sqlContext.streams.active.foreach(_.stop()) } - abstract class AddFileData extends AddData { - override def addData(query: Option[StreamExecution]): (Source, Offset) = { - require( - query.nonEmpty, - "Cannot add data when there is no query for finding the active file stream source") - - val sources = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source, _) if source.isInstanceOf[FileStreamSource] => - source.asInstanceOf[FileStreamSource] - } - if (sources.isEmpty) { - throw new Exception( - "Could not find file source in the StreamExecution logical plan to add data to") - } else if (sources.size > 1) { - throw new Exception( - "Could not select the file source in the StreamExecution logical plan as there" + - "are multiple file sources:\n\t" + sources.mkString("\n\t")) - } - val source = sources.head - val newOffset = source.withBatchingLocked { - addData(source) - source.currentOffset + 1 - } - logInfo(s"Added file to $source at offset $newOffset") - (source, newOffset) - } - - protected def addData(source: FileStreamSource): Unit - } - - case class AddTextFileData(content: String, src: File, tmp: File) - extends AddFileData { - - override def addData(source: FileStreamSource): Unit = { - val tempFile = Utils.tempFileWith(new File(tmp, "text")) - val finalFile = new File(src, tempFile.getName) - src.mkdirs() - require(stringToFile(tempFile, content).renameTo(finalFile)) - logInfo(s"Written text '$content' to file $finalFile") - } - } - - test("names unique across active queries, ids unique across all started queries") { val inputData = MemoryStream[Int] val mapped = inputData.toDS().map { 6 / _} @@ -275,9 +232,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // Trigger input has 10 rows, static input has 2 rows, // therefore after the first trigger, the calculated input rows should be 10 val status = getFirstTriggerStatus(streamingInputDF.join(staticInputDF, "value")) - assert(status.triggerStatus.get("numRows.input.total") === "10") + assert(status.triggerDetails.get("numRows.input.total") === "10") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "10") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") } test("input row calculation with trigger DF having multiple leaves") { @@ -288,9 +245,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { // After the first trigger, the calculated input rows should be 10 val status = getFirstTriggerStatus(streamingInputDF) - assert(status.triggerStatus.get("numRows.input.total") === "10") + assert(status.triggerDetails.get("numRows.input.total") === "10") assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerStatus.get("numRows.input.source") === "10") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "10") } testQuietly("StreamExecution metadata garbage collection") { From 00a741519e07fdda6dc2e4161e0f0d4382ef7c0a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 17:01:05 -0700 Subject: [PATCH 39/43] Fixed test --- .../scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index c969c8d4bc3b6..3e2c75b74837c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -26,7 +26,6 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.streaming.StreamingQueryListenerSuite.QueryStatusCollector import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.SharedSQLContext From cafbeb72f064295a6d9b07c31515e59f14c17305 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 17:50:54 -0700 Subject: [PATCH 40/43] Addressed more comments --- .../spark/sql/kafka010/KafkaSourceSuite.scala | 30 ++----- .../sql/streaming/FileStreamSourceSuite.scala | 30 ++----- .../spark/sql/streaming/StreamTest.scala | 14 +++ .../StreamingQueryListenerSuite.scala | 85 ++++++++----------- 4 files changed, 69 insertions(+), 90 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala index 3e2c75b74837c..8b5296ea135c7 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceSuite.scala @@ -22,7 +22,6 @@ import java.util.concurrent.atomic.AtomicInteger import scala.util.Random import org.apache.kafka.clients.producer.RecordMetadata -import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql.execution.streaming._ @@ -281,26 +280,15 @@ class KafkaSourceSuite extends KafkaSourceTest { .as[(String, String)] val mapped = kafka.map(kv => kv._2.toInt + 1) - val listener = new QueryStatusCollector - spark.streams.addListener(listener) - try { - testStream(mapped)( - makeSureGetOffsetCalled, - AddKafkaData(Set(topic), 1, 2, 3), - CheckAnswer(2, 3, 4), - AssertOnQuery { query => - eventually(timeout(streamingTimeout)) { - assert(listener.lastTriggerStatus.nonEmpty) - } - val status = listener.lastTriggerStatus.get - assert(status.triggerDetails.get("numRows.input.total").toInt > 0) - assert(status.sourceStatuses(0).processingRate > 0.0) - true - } - ) - } finally { - spark.streams.removeListener(listener) - } + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 1, 2, 3), + CheckAnswer(2, 3, 4), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total").toInt > 0) + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) } private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 917342d506f87..aabdccaaf319d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.streaming import java.io.File import org.scalatest.PrivateMethodTester -import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.sql._ @@ -1002,26 +1001,15 @@ class FileStreamSourceSuite extends FileStreamSourceTest { test("input row metrics") { withTempDirs { case (src, tmp) => - val input = spark.readStream.format("text").load(src.getCanonicalPath).as[String] - val listener = new QueryStatusCollector - spark.streams.addListener(listener) - try { - testStream(input)( - AddTextFileData("100", src, tmp), - CheckAnswer("100"), - AssertOnQuery { query => - eventually(timeout(streamingTimeout)) { - assert(listener.lastTriggerStatus.nonEmpty) - } - val status = listener.lastTriggerStatus.get - assert(status.triggerDetails.get("numRows.input.total") === "1") - assert(status.sourceStatuses(0).processingRate > 0.0) - true - } - ) - } finally { - spark.streams.removeListener(listener) - } + val input = spark.readStream.format("text").load(src.getCanonicalPath) + testStream(input)( + AddTextFileData("100", src, tmp), + CheckAnswer("100"), + AssertOnLastQueryStatus { status => + assert(status.triggerDetails.get("numRows.input.total") === "1") + assert(status.sourceStatuses(0).processingRate > 0.0) + } + ) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 747e78e76fd8f..3b9d3786349ad 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -201,6 +201,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { } } + case class AssertOnLastQueryStatus(condition: StreamingQueryStatus => Unit) + extends StreamAction + + /** * Executes the specified actions on the given streaming DataFrame and provides helpful * error messages in the case of failures or incorrect answers. @@ -302,8 +306,10 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val testThread = Thread.currentThread() val metadataRoot = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath + val statusCollector = new QueryStatusCollector try { + spark.streams.addListener(statusCollector) startedTest.foreach { action => logInfo(s"Processing test stream action: $action") action match { @@ -403,6 +409,13 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { val streamToAssert = Option(currentStream).getOrElse(lastStream) verify({ a.run(); true }, s"Assert failed: ${a.message}") + case a: AssertOnLastQueryStatus => + Eventually.eventually(timeout(streamingTimeout)) { + require(statusCollector.lastTriggerStatus.nonEmpty) + } + val status = statusCollector.lastTriggerStatus.get + verify({ a.condition(status); true }, "Assert on last query status failed") + case a: AddData => try { // Add data and get the source where it was added, and the expected offset of the @@ -477,6 +490,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { if (currentStream != null && currentStream.microBatchThread.isAlive) { currentStream.stop() } + spark.streams.removeListener(statusCollector) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index 1ceafe530f365..6256385dfd0e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.streaming import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.PrivateMethodTester._ -import org.scalatest.concurrent.Eventually._ -import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.apache.spark.SparkException import org.apache.spark.sql.DataFrame @@ -73,52 +71,43 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { x } - val listener = new QueryStatusCollector - withListenerAdded(listener) { - testStream(mapped, OutputMode.Complete)( - StartStream(triggerClock = clock), - AddData(inputData, 1, 2), - AdvanceManualClock(100), // unblock getOffset, will block on getBatch - AdvanceManualClock(200), // unblock getBatch, will block on computation - AdvanceManualClock(300), // unblock computation - AssertOnQuery("Incorrect trigger info") { query => - require(clock.getTimeMillis() === 600) - eventually(Timeout(streamingTimeout)) { - assert(listener.lastTriggerStatus.nonEmpty) - } - - // Check the correctness of the trigger info of the first completed batch reported by - // onQueryProgress - val status = listener.lastTriggerStatus.get - assert(status.triggerDetails.get("triggerId") == "0") - assert(status.triggerDetails.get("isTriggerActive") === "false") - assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") - - assert(status.triggerDetails.get("timestamp.triggerStart") === "0") - assert(status.triggerDetails.get("timestamp.afterGetOffset") === "100") - assert(status.triggerDetails.get("timestamp.afterGetBatch") === "300") - assert(status.triggerDetails.get("timestamp.triggerFinish") === "600") - - assert(status.triggerDetails.get("latency.getOffset.total") === "100") - assert(status.triggerDetails.get("latency.getBatch.total") === "200") - assert(status.triggerDetails.get("latency.optimizer") === "0") - assert(status.triggerDetails.get("latency.offsetLogWrite") === "0") - assert(status.triggerDetails.get("latency.fullTrigger") === "600") - - assert(status.triggerDetails.get("numRows.input.total") === "2") - assert(status.triggerDetails.get("numRows.state.aggregation1.total") === "1") - assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") - - assert(status.sourceStatuses.size === 1) - assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") - assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") - assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") - assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") - true - }, - CheckAnswer(2) - ) - } + testStream(mapped, OutputMode.Complete)( + StartStream(triggerClock = clock), + AddData(inputData, 1, 2), + AdvanceManualClock(100), // unblock getOffset, will block on getBatch + AdvanceManualClock(200), // unblock getBatch, will block on computation + AdvanceManualClock(300), // unblock computation + AssertOnQuery { _ => clock.getTimeMillis() === 600 }, + AssertOnLastQueryStatus { status: StreamingQueryStatus => + // Check the correctness of the trigger info of the last completed batch reported by + // onQueryProgress + assert(status.triggerDetails.get("triggerId") == "0") + assert(status.triggerDetails.get("isTriggerActive") === "false") + assert(status.triggerDetails.get("isDataPresentInTrigger") === "true") + + assert(status.triggerDetails.get("timestamp.triggerStart") === "0") + assert(status.triggerDetails.get("timestamp.afterGetOffset") === "100") + assert(status.triggerDetails.get("timestamp.afterGetBatch") === "300") + assert(status.triggerDetails.get("timestamp.triggerFinish") === "600") + + assert(status.triggerDetails.get("latency.getOffset.total") === "100") + assert(status.triggerDetails.get("latency.getBatch.total") === "200") + assert(status.triggerDetails.get("latency.optimizer") === "0") + assert(status.triggerDetails.get("latency.offsetLogWrite") === "0") + assert(status.triggerDetails.get("latency.fullTrigger") === "600") + + assert(status.triggerDetails.get("numRows.input.total") === "2") + assert(status.triggerDetails.get("numRows.state.aggregation1.total") === "1") + assert(status.triggerDetails.get("numRows.state.aggregation1.updated") === "1") + + assert(status.sourceStatuses.length === 1) + assert(status.sourceStatuses(0).triggerDetails.get("triggerId") === "0") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getOffset.source") === "100") + assert(status.sourceStatuses(0).triggerDetails.get("latency.getBatch.source") === "200") + assert(status.sourceStatuses(0).triggerDetails.get("numRows.input.source") === "2") + }, + CheckAnswer(2) + ) } test("adding and removing listener") { From 49da4b144907f3a2ad65c8ee82636a2b79c65493 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 23:51:30 -0700 Subject: [PATCH 41/43] Fixed trigger details bug --- .../execution/streaming/StreamMetrics.scala | 3 +- .../streaming/StreamMetricsSuite.scala | 33 ++++++++++++++++--- .../sql/streaming/StreamingQuerySuite.scala | 1 + 3 files changed, 32 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala index 4d679851a7573..e98d1883e4596 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetrics.scala @@ -106,7 +106,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam require(currentTriggerStartTimestamp >= 0) val currentTriggerFinishTimestamp = triggerClock.getTimeMillis() reportTriggerDetail(FINISH_TIMESTAMP, currentTriggerFinishTimestamp) - reportTriggerDetail(STATUS_MESSAGE, "") + triggerDetails.remove(STATUS_MESSAGE) reportTriggerDetail(IS_TRIGGER_ACTIVE, false) // Report number of rows @@ -190,6 +190,7 @@ class StreamMetrics(sources: Set[Source], triggerClock: Clock, codahaleSourceNam } def stop(): Unit = synchronized { + triggerDetails.clear() inputRates.valuesIterator.foreach { _.stop() } processingRates.valuesIterator.foreach { _.stop() } latency = None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala index c00043689186a..938423db64745 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/StreamMetricsSuite.scala @@ -25,25 +25,35 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.util.ManualClock class StreamMetricsSuite extends SparkFunSuite { + import StreamMetrics._ // To make === between double tolerate inexact values implicit val doubleEquality = TolerantNumerics.tolerantDoubleEquality(0.01) - test("rates and latencies - basic life cycle") { + test("rates, latencies, trigger details - basic life cycle") { val sm = newStreamMetrics(source) assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) - // When trigger started, the rates should not change + // When trigger started, the rates should not change, but should return + // reported trigger details sm.reportTriggerStarted(1) + sm.reportTriggerDetail("key", "value") + sm.reportSourceTriggerDetail(source, "key2", "value2") assert(sm.currentInputRate() === 0.0) assert(sm.currentProcessingRate() === 0.0) assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "true", + START_TIMESTAMP -> "0", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", "key2" -> "value2")) // Finishing the trigger should calculate the rates, except input rate which needs // to have another trigger interval @@ -55,10 +65,24 @@ class StreamMetricsSuite extends SparkFunSuite { assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 100.0) assert(sm.currentLatency() === None) - - // Another trigger should calculate the input rate + assert(sm.currentTriggerDetails() === + Map(TRIGGER_ID -> "1", IS_TRIGGER_ACTIVE -> "false", + START_TIMESTAMP -> "0", FINISH_TIMESTAMP -> "1000", + NUM_INPUT_ROWS -> "100", "key" -> "value")) + assert(sm.currentSourceTriggerDetails(source) === + Map(TRIGGER_ID -> "1", NUM_SOURCE_INPUT_ROWS -> "100", "key2" -> "value2")) + + // After another trigger starts, the rates and latencies should not change until + // new rows are reported clock.advance(1000) sm.reportTriggerStarted(2) + assert(sm.currentInputRate() === 0.0) + assert(sm.currentProcessingRate() === 100.0) + assert(sm.currentSourceInputRate(source) === 0.0) + assert(sm.currentSourceProcessingRate(source) === 100.0) + assert(sm.currentLatency() === None) + + // Reporting new rows should update the rates and latencies sm.reportNumInputRows(Map(source -> 200L)) // 200 input rows clock.advance(500) sm.reportTriggerFinished() @@ -75,6 +99,7 @@ class StreamMetricsSuite extends SparkFunSuite { assert(sm.currentSourceInputRate(source) === 0.0) assert(sm.currentSourceProcessingRate(source) === 0.0) assert(sm.currentLatency() === None) + assert(sm.currentTriggerDetails().isEmpty) } test("rates and latencies - after trigger with no data") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 54d2e2030c452..4c90eafc91bc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -174,6 +174,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging { AssertOnQuery(_.sourceStatuses(0).inputRate === 0.0), AssertOnQuery(_.sourceStatuses(0).processingRate === 0.0), AssertOnQuery(_.sinkStatus.offsetDesc === CompositeOffset.fill(LongOffset(1)).toString), + AssertOnQuery(_.status.triggerDetails.isEmpty), StartStream(), AddData(inputData, 0), From 839d402d6f5a04e1c1f4994d823ac5845ec4ab76 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 23:55:40 -0700 Subject: [PATCH 42/43] Removed unnecessary imports --- .../org/apache/spark/sql/streaming/StreamingQuerySuite.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 4c90eafc91bc5..9f8e2db966367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.sql.streaming -import java.io.File - import org.scalactic.TolerantNumerics import org.scalatest.concurrent.Eventually._ import org.scalatest.BeforeAndAfter import org.apache.spark.internal.Logging import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.types.StructType import org.apache.spark.SparkException From 59722126a74a03ca2e0b0e8c2f3e3477a302aad0 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 12 Oct 2016 23:57:34 -0700 Subject: [PATCH 43/43] Updated python docs --- python/pyspark/sql/streaming.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index e17ba1ba9f274..ce47bd1640fb1 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -336,7 +336,7 @@ def sinkStatus(self): @since(2.1) def triggerDetails(self): """ - Low-level detailed status of the currently active trigger (e.g. number of rows processed + Low-level details of the currently active trigger (e.g. number of rows processed in trigger, latency of intermediate steps, etc.). If no trigger is currently active, then it will have details of the last completed trigger. @@ -428,7 +428,7 @@ def processingRate(self): @since(2.1) def triggerDetails(self): """ - Low-level detailed status of the currently active trigger (e.g. number of rows processed + Low-level details of the currently active trigger (e.g. number of rows processed in trigger, latency of intermediate steps, etc.). If no trigger is currently active, then it will have details of the last completed trigger.