Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
20 commits
Select commit Hold shift + click to select a range
aece5d8
feat(connect): add sqlID/jobID reverse indexes on AppBase
sayedbilalbari Apr 21, 2026
20e7046
feat(connect): populate operationIdToSqlIds from SQLExecutionStart.jo…
sayedbilalbari Apr 21, 2026
8c855dc
feat(connect): populate operationIdToJobIds from JobStart spark.job.tags
sayedbilalbari Apr 21, 2026
1ef9ae0
feat(connect): add ConnectSession/ConnectOperation profile results
sayedbilalbari Apr 21, 2026
4a79192
feat(connect): emit connect_sessions.csv and connect_operations.csv f…
sayedbilalbari Apr 21, 2026
227c206
feat(connect): write statementText sidecar files per Connect operation
sayedbilalbari Apr 21, 2026
a68c9ce
feat(connect): emit connect CSVs and statement sidecars from qualific…
sayedbilalbari Apr 22, 2026
6457e39
feat(connect): register connectReport in qual and prof YAML catalogs
sayedbilalbari Apr 22, 2026
4e96bdc
feat(connect): Python API helpers for listing and reading statement f…
sayedbilalbari Apr 22, 2026
244252e
test(connect): add profiler-to-python roundtrip golden fixture
sayedbilalbari Apr 22, 2026
afea0dd
test(connect): assert golden connect CSV header order
sayedbilalbari Apr 22, 2026
225ba74
chore(connect): fix Scala style issues from verification
sayedbilalbari Apr 22, 2026
f0fc5b4
chore: refresh expired license headers
sayedbilalbari Apr 22, 2026
b2746b5
fix(connect): sanitize sidecar paths and keep session-only logs
sayedbilalbari Apr 22, 2026
1ad999a
fix(connect): tighten sidecar handling and trim schema
sayedbilalbari Apr 22, 2026
ff0ffda
test(connect): clarify truncation marker semantics
sayedbilalbari Apr 22, 2026
ed5a4cf
fix(connect): address greptile review feedback
sayedbilalbari Apr 22, 2026
623b464
fix(connect): scalastyle, scaladoc, and audit cleanups
sayedbilalbari Apr 22, 2026
9ed68bc
Merge nv/dev into sbari-issue-2065
sayedbilalbari Apr 29, 2026
04de3cc
fix: add Spark Connect runtime dependencies
sayedbilalbari May 1, 2026
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
121 changes: 121 additions & 0 deletions core/src/main/resources/configs/reports/connectReport.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1,121 @@
# Copyright (c) 2026, NVIDIA CORPORATION.
#
# Licensed 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.

# Spark Connect session / operation report written per application by both the
# profiling and qualification tools.

reportDefinitions:
- reportId: connectReport
description: >-
Per-application Spark Connect session and operation metadata plus a
directory of per-operation protobuf-text statement payloads.
scope: per-app
tableDefinitions:
- label: connectSessions
description: >-
One row per Spark Connect session observed in the event log.
fileName: connect_sessions.csv
scope: per-app
columns:
- name: appID
dataType: String
description: Application ID.
- name: sessionId
dataType: String
description: UUID of the Spark Connect session.
- name: userId
dataType: String
description: User who owns the session.
- name: startTime
dataType: Long
description: Epoch millis when the session started.
- name: endTime
dataType: Long
description: Epoch millis when the session closed, if observed.
- name: durationMs
dataType: Long
description: Session duration in milliseconds, or -1 if still open.
- name: operationCount
dataType: Long
description: Number of operations observed in the session.
- label: connectOperations
description: >-
One row per Spark Connect operation with core lifecycle timestamps,
linked SQL/job identifiers, and statement sidecar metadata.
fileName: connect_operations.csv
scope: per-app
columns:
- name: appID
dataType: String
description: Application ID.
- name: operationId
dataType: String
description: UUID of the Spark Connect operation.
- name: sessionId
dataType: String
description: Session UUID that owns the operation.
- name: userId
dataType: String
description: User who submitted the operation.
- name: jobTag
dataType: String
description: Correlation tag shared with SQLExecutionStart and JobStart events.
- name: startTime
dataType: Long
description: Epoch millis when the operation started.
- name: finishTime
dataType: Long
description: Epoch millis when execution completed, if observed.
- name: closeTime
dataType: Long
description: Epoch millis when the operation closed, if observed.
- name: failTime
dataType: Long
description: Epoch millis when the operation failed, if observed.
- name: cancelTime
dataType: Long
description: Epoch millis when the operation was canceled, if observed.
- name: durationMs
dataType: Long
description: Derived end-to-end operation duration in milliseconds.
- name: status
dataType: String
description: Derived operation status (RUNNING, SUCCEEDED, FAILED, CANCELED).
- name: errorMessage
dataType: String
description: Error message reported by a failure event, if present.
- name: sqlIds
dataType: String
description: Semicolon-separated SQL execution IDs correlated to the operation.
- name: jobIds
dataType: String
description: Semicolon-separated Spark job IDs correlated to the operation.
- name: statementFile
dataType: String
description: Sidecar filename under connect_statements/ for this operation, if written.
- name: statementTruncated
dataType: Boolean
description: >-
True when statementText includes Spark's textual truncation
marker. This does not detect Spark 4.x depth-based subtree
elision, which collapses nested structures to {} without a
marker.
- label: connectStatements
description: >-
Directory of per-operation statementText sidecars. Each
<operationId>.txt file contains the protobuf debug-format text of the
Copy link
Copy Markdown
Collaborator

Choose a reason for hiding this comment

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

Are there any industry standard format file extension for storing TextProto files?

Copy link
Copy Markdown
Collaborator Author

Choose a reason for hiding this comment

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

Very valid point. The canonical protobuf TextFormat is .txtpb which means this is a valid proto output. But in our case the statementText is a diagnostic file that can be truncated. So better to have .txt file. Does not imply a complete protobug TextFormat file

Connect operation statement when Connect statement sidecars are enabled.
fileName: connect_statements
fileFormat: DIRECTORY
scope: per-app
3 changes: 2 additions & 1 deletion core/src/main/resources/configs/reports/profCoreReport.yaml
Original file line number Diff line number Diff line change
@@ -1,4 +1,4 @@
# Copyright (c) 2025, NVIDIA CORPORATION.
# Copyright (c) 2025-2026, NVIDIA CORPORATION.
#
# Licensed under the Apache License, Version 2.0 (the "License");
# you may not use this file except in compliance with the License.
Expand Down Expand Up @@ -32,6 +32,7 @@ reportDefinitions:
scope: global
nestedReports:
- reportId: coreRawMetrics
- reportId: connectReport
tableDefinitions:
# AppStatusResult
- label: coreCSVStatus
Expand Down
2 changes: 2 additions & 0 deletions core/src/main/resources/configs/reports/qualCoreReport.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,8 @@ reportDefinitions:
relativePath: qual_metrics
- reportId: coreRawMetrics
relativePath: raw_metrics
- reportId: connectReport
relativePath: raw_metrics
- reportId: qualTuningApps
relativePath: tuning_apps
tableDefinitions:
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,199 @@
/*
* Copyright (c) 2026, NVIDIA CORPORATION.
*
* Licensed 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 com.nvidia.spark.rapids.tool.profiling

import com.nvidia.spark.rapids.tool.views.OutHeaderRegistry

import org.apache.spark.sql.rapids.tool.util.StringUtils

/**
* CSV row for a Spark Connect session. Serializes the lifecycle metadata for a
* single session into the columns registered under
* `ConnectSessionProfileResult` in [[com.nvidia.spark.rapids.tool.views.OutHeaderRegistry]].
*
* `durationMs` is `endTime - startTime` when `endTime` is defined, else `-1`
* (matches the convention used for open/unfinished sessions in other result
* classes).
*/
case class ConnectSessionProfileResult(
appId: String,
sessionId: String,
userId: String,
startTime: Long,
endTime: Option[Long],
operationCount: Long) extends ProfileResult {

override def outputHeaders: Array[String] = {
OutHeaderRegistry.outputHeaders("ConnectSessionProfileResult")
}

override def convertToSeq(): Array[String] = {
Array(
appId,
sessionId,
userId,
startTime.toString,
endTime.map(_.toString).orNull,
endTime.map(e => (e - startTime).toString).getOrElse("-1"),
operationCount.toString)
}

override def convertToCSVSeq(): Array[String] = {
Array(
StringUtils.reformatCSVString(appId),
StringUtils.reformatCSVString(sessionId),
StringUtils.reformatCSVString(userId),
startTime.toString,
endTime.map(_.toString).orNull,
endTime.map(e => (e - startTime).toString).getOrElse("-1"),
operationCount.toString)
}
}

/**
* CSV row for a single Spark Connect operation. Captures the core lifecycle
* (start/finish/close/fail/cancel timestamps), derived status, error message,
* and the joined sqlIDs/jobIDs. Also records the sidecar basename for the
* `connect_statements/<operationId>.txt` artifact.
*
* sqlIds and jobIds are serialized semicolon-separated to keep the CSV
* single-column and avoid quoting issues.
*/
case class ConnectOperationProfileResult(
appId: String,
operationId: String,
sessionId: String,
userId: String,
jobTag: String,
startTime: Long,
finishTime: Option[Long],
closeTime: Option[Long],
failTime: Option[Long],
cancelTime: Option[Long],
durationMs: Long,
status: String,
errorMessage: Option[String],
sqlIds: Seq[Long],
jobIds: Seq[Int],
statementFile: Option[String],
statementTruncated: Boolean) extends ProfileResult {

override def outputHeaders: Array[String] = {
OutHeaderRegistry.outputHeaders("ConnectOperationProfileResult")
}

override def convertToSeq(): Array[String] = {
Array(
appId,
operationId,
sessionId,
userId,
jobTag,
startTime.toString,
finishTime.map(_.toString).orNull,
closeTime.map(_.toString).orNull,
failTime.map(_.toString).orNull,
cancelTime.map(_.toString).orNull,
durationMs.toString,
status,
errorMessage.getOrElse(""),
sqlIds.mkString(";"),
jobIds.mkString(";"),
statementFile.getOrElse(""),
statementTruncated.toString)
}

override def convertToCSVSeq(): Array[String] = {
Array(
StringUtils.reformatCSVString(appId),
StringUtils.reformatCSVString(operationId),
StringUtils.reformatCSVString(sessionId),
StringUtils.reformatCSVString(userId),
StringUtils.reformatCSVString(jobTag),
startTime.toString,
finishTime.map(_.toString).orNull,
closeTime.map(_.toString).orNull,
failTime.map(_.toString).orNull,
cancelTime.map(_.toString).orNull,
durationMs.toString,
StringUtils.reformatCSVString(status),
StringUtils.reformatCSVString(errorMessage.getOrElse("")),
StringUtils.reformatCSVString(sqlIds.mkString(";")),
StringUtils.reformatCSVString(jobIds.mkString(";")),
StringUtils.reformatCSVString(statementFile.getOrElse("")),
statementTruncated.toString)
}
}

object ConnectOperationProfileResult {

/**
* Marker substring embedded by the Spark Connect server-side abbreviator when
* a statement/plan text exceeds its configured limit. Presence of this marker
* in `statementText` indicates the artifact we persist is a truncated
* representation of the original plan.
*
* Note: Spark 4.x also performs depth-based structural elision by collapsing
* subtrees to `{}` once the protobuf text formatter exceeds its nesting cap.
* Those cases do not emit a textual marker, so `statementTruncated` is
* intentionally limited to marker-based truncation only.
*/
private[profiling] val TruncationMarker: String = "[truncated(size="

/**
* Derives operation status from the observed lifecycle timestamps.
* Priority: CANCELED -> FAILED -> SUCCEEDED -> RUNNING.
* CANCELED precedes FAILED because server-side cancellation sometimes
* surfaces a trailing failure event we should not misattribute.
*/
private def deriveStatus(op: ConnectOperationInfo): String = {
if (op.cancelTime.isDefined) "CANCELED"
else if (op.failTime.isDefined) "FAILED"
else if (op.finishTime.isDefined || op.closeTime.isDefined) "SUCCEEDED"
else "RUNNING"
}

def from(
appId: String,
op: ConnectOperationInfo,
sqlIds: Seq[Long],
jobIds: Seq[Int],
Comment thread
sayedbilalbari marked this conversation as resolved.
statementFile: Option[String]): ConnectOperationProfileResult = {
val endForDuration =
op.closeTime.orElse(op.finishTime).orElse(op.failTime).orElse(op.cancelTime)
val durationMs = endForDuration.map(_ - op.startTime).getOrElse(-1L)
val statementTruncated = op.statementText.contains(TruncationMarker)
ConnectOperationProfileResult(
appId = appId,
operationId = op.operationId,
sessionId = op.sessionId,
userId = op.userId,
jobTag = op.jobTag,
startTime = op.startTime,
finishTime = op.finishTime,
closeTime = op.closeTime,
failTime = op.failTime,
cancelTime = op.cancelTime,
durationMs = durationMs,
status = deriveStatus(op),
errorMessage = op.errorMessage,
sqlIds = sqlIds,
jobIds = jobIds,
statementFile = statementFile,
statementTruncated = statementTruncated)
}
}
Loading