Skip to content
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
Expand Up @@ -18,6 +18,8 @@
package org.apache.spark.sql.connector.catalog;

import org.apache.spark.annotation.Evolving;
import org.apache.spark.sql.connector.metric.CustomMetric;
import org.apache.spark.sql.connector.metric.CustomTaskMetric;

/**
* Represents a table which can be atomically truncated.
Expand All @@ -34,4 +36,25 @@ public interface TruncatableTable extends Table {
* @since 3.2.0
*/
boolean truncateTable();

/**
* Returns an array of supported custom metrics with name and description.
* By default it returns empty array.
*
* @since 4.2.0
*/
default CustomMetric[] supportedCustomMetrics() {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Do we need @since for the new methods?

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

Added.

return new CustomMetric[]{};
}

/**
* Returns an array of custom metrics which are collected with values at the driver side only.
* Note that these metrics must be included in the supported custom metrics reported by
* `supportedCustomMetrics`.
*
* @since 4.2.0
*/
default CustomTaskMetric[] reportDriverMetrics() {
return new CustomTaskMetric[]{};
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -129,7 +129,7 @@ case class BatchScanExec(
new DataSourceRDD(
sparkContext, filteredPartitions, readerFactory, supportsColumnar, customMetrics)
}
postDriverMetrics()
postDriverMetrics(scan.reportDriverMetrics())
rdd
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ case class ContinuousScanExec(
schema,
readerFactory,
customMetrics)
postDriverMetrics()
postDriverMetrics(scan.reportDriverMetrics())
inputRDD
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,23 +24,22 @@ import org.apache.spark.sql.catalyst.plans.physical
import org.apache.spark.sql.catalyst.plans.physical.KeyedPartitioning
import org.apache.spark.sql.catalyst.util.truncatedString
import org.apache.spark.sql.connector.read.{HasPartitionKey, InputPartition, PartitionReaderFactory, Scan}
import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode, SafeForKWayMerge, SQLExecution}
import org.apache.spark.sql.execution.metric.SQLMetrics
import org.apache.spark.sql.execution.{ExplainUtils, LeafExecNode, SafeForKWayMerge}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.internal.connector.SupportsMetadata
import org.apache.spark.sql.vectorized.ColumnarBatch
import org.apache.spark.util.ArrayImplicits._
import org.apache.spark.util.Utils

trait DataSourceV2ScanExecBase extends LeafExecNode with SafeForKWayMerge {
trait DataSourceV2ScanExecBase
extends LeafExecNode
with SafeForKWayMerge
with SupportsCustomDriverMetrics {

lazy val customMetrics = scan.supportedCustomMetrics().map { customMetric =>
customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric)
}.toMap
override lazy val customMetrics: Map[String, SQLMetric] =
createCustomMetrics(scan.supportedCustomMetrics())

override lazy val metrics = {
Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) ++
customMetrics
}
override protected lazy val sparkMetrics: Map[String, SQLMetric] =
Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))

def scan: Scan

Expand Down Expand Up @@ -145,18 +144,6 @@ trait DataSourceV2ScanExecBase extends LeafExecNode with SafeForKWayMerge {
}
}

protected def postDriverMetrics(): Unit = {
val driveSQLMetrics = scan.reportDriverMetrics().map(customTaskMetric => {
val metric = metrics(customTaskMetric.name())
metric.set(customTaskMetric.value())
metric
})

val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
SQLMetrics.postDriverMetricUpdates(sparkContext, executionId,
driveSQLMetrics.toImmutableArraySeq)
}

override def doExecuteColumnar(): RDD[ColumnarBatch] = {
val numOutputRows = longMetric("numOutputRows")
inputRDD.asInstanceOf[RDD[ColumnarBatch]].map { b =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,18 +23,29 @@ import org.apache.spark.sql.catalyst.transactions.TransactionUtils
import org.apache.spark.sql.connector.catalog.SupportsDeleteV2
import org.apache.spark.sql.connector.catalog.transactions.Transaction
import org.apache.spark.sql.connector.expressions.filter.Predicate
import org.apache.spark.sql.execution.metric.SQLMetric

case class DeleteFromTableExec(
table: SupportsDeleteV2,
condition: Array[Predicate],
refreshCache: () => Unit,
transaction: Option[Transaction] = None) extends LeafV2CommandExec with TransactionalExec {
transaction: Option[Transaction] = None)
extends LeafV2CommandExec
with TransactionalExec
with SupportsCustomDriverMetrics {

override lazy val customMetrics: Map[String, SQLMetric] =
createCustomMetrics(table.supportedCustomMetrics())

override def withTransaction(txn: Option[Transaction]): DeleteFromTableExec =
copy(transaction = txn)

override protected def run(): Seq[InternalRow] = {
table.deleteWhere(condition)
try {
table.deleteWhere(condition)
} finally {
postDriverMetrics(table.reportDriverMetrics())
}
transaction.foreach(TransactionUtils.commit)
refreshCache()
Seq.empty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@ case class MicroBatchScanExec(
override lazy val inputRDD: RDD[InternalRow] = {
val inputRDD = new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar,
customMetrics)
postDriverMetrics()
postDriverMetrics(scan.reportDriverMetrics())
inputRDD
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -166,7 +166,7 @@ case class RealTimeStreamScanExec(
supportsColumnar,
customMetrics
)
postDriverMetrics()
postDriverMetrics(scan.reportDriverMetrics())
inputRDD
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,80 @@
/*
* 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.datasources.v2

import org.apache.spark.sql.connector.metric.{CustomMetric, CustomTaskMetric}
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.util.ArrayImplicits._

/**
* A mixin for Spark plan nodes that expose driver-side custom metrics reported by a connector.
* Implementations declare the connector-owned metrics via [[customMetrics]]; after the underlying
* operation has executed they call [[postDriverMetrics]] with the connector's reported values so
* they are visible in the SQL UI.
*
* Nodes that also expose Spark-owned metrics supply them via [[sparkMetrics]]. Names in
* [[sparkMetrics]] are reserved: if the connector happens to report a value under the same name,
* Spark's value wins and the connector's is dropped.
*/
trait SupportsCustomDriverMetrics { self: SparkPlan =>

/**
* The custom metrics the connector supports for this operation, keyed by name.
*/
def customMetrics: Map[String, SQLMetric]

/**
* Spark-owned metrics that should appear alongside the connector-declared ones. Values under
* these names are owned by Spark and take precedence on a name collision.
*/
protected def sparkMetrics: Map[String, SQLMetric] = Map.empty

override lazy val metrics: Map[String, SQLMetric] = customMetrics ++ sparkMetrics

/**
* Converts an array of connector-declared metrics into the map shape [[customMetrics]] uses.
*/
protected def createCustomMetrics(metrics: Array[CustomMetric]): Map[String, SQLMetric] = {
metrics.map { m =>
m.name -> SQLMetrics.createV2CustomMetric(sparkContext, m)
}.toMap
}

/**
* Applies the values reported by the connector to the declared metrics and posts them so the
* SQL UI reflects the final values. Metrics not declared via [[customMetrics]] are ignored.
* Metrics whose name collides with [[sparkMetrics]] are also ignored so Spark-owned values
* are preserved.
*/
protected def postDriverMetrics(taskMetrics: Array[CustomTaskMetric]): Unit = {
val updated = taskMetrics.flatMap { t =>
if (sparkMetrics.contains(t.name())) {
// Spark metrics take precedence on collisions.
None
} else {
metrics.get(t.name()).map { metric =>
metric.set(t.value())
metric
}
}
}
val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, updated.toImmutableArraySeq)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -20,18 +20,28 @@ package org.apache.spark.sql.execution.datasources.v2
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.connector.catalog.TruncatableTable
import org.apache.spark.sql.execution.metric.SQLMetric

/**
* Physical plan node for table truncation.
*/
case class TruncateTableExec(
table: TruncatableTable,
refreshCache: () => Unit) extends LeafV2CommandExec {
refreshCache: () => Unit)
extends LeafV2CommandExec
with SupportsCustomDriverMetrics {

override lazy val customMetrics: Map[String, SQLMetric] =
createCustomMetrics(table.supportedCustomMetrics())

override def output: Seq[Attribute] = Seq.empty

override protected def run(): Seq[InternalRow] = {
if (table.truncateTable()) refreshCache()
Seq.empty
try {
if (table.truncateTable()) refreshCache()
Seq.empty
} finally {
postDriverMetrics(table.reportDriverMetrics())
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.classic.Dataset
import org.apache.spark.sql.connector.catalog.SupportsWrite
import org.apache.spark.sql.connector.write.V1Write
import org.apache.spark.sql.execution.{SparkPlan, SQLExecution}
import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics}
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.sources.InsertableRelation

/**
Expand Down Expand Up @@ -56,13 +56,15 @@ case class OverwriteByExpressionExecV1(
write: V1Write) extends V1FallbackWriters

/** Some helper interfaces that use V2 write semantics through the V1 writer interface. */
sealed trait V1FallbackWriters extends LeafV2CommandExec with SupportsV1Write {
sealed trait V1FallbackWriters
extends LeafV2CommandExec
with SupportsV1Write
with SupportsCustomDriverMetrics {

override def output: Seq[Attribute] = Nil

override val metrics: Map[String, SQLMetric] =
write.supportedCustomMetrics().map { customMetric =>
customMetric.name() -> SQLMetrics.createV2CustomMetric(sparkContext, customMetric)
}.toMap
override lazy val customMetrics: Map[String, SQLMetric] =
createCustomMetrics(write.supportedCustomMetrics())

def table: SupportsWrite
def refreshCache: () => Unit
Expand All @@ -75,12 +77,7 @@ sealed trait V1FallbackWriters extends LeafV2CommandExec with SupportsV1Write {

Nil
} finally {
write.reportDriverMetrics().foreach { customTaskMetric =>
metrics.get(customTaskMetric.name()).foreach(_.set(customTaskMetric.value()))
}

val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq)
postDriverMetrics(write.reportDriverMetrics())
}
}
}
Expand Down
Loading