-
Notifications
You must be signed in to change notification settings - Fork 28.4k
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-24748][SS] Support for reporting custom metrics via StreamingQuery Progress #21721
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,33 @@ | ||
/* | ||
* 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.sources.v2; | ||
|
||
import org.apache.spark.annotation.InterfaceStability; | ||
|
||
/** | ||
* An interface for reporting custom metrics from streaming sources and sinks | ||
*/ | ||
@InterfaceStability.Evolving | ||
public interface CustomMetrics { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Java side should also be 2 spaced indented (see "Code Style Guide" in https://spark.apache.org/contributing.html) |
||
/** | ||
* Returns a JSON serialized representation of custom metrics | ||
* | ||
* @return JSON serialized representation of custom metrics | ||
*/ | ||
String json(); | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.sources.v2.reader.streaming; | ||
|
||
import org.apache.spark.annotation.InterfaceStability; | ||
import org.apache.spark.sql.sources.v2.CustomMetrics; | ||
import org.apache.spark.sql.sources.v2.reader.DataSourceReader; | ||
|
||
/** | ||
* A mix in interface for {@link DataSourceReader}. Data source readers can implement this | ||
* interface to report custom metrics that gets reported under the | ||
* {@link org.apache.spark.sql.streaming.SourceProgress} | ||
* | ||
*/ | ||
@InterfaceStability.Evolving | ||
public interface SupportsCustomReaderMetrics extends DataSourceReader { | ||
/** | ||
* Returns custom metrics specific to this data source. | ||
*/ | ||
CustomMetrics getCustomMetrics(); | ||
|
||
/** | ||
* Invoked if the custom metrics returned by {@link #getCustomMetrics()} is invalid | ||
* (e.g. Invalid data that cannot be parsed). Throwing an error here would ensure that | ||
* your custom metrics work right and correct values are reported always. The default action | ||
* on invalid metrics is to ignore it. | ||
* | ||
* @param ex the exception | ||
*/ | ||
default void onInvalidMetrics(Exception ex) { | ||
// default is to ignore invalid custom metrics | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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.sources.v2.writer.streaming; | ||
|
||
import org.apache.spark.annotation.InterfaceStability; | ||
import org.apache.spark.sql.sources.v2.CustomMetrics; | ||
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; | ||
|
||
/** | ||
* A mix in interface for {@link DataSourceWriter}. Data source writers can implement this | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If we intend creating a new interface as mix-in, we may not need to create individual interfaces for each DataSourceReader and DataSourceWriter. We could have only one interface and let DataSourceReader and DataSourceWriter add such mix-in interface. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The intention was to restrict the mixin so that it can be applied only to A few options:
I prefer option 1, but would like to proceed based on the feedback. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. OK got your intention. I think it makes sense. I'm OK with all three options and personally prefer 1 or 2 if the intention is to mix-in, but let's see committers' feedback on this. |
||
* interface to report custom metrics that gets reported under the | ||
* {@link org.apache.spark.sql.streaming.SinkProgress} | ||
* | ||
*/ | ||
@InterfaceStability.Evolving | ||
public interface SupportsCustomWriterMetrics extends DataSourceWriter { | ||
/** | ||
* Returns custom metrics specific to this data source. | ||
*/ | ||
CustomMetrics getCustomMetrics(); | ||
|
||
/** | ||
* Invoked if the custom metrics returned by {@link #getCustomMetrics()} is invalid | ||
* (e.g. Invalid data that cannot be parsed). Throwing an error here would ensure that | ||
* your custom metrics work right and correct values are reported always. The default action | ||
* on invalid metrics is to ignore it. | ||
* | ||
* @param ex the exception | ||
*/ | ||
default void onInvalidMetrics(Exception ex) { | ||
// default is to ignore invalid custom metrics | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -22,14 +22,22 @@ import java.util.{Date, UUID} | |
|
||
import scala.collection.JavaConverters._ | ||
import scala.collection.mutable | ||
import scala.util.control.NonFatal | ||
|
||
import org.json4s.JsonAST.JValue | ||
import org.json4s.jackson.JsonMethods.parse | ||
|
||
import org.apache.spark.internal.Logging | ||
import org.apache.spark.sql.{DataFrame, SparkSession} | ||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} | ||
import org.apache.spark.sql.catalyst.util.DateTimeUtils | ||
import org.apache.spark.sql.execution.QueryExecution | ||
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec | ||
import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader | ||
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, WriteToDataSourceV2Exec} | ||
import org.apache.spark.sql.execution.streaming.sources.MicroBatchWriter | ||
import org.apache.spark.sql.sources.v2.CustomMetrics | ||
import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, SupportsCustomReaderMetrics} | ||
import org.apache.spark.sql.sources.v2.writer.DataSourceWriter | ||
import org.apache.spark.sql.sources.v2.writer.streaming.SupportsCustomWriterMetrics | ||
import org.apache.spark.sql.streaming._ | ||
import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent | ||
import org.apache.spark.util.Clock | ||
|
@@ -156,18 +164,51 @@ trait ProgressReporter extends Logging { | |
} | ||
logDebug(s"Execution stats: $executionStats") | ||
|
||
// extracts and validates custom metrics from readers and writers | ||
def extractMetrics( | ||
getMetrics: () => Option[CustomMetrics], | ||
onInvalidMetrics: (Exception) => Unit): Option[String] = { | ||
try { | ||
getMetrics().map(m => { | ||
val json = m.json() | ||
parse(json) | ||
json | ||
}) | ||
} catch { | ||
case ex: Exception if NonFatal(ex) => | ||
onInvalidMetrics(ex) | ||
None | ||
} | ||
} | ||
|
||
val sourceProgress = sources.distinct.map { source => | ||
val customReaderMetrics = source match { | ||
case s: SupportsCustomReaderMetrics => | ||
extractMetrics(() => Option(s.getCustomMetrics), s.onInvalidMetrics) | ||
|
||
case _ => None | ||
} | ||
|
||
val numRecords = executionStats.inputRows.getOrElse(source, 0L) | ||
new SourceProgress( | ||
description = source.toString, | ||
startOffset = currentTriggerStartOffsets.get(source).orNull, | ||
endOffset = currentTriggerEndOffsets.get(source).orNull, | ||
numInputRows = numRecords, | ||
inputRowsPerSecond = numRecords / inputTimeSec, | ||
processedRowsPerSecond = numRecords / processingTimeSec | ||
processedRowsPerSecond = numRecords / processingTimeSec, | ||
customReaderMetrics.orNull | ||
) | ||
} | ||
val sinkProgress = new SinkProgress(sink.toString) | ||
|
||
val customWriterMetrics = dataSourceWriter match { | ||
case Some(s: SupportsCustomWriterMetrics) => | ||
extractMetrics(() => Option(s.getCustomMetrics), s.onInvalidMetrics) | ||
|
||
case _ => None | ||
} | ||
|
||
val sinkProgress = new SinkProgress(sink.toString, customWriterMetrics.orNull) | ||
|
||
val newProgress = new StreamingQueryProgress( | ||
id = id, | ||
|
@@ -196,6 +237,18 @@ trait ProgressReporter extends Logging { | |
currentStatus = currentStatus.copy(isTriggerActive = false) | ||
} | ||
|
||
/** Extract writer from the executed query plan. */ | ||
private def dataSourceWriter: Option[DataSourceWriter] = { | ||
if (lastExecution == null) return None | ||
lastExecution.executedPlan.collect { | ||
case p if p.isInstanceOf[WriteToDataSourceV2Exec] => | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. this only works for microbatch mode, do we have a plan to support continuous mode? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yes, currently the progress is reported only for micro-batch mode. This should be supported for continuous mode as well when we start reporting progress, but needs some more work - https://issues.apache.org/jira/browse/SPARK-23887 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. i see, thanks! |
||
p.asInstanceOf[WriteToDataSourceV2Exec].writer | ||
}.headOption match { | ||
case Some(w: MicroBatchWriter) => Some(w.writer) | ||
case _ => None | ||
} | ||
} | ||
|
||
/** Extract statistics about stateful operators from the executed query plan. */ | ||
private def extractStateOperatorMetrics(hasNewData: Boolean): Seq[StateOperatorProgress] = { | ||
if (lastExecution == null) return Nil | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Shall we switch this to
Unstable
instead for now?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I opened #22296 in case we want.