Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-38564][SS] Support collecting metrics from streaming sinks #35872

Closed
wants to merge 5 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,36 @@
/*
* 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.connector.read.streaming;

import org.apache.spark.annotation.Evolving;

import java.util.Map;

/**
* A mix-in interface for streaming sinks to signal that they can report
* metrics.
*
* @since 3.4.0
*/
@Evolving
public interface ReportsSinkMetrics {
/**
* Returns the metrics reported by the sink for this micro-batch
*/
Map<String, String> metrics();
}
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalP
import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReportsSourceMetrics, SparkDataStream}
import org.apache.spark.sql.connector.read.streaming.{MicroBatchStream, ReportsSinkMetrics, ReportsSourceMetrics, SparkDataStream}
import org.apache.spark.sql.execution.QueryExecution
import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2Relation, StreamWriterCommitProgress}
import org.apache.spark.sql.streaming._
Expand Down Expand Up @@ -200,7 +200,16 @@ trait ProgressReporter extends Logging {
} else {
sinkCommitProgress.map(_ => 0L)
}
val sinkProgress = SinkProgress(sink.toString, sinkOutput)

val sinkMetrics = sink match {
case withMetrics: ReportsSinkMetrics =>
withMetrics.metrics()
case _ => Map[String, String]().asJava
}

val sinkProgress = SinkProgress(
sink.toString, sinkOutput, sinkMetrics)

val observedMetrics = extractObservedMetrics(hasNewData, lastExecution)

val newProgress = new StreamingQueryProgress(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -238,7 +238,8 @@ class SourceProgress protected[sql](
@Evolving
class SinkProgress protected[sql](
val description: String,
val numOutputRows: Long) extends Serializable {
val numOutputRows: Long,
val metrics: ju.Map[String, String] = Map[String, String]().asJava) extends Serializable {

/** SinkProgress without custom metrics. */
protected[sql] def this(description: String) = {
Expand All @@ -255,15 +256,17 @@ class SinkProgress protected[sql](

private[sql] def jsonValue: JValue = {
("description" -> JString(description)) ~
("numOutputRows" -> JInt(numOutputRows))
("numOutputRows" -> JInt(numOutputRows)) ~
("metrics" -> safeMapToJValue[String](metrics, s => JString(s)))
}
}

private[sql] object SinkProgress {
val DEFAULT_NUM_OUTPUT_ROWS: Long = -1L

def apply(description: String, numOutputRows: Option[Long]): SinkProgress =
new SinkProgress(description, numOutputRows.getOrElse(DEFAULT_NUM_OUTPUT_ROWS))
def apply(description: String, numOutputRows: Option[Long],
metrics: ju.Map[String, String] = Map[String, String]().asJava): SinkProgress =
new SinkProgress(description, numOutputRows.getOrElse(DEFAULT_NUM_OUTPUT_ROWS), metrics)
}

private object SafeJsonSerializer {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,153 @@
/*
* 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.streaming

import scala.collection.JavaConverters._

import org.apache.spark.internal.Logging
import org.apache.spark.sql._
import org.apache.spark.sql.connector.catalog.{SupportsWrite, Table, TableCapability}
import org.apache.spark.sql.connector.read.streaming.ReportsSinkMetrics
import org.apache.spark.sql.connector.write._
import org.apache.spark.sql.connector.write.streaming.{StreamingDataWriterFactory, StreamingWrite}
import org.apache.spark.sql.execution.streaming.MemoryStream
import org.apache.spark.sql.execution.streaming.sources.PackedRowWriterFactory
import org.apache.spark.sql.internal.connector.{SimpleTableProvider, SupportsStreamingUpdateAsAppend}
import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.util.CaseInsensitiveStringMap

class ReportSinkMetricsSuite extends StreamTest {
Copy link
Member

Choose a reason for hiding this comment

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

The tests added here seem flaky:

ReportSinkMetricsSuite:
- test ReportSinkMetrics *** FAILED *** (244 milliseconds)
  Expected null, but got {"metrics-1"="value-1", "metrics-2"="value-2"} (ReportSinkMetricsSuite.scala:75)
  org.scalatest.exceptions.TestFailedException:
  at org.scalatest.Assertions.newAssertionFailedException(Assertions.scala:472)
  at org.scalatest.Assertions.newAssertionFailedException$(Assertions.scala:471)
  at org.scalatest.funsuite.AnyFunSuite.newAssertionFailedException(AnyFunSuite.scala:1563)
  at org.scalatest.Assertions.assertResult(Assertions.scala:867)
  at org.scalatest.Assertions.assertResult$(Assertions.scala:863)
  at org.scalatest.funsuite.AnyFunSuite.assertResult(AnyFunSuite.scala:1563)
  at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$2(ReportSinkMetricsSuite.scala:75)
  at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$2$adapted(ReportSinkMetricsSuite.scala:60)
  at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1(SQLTestUtils.scala:79)
  at org.apache.spark.sql.test.SQLTestUtils.$anonfun$withTempDir$1$adapted(SQLTestUtils.scala:78)
  at org.apache.spark.SparkFunSuite.withTempDir(SparkFunSuite.scala:221)
  at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.org$apache$spark$sql$test$SQLTestUtils$$super$withTempDir(ReportSinkMetricsSuite.scala:35)
  at org.apache.spark.sql.test.SQLTestUtils.withTempDir(SQLTestUtils.scala:78)
  at org.apache.spark.sql.test.SQLTestUtils.withTempDir$(SQLTestUtils.scala:77)
  at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.withTempDir(ReportSinkMetricsSuite.scala:35)
  at org.apache.spark.sql.streaming.ReportSinkMetricsSuite.$anonfun$new$1(ReportSinkMetricsSuite.scala:60)
  at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.java:23)

https://github.com/apache/spark/runs/5646670314?check_suite_focus=true

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for reporting. Seems odd. @jerrypeng Could you please check this?

Copy link
Member

Choose a reason for hiding this comment

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

Actually, the fix seems pretty simple .. I made a quick followup- #35945

Copy link
Contributor

Choose a reason for hiding this comment

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


import testImplicits._

test("test ReportSinkMetrics") {
val inputData = MemoryStream[Int]
val df = inputData.toDF()
var query: StreamingQuery = null

var metricsMap: java.util.Map[String, String] = null

val listener = new StreamingQueryListener {

override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {}

override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {
metricsMap = event.progress.sink.metrics
}

override def onQueryTerminated(
event: StreamingQueryListener.QueryTerminatedEvent): Unit = {}
}

spark.streams.addListener(listener)

withTempDir { dir =>
try {
query =
df.writeStream
.outputMode("append")
.format("org.apache.spark.sql.streaming.TestSinkProvider")
.option("checkPointLocation", dir.toString)
.start()

inputData.addData(1, 2, 3)

failAfter(streamingTimeout) {
query.processAllAvailable()
}

assertResult(metricsMap) {
Copy link
Contributor

Choose a reason for hiding this comment

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

OK I think we missed that listener callback happens in different thread than stream thread.

@jerrypeng
We may need to add sc.listenerBus.waitUntilEmpty(), or wrap this with eventually. Could you please create a follow-up PR? Thanks in advance!

Map("metrics-1" -> "value-1", "metrics-2" -> "value-2").asJava
}
} finally {
if (query != null) {
query.stop()
}

spark.streams.removeListener(listener)
}
}
}
}

case class TestSinkRelation(override val sqlContext: SQLContext, data: DataFrame)
extends BaseRelation {
override def schema: StructType = data.schema
}

class TestSinkProvider extends SimpleTableProvider
with DataSourceRegister
with CreatableRelationProvider with Logging {

override def getTable(options: CaseInsensitiveStringMap): Table = {
TestSinkTable
}

def createRelation(
sqlContext: SQLContext,
mode: SaveMode,
parameters: Map[String, String],
data: DataFrame): BaseRelation = {

TestSinkRelation(sqlContext, data)
}

def shortName(): String = "test"
}

object TestSinkTable extends Table with SupportsWrite with ReportsSinkMetrics with Logging {

override def name(): String = "test"

override def schema(): StructType = StructType(Nil)

override def capabilities(): java.util.Set[TableCapability] = {
java.util.EnumSet.of(TableCapability.STREAMING_WRITE)
}

override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = {
new WriteBuilder with SupportsTruncate with SupportsStreamingUpdateAsAppend {

override def truncate(): WriteBuilder = this

override def build(): Write = {
new Write {
override def toStreaming: StreamingWrite = {
new TestSinkWrite()
}
}
}
}
}

override def metrics(): java.util.Map[String, String] = {
Map("metrics-1" -> "value-1", "metrics-2" -> "value-2").asJava
}
}

class TestSinkWrite()
extends StreamingWrite with Logging with Serializable {

def createStreamingWriterFactory(info: PhysicalWriteInfo): StreamingDataWriterFactory =
PackedRowWriterFactory

override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}

def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {}
}