Skip to content
Open
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 @@ -16,6 +16,7 @@
*/
package org.apache.spark.util

import java.nio.charset.{CodingErrorAction, StandardCharsets}
import java.util.HexFormat
import java.util.concurrent.atomic.AtomicBoolean

Expand Down Expand Up @@ -58,6 +59,28 @@ private[spark] trait SparkStringUtils {

def abbreviate(str: String, len: Int): String = abbreviate(str, "...", len)

def abbreviateByBytes(str: String, abbrevMarker: String, maxBytes: Int): String = {
if (str == null || abbrevMarker == null) {
null
} else {
val buf = StandardCharsets.UTF_8.encode(str)
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we try to be a bit more careful with allocations. sizeInBytes(..) and this call both encode the string to bytes. At this point you have allocated 3 ByteBuffers. Can we bring it down to 1?

Copy link
Author

Choose a reason for hiding this comment

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

I removed one. To remove both, the implementation was quite convoluted. LMK if this is enough.

if (buf.limit() <= maxBytes) {
str
} else {
// Truncate at the byte budget, then decode with IGNORE so any incomplete
// trailing UTF-8 sequence is silently dropped.
buf.limit(math.min(maxBytes - abbrevMarker.length, buf.limit()))
val decoder = StandardCharsets.UTF_8.newDecoder()
.onMalformedInput(CodingErrorAction.IGNORE)
.onUnmappableCharacter(CodingErrorAction.IGNORE)
decoder.decode(buf).toString + abbrevMarker
}
}
}

def abbreviateByBytes(str: String, maxBytes: Int): String =
abbreviateByBytes(str, "...", maxBytes)

def sideBySide(left: String, right: String): Seq[String] = {
sideBySide(left.split("\n").toImmutableArraySeq, right.split("\n").toImmutableArraySeq)
}
Expand Down Expand Up @@ -87,6 +110,8 @@ private[spark] trait SparkStringUtils {
def rightPad(str: String, width: Int): String =
if (str == null || str.length >= width) str else String.format(s"%-${width}s", str)

def sizeInBytes(str: String): Int = StandardCharsets.UTF_8.encode(str).limit()

def rightPad(str: String, width: Int, s: String): String =
if (str == null || str.length >= width) {
str
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -217,59 +217,96 @@ private[connect] object ErrorUtils extends Logging {
.newBuilder()
.setReason(st.getClass.getName)
.setDomain("org.apache.spark")
.putMetadata(
"classes",
JsonMethods.compact(JsonMethods.render(allClasses(st.getClass).map(_.getName))))

val maxMetadataSize = SparkEnv.get.conf.get(Connect.CONNECT_GRPC_MAX_METADATA_SIZE)

// Tracks the shared metadata budget against the gRPC total-metadata-size limit.
// Keys are ASCII so key.length == sizeInBytes(key); values may be multi-byte.
class BudgetedMetadata(private var remaining: Long) {
// Adds the field if it fits within the remaining budget; returns true if added.
def tryPut(key: String, value: String): Boolean = {
val cost = key.length + Utils.sizeInBytes(value)
if (cost <= remaining) {
errorInfo.putMetadata(key, value)
remaining -= cost
true
} else false
}

// Adds both fields atomically if their combined cost fits; returns true if added.
// avoiding a single method with varargs to not allocate iterator etc
def tryPut(k1: String, v1: String, k2: String, v2: String): Boolean = {
val cost = k1.length + Utils.sizeInBytes(v1) + k2.length + Utils.sizeInBytes(v2)
if (cost <= remaining) {
errorInfo.putMetadata(k1, v1)
errorInfo.putMetadata(k2, v2)
remaining -= cost
true
} else false
}

// Always adds the field, truncating the value to fit within the budget
// (and optionally a caller-supplied cap on value byte size).
def putTruncated(key: String, value: String, maxValueSize: Int = Int.MaxValue): Unit = {
val valueBudget = math.min(maxValueSize, (remaining - key.length).toInt)
if (valueBudget > 0) {
val truncated = Utils.abbreviateByBytes(value, valueBudget)
errorInfo.putMetadata(key, truncated)
remaining -= key.length + Utils.sizeInBytes(truncated)
}
}
}
val metadata = new BudgetedMetadata(maxMetadataSize)

metadata.putTruncated(
"classes",
JsonMethods.compact(JsonMethods.render(allClasses(st.getClass).map(_.getName))))

// moved errorId up to ensure we do not run out of budget, as it is needed for FetchErrorDetails
val enrichErrorEnabled = sessionHolderOpt.exists(
_.session.sessionState.conf.getConf(Connect.CONNECT_ENRICH_ERROR_ENABLED))
if (enrichErrorEnabled) {
// Generate a new unique key for this exception.
val errorId = UUID.randomUUID().toString
metadata.tryPut("errorId", errorId)
sessionHolderOpt.get.errorIdToError.put(errorId, st)
}

// Add the SQL State and Error Class to the response metadata of the ErrorInfoObject.
st match {
case e: SparkThrowable =>
val state = e.getSqlState
if (state != null && state.nonEmpty) {
errorInfo.putMetadata("sqlState", state)
metadata.tryPut("sqlState", state)
}
val errorClass = e.getCondition
if (errorClass != null && errorClass.nonEmpty) {
val messageParameters = JsonMethods.compact(
JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap)))
if (messageParameters.length <= maxMetadataSize) {
errorInfo.putMetadata("errorClass", errorClass)
errorInfo.putMetadata("messageParameters", messageParameters)
if (!metadata.tryPut(
"errorClass", errorClass, "messageParameters", messageParameters)) {
// adds the error class in a separate metadata key, to allow being inspected by
// a proxy, without interfering with the FetchErrorDetails client/server flow
metadata.tryPut("errorClassFallback", errorClass)
}
}
case _ =>
}

val enrichErrorEnabled = sessionHolderOpt.exists(
_.session.sessionState.conf.getConf(Connect.CONNECT_ENRICH_ERROR_ENABLED))
if (enrichErrorEnabled) {
// Generate a new unique key for this exception.
val errorId = UUID.randomUUID().toString

errorInfo.putMetadata("errorId", errorId)

sessionHolderOpt.get.errorIdToError
.put(errorId, st)
}

lazy val stackTrace = Option(Utils.stackTraceToString(st))
val stackTraceEnabled = sessionHolderOpt.exists(
_.session.sessionState.conf.getConf(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED))
val withStackTrace =
if (stackTraceEnabled && stackTrace.nonEmpty) {
val maxSize = Math.min(
SparkEnv.get.conf.get(Connect.CONNECT_JVM_STACK_TRACE_MAX_SIZE),
maxMetadataSize)
errorInfo.putMetadata("stackTrace", Utils.abbreviate(stackTrace.get, maxSize.toInt))
} else {
errorInfo
}
if (stackTraceEnabled && stackTrace.nonEmpty) {
metadata.putTruncated(
"stackTrace",
stackTrace.get,
SparkEnv.get.conf.get(Connect.CONNECT_JVM_STACK_TRACE_MAX_SIZE).toInt)
}

RPCStatus
.newBuilder()
.setCode(RPCCode.INTERNAL_VALUE)
.addDetails(ProtoAny.pack(withStackTrace.build()))
.addDetails(ProtoAny.pack(errorInfo.build()))
.setMessage(SparkConnectService.extractErrorMessage(st))
.build()
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,222 @@
/*
* 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.connect.utils

import scala.jdk.CollectionConverters._

import com.google.rpc.ErrorInfo

import org.apache.spark.SparkException
import org.apache.spark.sql.connect.SparkConnectTestUtils
import org.apache.spark.sql.connect.config.Connect
import org.apache.spark.sql.connect.service.SessionHolder
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.SparkStringUtils

class ErrorUtilsSuite extends SharedSparkSession {

private def makeThrowable(
errorClass: String,
messageParameters: Map[String, String]): SparkException = {
new SparkException(
message = s"Test error for $errorClass",
cause = null,
errorClass = Some(errorClass),
messageParameters = messageParameters)
}

private def buildAndExtractErrorInfo(
st: Throwable,
sessionHolderOpt: Option[SessionHolder] = None): ErrorInfo = {
val status = ErrorUtils.buildStatusFromThrowable(st, sessionHolderOpt)
status.getDetailsList.asScala
.find(_.is(classOf[ErrorInfo]))
.map(_.unpack(classOf[ErrorInfo]))
.getOrElse(fail("No ErrorInfo found in status details"))
}

private def withEnrichedSession[T](f: SessionHolder => T): T = {
val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark)
sessionHolder.session.conf.set(Connect.CONNECT_ENRICH_ERROR_ENABLED.key, "true")
try f(sessionHolder)
finally sessionHolder.session.conf.unset(Connect.CONNECT_ENRICH_ERROR_ENABLED.key)
}

test("buildStatusFromThrowable includes errorClass and messageParameters when small enough") {
withEnrichedSession { sessionHolder =>
val throwable = makeThrowable("TEST_ERROR", Map("key" -> "value"))
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(info.getMetadataMap.get("errorClass") === "TEST_ERROR")
assert(info.getMetadataMap.containsKey("messageParameters"))
assert(!info.getMetadataMap.containsKey("errorClassFallback"))
}
}

test("buildStatusFromThrowable uses errorClassFallback when messageParameters exceed limit") {
// 50 small parameters whose combined JSON barely exceeds the 1024-byte default limit,
// even though no single parameter is large.
withEnrichedSession { sessionHolder =>
val params = (1 to 50).map(i => f"key$i%03d" -> ("v" * 10)).toMap
val throwable = makeThrowable("TEST_ERROR", params)
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(!info.getMetadataMap.containsKey("errorClass"))
assert(!info.getMetadataMap.containsKey("messageParameters"))
assert(info.getMetadataMap.get("errorClassFallback") === "TEST_ERROR")
}
}

test("buildStatusFromThrowable does not include errorId when no session holder is provided") {
val throwable = makeThrowable("TEST_ERROR", Map.empty)
val info = buildAndExtractErrorInfo(throwable)
assert(!info.getMetadataMap.containsKey("errorId"))
}

test("buildStatusFromThrowable includes errorId and stores throwable when enrichError enabled") {
withEnrichedSession { sessionHolder =>
val throwable = makeThrowable("TEST_ERROR", Map.empty)
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
val errorId = info.getMetadataMap.get("errorId")
assert(errorId !== null)
assert(sessionHolder.errorIdToError.getIfPresent(errorId) === throwable)
}
}

test("buildStatusFromThrowable does not include errorId when enrichError is disabled") {
val sessionHolder = SparkConnectTestUtils.createDummySessionHolder(spark)
sessionHolder.session.conf.set(Connect.CONNECT_ENRICH_ERROR_ENABLED.key, "false")
try {
val throwable = makeThrowable("TEST_ERROR", Map.empty)
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(!info.getMetadataMap.containsKey("errorId"))
assert(sessionHolder.errorIdToError.size() === 0)
} finally {
sessionHolder.session.conf.unset(Connect.CONNECT_ENRICH_ERROR_ENABLED.key)
}
}

test("buildStatusFromThrowable does not include stackTrace when stackTrace is disabled") {
withEnrichedSession { sessionHolder =>
sessionHolder.session.conf.set(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key, "false")
try {
val throwable = makeThrowable("TEST_ERROR", Map.empty)
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(!info.getMetadataMap.containsKey("stackTrace"))
} finally {
sessionHolder.session.conf.unset(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key)
}
}
}

test("buildStatusFromThrowable includes stackTrace when stackTrace is enabled") {
withEnrichedSession { sessionHolder =>
sessionHolder.session.conf.set(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key, "true")
try {
val throwable = makeThrowable("TEST_ERROR", Map.empty)
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(info.getMetadataMap.containsKey("stackTrace"))
assert(info.getMetadataMap.get("stackTrace").nonEmpty)
} finally {
sessionHolder.session.conf.unset(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key)
}
}
}

// U+4E2D: a CJK character that encodes to 3 UTF-8 bytes but counts as 1 Java char,
// producing different results under byte-based vs length-based size checks.
private val cjk = 0x4e2d.toChar.toString

test("abbreviateByBytes drops incomplete trailing multi-byte sequence") {
// budget=7 cuts after 1 byte of the 3rd CJK char (3 bytes each), leaving an incomplete
// sequence that CodingErrorAction.IGNORE must drop. Without IGNORE the result would
// contain a replacement character (U+FFFD) and fail the equality check.
assert(SparkStringUtils.abbreviateByBytes(cjk * 10, "...", 10) === cjk + cjk + "...")
}

test("messageParameters with multi-byte chars exceeding byte limit are excluded") {
// 340 CJK chars: JSON ~1030 bytes (> 1024 limit) but only ~350 chars (< 1024).
withEnrichedSession { sessionHolder =>
val throwable = makeThrowable("TEST_ERROR", Map("key" -> (cjk * 340)))
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(!info.getMetadataMap.containsKey("errorClass"))
assert(!info.getMetadataMap.containsKey("messageParameters"))
assert(info.getMetadataMap.get("errorClassFallback") === "TEST_ERROR")
}
}

test("messageParameters with multi-byte chars within byte limit are included") {
// 100 CJK chars: JSON ~310 bytes, well within the shared budget after prior fields.
withEnrichedSession { sessionHolder =>
val throwable = makeThrowable("TEST_ERROR", Map("key" -> (cjk * 100)))
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(info.getMetadataMap.containsKey("errorClass"))
assert(info.getMetadataMap.containsKey("messageParameters"))
}
}

test("earlier fields consume shared budget pushing errorClass to fallback") {
// Budget of 180 bytes: classes (~105) + errorId (43) ≈ 148, leaving ~32 bytes —
// not enough for errorClass (20) + messageParameters (32) = 52, so fallback is used.
spark.sparkContext.conf.set(Connect.CONNECT_GRPC_MAX_METADATA_SIZE.key, "180")
try {
withEnrichedSession { sessionHolder =>
val throwable = makeThrowable("TEST_ERROR", Map("key" -> "value"))
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
assert(!info.getMetadataMap.containsKey("errorClass"))
assert(!info.getMetadataMap.containsKey("messageParameters"))
assert(info.getMetadataMap.get("errorClassFallback") === "TEST_ERROR")
}
} finally {
spark.sparkContext.conf.remove(Connect.CONNECT_GRPC_MAX_METADATA_SIZE.key)
}
}

test("buildStatusFromThrowable truncates stackTrace to byte size limit") {
// Exception message with 400 CJK chars (~1200 bytes) ensures truncation falls within
// the multi-byte content. The result must end with a complete CJK char + "...", proving
// the cut respected byte boundaries. A length-based cut would retain ~3x more bytes.
// The budget is shared across all fields, so the stackTrace limit is the remaining budget
// after prior fields are written, capped by CONNECT_JVM_STACK_TRACE_MAX_SIZE.
val throwable = new SparkException(
message = cjk * 400,
cause = null,
errorClass = Some("TEST_ERROR"),
messageParameters = Map.empty)
withEnrichedSession { sessionHolder =>
sessionHolder.session.conf.set(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key, "true")
try {
val info = buildAndExtractErrorInfo(throwable, Some(sessionHolder))
assert(info.getMetadataMap.containsKey("errorId"))
val maxMetadataSize =
spark.sparkContext.conf.getLong(Connect.CONNECT_GRPC_MAX_METADATA_SIZE.key, 1024L)
val stackTrace = info.getMetadataMap.get("stackTrace")
assert(stackTrace != null && stackTrace.nonEmpty)
assert(SparkStringUtils.sizeInBytes(stackTrace) <= maxMetadataSize)
assert(stackTrace.endsWith(cjk + "..."))
} finally {
sessionHolder.session.conf.unset(SQLConf.PYSPARK_JVM_STACKTRACE_ENABLED.key)
}
}
}
}