Skip to content

Commit

Permalink
Add Delta command operation metrics to SQL metrics (#3328)
Browse files Browse the repository at this point in the history
<!--
Thanks for sending a pull request!  Here are some tips for you:
1. If this is your first time, please read our contributor guidelines:
https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md
2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]
Your PR title ...'.
  3. Be sure to keep the PR description updated to reflect all changes.
  4. Please write your PR title to summarize what this PR proposes.
5. If possible, provide a concise example to reproduce the issue for a
faster review.
6. If applicable, include the corresponding issue number in the PR title
and link it in the body.
-->

#### Which Delta project/connector is this regarding?
<!--
Please add the component selected below to the beginning of the pull
request title
For example: [Spark] Title of my pull request
-->

- [x] Spark
- [ ] Standalone
- [ ] Flink
- [ ] Kernel
- [ ] Other (fill in here)

## Description
Add missing operation metrics to SQL metrics emitted by Delta command
Spark plans.
<!--
- Describe what this PR changes.
- Describe why we need the change.
 
If this PR resolves an issue be sure to include "Resolves #XXX" to
correctly link and close the issue upon merge.
-->

## How was this patch tested?

<!--
If tests were added, say they were added here. Please make sure to test
the changes thoroughly including negative and positive cases if
possible.
If the changes were tested in any way other than unit tests, please
clarify how you tested step by step (ideally copy and paste-able, so
that other reviewers can test and check, and descendants can verify in
the future).
If the changes were not tested, please explain why.
-->
existing tests
## Does this PR introduce _any_ user-facing changes?
no
<!--
If yes, please clarify the previous behavior and the change this PR
proposes - provide the console output, description and/or an example to
show the behavior difference if possible.
If possible, please also clarify if this is a user-facing change
compared to the released Delta Lake versions or within the unreleased
branches such as master.
If no, write 'No'.
-->
  • Loading branch information
sabir-akhadov authored Jul 2, 2024
1 parent f61b12d commit 0a99248
Show file tree
Hide file tree
Showing 4 changed files with 61 additions and 7 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import org.apache.spark.sql.{Dataset, Row, SparkSession}
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.plans.logical.LeafCommand
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.{Clock, SerializableConfiguration}
// scalastyle:on import.ordering.noEmptyLine
Expand Down Expand Up @@ -179,7 +180,8 @@ abstract class CloneTableBase(
txn: OptimisticTransaction,
destinationTable: DeltaLog,
hdpConf: Configuration,
deltaOperation: DeltaOperations.Operation): Seq[Row] = {
deltaOperation: DeltaOperations.Operation,
commandMetrics: Option[Map[String, SQLMetric]]): Seq[Row] = {
val targetFs = targetPath.getFileSystem(hdpConf)
val qualifiedTarget = targetFs.makeQualified(targetPath).toString
val qualifiedSource = {
Expand Down Expand Up @@ -246,6 +248,13 @@ abstract class CloneTableBase(
addedFilesSize)
val commitOpMetrics = getOperationMetricsForDeltaLog(opMetrics)

// Propagate the metrics back to the caller.
commandMetrics.foreach { commandMetrics =>
commitOpMetrics.foreach { kv =>
commandMetrics(kv._1).set(kv._2)
}
}

recordDeltaOperation(
destinationTable, s"delta.${deltaOperation.name.toLowerCase()}.commit") {
txn.commitLarge(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
import org.apache.spark.sql.connector.catalog.Table
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{LongType, StructType}
Expand Down Expand Up @@ -84,7 +85,8 @@ case class CloneTableCommand(
def handleClone(
sparkSession: SparkSession,
txn: OptimisticTransaction,
targetDeltaLog: DeltaLog): Seq[Row] = {
targetDeltaLog: DeltaLog,
commandMetrics: Option[Map[String, SQLMetric]] = None): Seq[Row] = {
if (!targetPath.isAbsolute) {
throw DeltaErrors.cloneOnRelativePath(targetIdent.toString)
}
Expand Down Expand Up @@ -116,7 +118,8 @@ case class CloneTableCommand(
hdpConf = hdpConf,
deltaOperation = Clone(
sourceTable.name, sourceTable.snapshot.map(_.version).getOrElse(-1)
))
),
commandMetrics = commandMetrics)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,13 +33,16 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path}

import org.apache.spark.SparkContext
import org.apache.spark.internal.MDC
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType}
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.connector.catalog.Identifier
import org.apache.spark.sql.execution.command.{LeafRunnableCommand, RunnableCommand}
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType

Expand Down Expand Up @@ -69,6 +72,27 @@ case class CreateDeltaTableCommand(
with DeltaCommand
with DeltaLogging {

@transient
private lazy val sc: SparkContext = SparkContext.getOrCreate()

override lazy val metrics = Map[String, SQLMetric](
"numCopiedFiles" -> createMetric(sc, "number of files copied"),
"copiedFilesSize" -> createMetric(sc, "size of files copied"),
"executionTimeMs" -> createMetric(sc, "time taken to execute the entire operation"),
"numRemovedBytes" -> createMetric(sc, "number of bytes removed"),
"removedFilesSize" -> createMetric(sc, "size of files removed"),
"sourceTableSize" -> createMetric(sc, "size of source table"),
"numOutputRows" -> createMetric(sc, "number of output rows"),
"numParts" -> createMetric(sc, "number of partitions"),
"numFiles" -> createMetric(sc, "number of written files"),
"sourceNumOfFiles" -> createMetric(sc, "number of files in source table"),
"numRemovedFiles" -> createMetric(sc, "number of files removed."),
"numOutputBytes" -> createMetric(sc, "number of output bytes"),
"taskCommitTime" -> createMetric(sc, "task commit time"),
"jobCommitTime" -> createMetric(sc, "job commit time"),
"numOfSyncedTransactions" -> createMetric(sc, "number of synced transactions")
)

override def run(sparkSession: SparkSession): Seq[Row] = {

assert(table.tableType != CatalogTableType.VIEW)
Expand Down Expand Up @@ -109,7 +133,9 @@ case class CreateDeltaTableCommand(
val deltaLog = DeltaLog.forTable(sparkSession, tableLocation)

recordDeltaOperation(deltaLog, "delta.ddl.createTable") {
handleCommit(sparkSession, deltaLog, tableWithLocation)
val result = handleCommit(sparkSession, deltaLog, tableWithLocation)
sendDriverMetrics(sparkSession, metrics)
result
}
}

Expand Down Expand Up @@ -147,7 +173,11 @@ case class CreateDeltaTableCommand(
// CLONE handled separately from other CREATE TABLE syntax
case Some(cmd: CloneTableCommand) =>
checkPathEmpty(txn)
cmd.handleClone(sparkSession, txn, targetDeltaLog = deltaLog)
cmd.handleClone(
sparkSession,
txn,
targetDeltaLog = deltaLog,
commandMetrics = Some(metrics))
case Some(deltaWriter: WriteIntoDeltaLike) =>
checkPathEmpty(txn)
handleCreateTableAsSelect(sparkSession, txn, deltaLog, deltaWriter, tableWithLocation)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
dryRun: Boolean = true,
retentionHours: Option[Double] = None,
inventory: Option[DataFrame] = None,
commandMetrics: Map[String, SQLMetric] = Map.empty,
clock: Clock = new SystemClock): DataFrame = {
recordDeltaOperation(deltaLog, "delta.gc") {

Expand Down Expand Up @@ -370,7 +371,7 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
hadoopConf, parallelDeleteEnabled, parallelDeletePartitions)
} catch {
case t: Throwable =>
logVacuumEnd(deltaLog, spark, path)
logVacuumEnd(deltaLog, spark, path, commandMetrics = commandMetrics)
throw t
}
val timeTakenForDelete = System.currentTimeMillis() - deleteStartTime
Expand All @@ -389,7 +390,13 @@ object VacuumCommand extends VacuumCommandImpl with Serializable {
vacuumEndTime = System.currentTimeMillis,
numPartitionColumns = partitionColumns.size)
recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats)
logVacuumEnd(deltaLog, spark, path, Some(filesDeleted), Some(dirCounts))
logVacuumEnd(
deltaLog,
spark,
path,
commandMetrics = commandMetrics,
Some(filesDeleted),
Some(dirCounts))
logInfo(log"Deleted ${MDC(DeltaLogKeys.NUM_FILES, filesDeleted)} files " +
log"(${MDC(DeltaLogKeys.NUM_BYTES, sizeOfDataToDelete)} bytes) and directories in " +
log"a total of ${MDC(DeltaLogKeys.NUM_DIRS, dirCounts)} directories. " +
Expand Down Expand Up @@ -501,12 +508,17 @@ trait VacuumCommandImpl extends DeltaCommand {
deltaLog: DeltaLog,
spark: SparkSession,
path: Path,
commandMetrics: Map[String, SQLMetric],
filesDeleted: Option[Long] = None,
dirCounts: Option[Long] = None): Unit = {
if (shouldLogVacuum(spark, deltaLog, deltaLog.newDeltaHadoopConf(), path)) {
val txn = deltaLog.startTransaction()
val status = if (filesDeleted.isEmpty && dirCounts.isEmpty) { "FAILED" } else { "COMPLETED" }
if (filesDeleted.nonEmpty && dirCounts.nonEmpty) {
// Populate top level metrics.
commandMetrics.get("numDeletedFiles").foreach(_.set(filesDeleted.get))
commandMetrics.get("numVacuumedDirectories").foreach(_.set(dirCounts.get))
// Additionally, create a separate metrics map in case the commandMetrics is empty.
val metrics = Map[String, SQLMetric](
"numDeletedFiles" -> createMetric(spark.sparkContext, "number of files deleted."),
"numVacuumedDirectories" ->
Expand Down

0 comments on commit 0a99248

Please sign in to comment.