Skip to content

Commit

Permalink
[SPARK-42554][CONNECT] Implement GRPC exceptions interception for con…
Browse files Browse the repository at this point in the history
…version

### What changes were proposed in this pull request?

This PR adds GrpcStub and GrpcExceptionConverter utilities to the Spark Connect Scala Client so that  the client can intercept the GRPC Exceptions and convert them to Spark related exceptions. The change converts all GRPC RuntimeStatusException to SparkException as a starting point.

### Why are the changes needed?

Intercept GRPC Exceptions in Spark Connect Scala Client and convert them to Spark related exceptions for making exceptions more compatible with the existing behaviors

### Does this PR introduce _any_ user-facing change?

No

### How was this patch tested?

Existing tests

Closes apache#41743 from heyihong/SPARK-42554.

Authored-by: Yihong He <yihong.he@databricks.com>
Signed-off-by: Hyukjin Kwon <gurwls223@apache.org>
  • Loading branch information
heyihong authored and ragnarok56 committed Mar 2, 2024
1 parent 8d24740 commit e1a525e
Show file tree
Hide file tree
Showing 10 changed files with 130 additions and 21 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.client

import io.grpc.ManagedChannel

import org.apache.spark.connect.proto.{AnalyzePlanRequest, AnalyzePlanResponse, ConfigRequest, ConfigResponse, ExecutePlanRequest, ExecutePlanResponse, InterruptRequest, InterruptResponse}
import org.apache.spark.connect.proto

private[client] class CustomSparkConnectBlockingStub(channel: ManagedChannel) {

private val stub = proto.SparkConnectServiceGrpc.newBlockingStub(channel)

def executePlan(request: ExecutePlanRequest): java.util.Iterator[ExecutePlanResponse] = {
GrpcExceptionConverter.convert {
GrpcExceptionConverter.convertIterator[ExecutePlanResponse](stub.executePlan(request))
}
}

def analyzePlan(request: AnalyzePlanRequest): AnalyzePlanResponse = {
GrpcExceptionConverter.convert {
stub.analyzePlan(request)
}
}

def config(request: ConfigRequest): ConfigResponse = {
GrpcExceptionConverter.convert {
stub.config(request)
}
}

def interrupt(request: InterruptRequest): InterruptResponse = {
GrpcExceptionConverter.convert {
stub.interrupt(request)
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,57 @@
/*
* 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.client

import io.grpc.StatusRuntimeException
import io.grpc.protobuf.StatusProto

import org.apache.spark.{SparkException, SparkThrowable}

private[client] object GrpcExceptionConverter {
def convert[T](f: => T): T = {
try {
f
} catch {
case e: StatusRuntimeException =>
throw toSparkThrowable(e)
}
}

def convertIterator[T](iter: java.util.Iterator[T]): java.util.Iterator[T] = {
new java.util.Iterator[T] {
override def hasNext: Boolean = {
convert {
iter.hasNext
}
}

override def next(): T = {
convert {
iter.next()
}
}
}
}

private def toSparkThrowable(ex: StatusRuntimeException): SparkThrowable with Throwable = {
val status = StatusProto.fromThrowable(ex)
// TODO: Add finer grained error conversion
new SparkException(status.getMessage, ex.getCause)
}
}


Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ private[sql] class SparkConnectClient(

private val userContext: UserContext = configuration.userContext

private[this] val stub = proto.SparkConnectServiceGrpc.newBlockingStub(channel)
private[this] val stub = new CustomSparkConnectBlockingStub(channel)

private[client] def userAgent: String = configuration.userAgent

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ package org.apache.spark.sql

import java.io.{File, FilenameFilter}

import io.grpc.StatusRuntimeException
import org.apache.commons.io.FileUtils

import org.apache.spark.SparkException
import org.apache.spark.sql.connect.client.util.RemoteSparkSession
import org.apache.spark.sql.types.{DoubleType, LongType, StructType}
import org.apache.spark.storage.StorageLevel
Expand All @@ -46,7 +46,7 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper {
assert(databasesWithPattern.length == 0)
val database = spark.catalog.getDatabase(db)
assert(database.name == db)
val message = intercept[StatusRuntimeException] {
val message = intercept[SparkException] {
spark.catalog.getDatabase("notExists")
}.getMessage
assert(message.contains("SCHEMA_NOT_FOUND"))
Expand All @@ -66,7 +66,7 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper {
val catalogs = spark.catalog.listCatalogs().collect()
assert(catalogs.length == 1)
assert(catalogs.map(_.name) sameElements Array("spark_catalog"))
val message = intercept[StatusRuntimeException] {
val message = intercept[SparkException] {
spark.catalog.setCurrentCatalog("notExists")
}.getMessage
assert(message.contains("plugin class not found"))
Expand Down Expand Up @@ -141,7 +141,7 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper {
assert(spark.catalog.listTables().collect().map(_.name).toSet == Set(parquetTableName))
}
}
val message = intercept[StatusRuntimeException] {
val message = intercept[SparkException] {
spark.catalog.getTable(parquetTableName)
}.getMessage
assert(message.contains("TABLE_OR_VIEW_NOT_FOUND"))
Expand Down Expand Up @@ -207,7 +207,7 @@ class CatalogSuite extends RemoteSparkSession with SQLHelper {
assert(spark.catalog.getFunction(absFunctionName).name == absFunctionName)
val notExistsFunction = "notExists"
assert(!spark.catalog.functionExists(notExistsFunction))
val message = intercept[StatusRuntimeException] {
val message = intercept[SparkException] {
spark.catalog.getFunction(notExistsFunction)
}.getMessage
assert(message.contains("UNRESOLVED_ROUTINE"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import java.util.Properties
import scala.collection.JavaConverters._
import scala.collection.mutable

import io.grpc.StatusRuntimeException
import org.apache.commons.io.FileUtils
import org.apache.commons.io.output.TeeOutputStream
import org.apache.commons.lang3.{JavaVersion, SystemUtils}
Expand Down Expand Up @@ -65,7 +64,7 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
assume(IntegrationTestUtils.isSparkHiveJarAvailable)
withTable("test_martin") {
// Fails, because table does not exist.
assertThrows[StatusRuntimeException] {
assertThrows[SparkException] {
spark.sql("select * from test_martin").collect()
}
// Execute eager, DML
Expand Down Expand Up @@ -153,7 +152,7 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
StructField("job", StringType) :: Nil))
.csv(testDataPath.toString)
// Failed because the path cannot be provided both via option and load method (csv).
assertThrows[StatusRuntimeException] {
assertThrows[SparkException] {
df.collect()
}
}
Expand Down Expand Up @@ -237,7 +236,7 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
assert(result.length == 10)
} finally {
// clean up
assertThrows[StatusRuntimeException] {
assertThrows[SparkException] {
spark.read.jdbc(url = s"$url;drop=true", table, new Properties()).collect()
}
}
Expand Down Expand Up @@ -354,7 +353,7 @@ class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with PrivateM
val df = spark.range(10)
val outputFolderPath = Files.createTempDirectory("output").toAbsolutePath
// Failed because the path cannot be provided both via option and save method.
assertThrows[StatusRuntimeException] {
assertThrows[SparkException] {
df.write.option("path", outputFolderPath.toString).save(outputFolderPath.toString)
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ package org.apache.spark.sql

import scala.collection.JavaConverters._

import org.apache.spark.SparkException
import org.apache.spark.sql.connect.client.util.QueryTest
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StringType, StructType}
Expand Down Expand Up @@ -278,7 +279,7 @@ class DataFrameNaFunctionSuite extends QueryTest with SQLHelper {

test("drop with col(*)") {
val df = createDF()
val ex = intercept[RuntimeException] {
val ex = intercept[SparkException] {
df.na.drop("any", Seq("*")).collect()
}
assert(ex.getMessage.contains("UNRESOLVED_COLUMN.WITH_SUGGESTION"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,9 @@ package org.apache.spark.sql

import java.util.Random

import io.grpc.StatusRuntimeException
import org.scalatest.matchers.must.Matchers._

import org.apache.spark.SparkException
import org.apache.spark.sql.connect.client.util.RemoteSparkSession

class DataFrameStatSuite extends RemoteSparkSession {
Expand Down Expand Up @@ -87,7 +87,7 @@ class DataFrameStatSuite extends RemoteSparkSession {

val results = df.stat.cov("singles", "doubles")
assert(math.abs(results - 55.0 / 3) < 1e-12)
intercept[StatusRuntimeException] {
intercept[SparkException] {
df.stat.cov("singles", "letters") // doesn't accept non-numerical dataTypes
}
val decimalData = Seq.tabulate(6)(i => (BigDecimal(i % 3), BigDecimal(i % 2))).toDF("a", "b")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,7 @@ package org.apache.spark.sql
import java.sql.Timestamp
import java.util.Arrays

import io.grpc.StatusRuntimeException

import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Append
import org.apache.spark.sql.connect.client.util.QueryTest
import org.apache.spark.sql.functions._
Expand Down Expand Up @@ -179,7 +178,7 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with SQLHelper {
assert(values == Arrays.asList[String]("0", "8,6,4,2,0", "1", "9,7,5,3,1"))

// Star is not allowed as group sort column
val message = intercept[StatusRuntimeException] {
val message = intercept[SparkException] {
grouped
.flatMapSortedGroups(col("*")) { (g, iter) =>
Iterator(String.valueOf(g), iter.mkString(","))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import scala.util.{Failure, Success}

import org.scalatest.concurrent.Eventually._

import org.apache.spark.SparkException
import org.apache.spark.sql.connect.client.util.RemoteSparkSession
import org.apache.spark.util.ThreadUtils

Expand Down Expand Up @@ -85,11 +86,11 @@ class SparkSessionE2ESuite extends RemoteSparkSession {
}
finished
}
val e1 = intercept[io.grpc.StatusRuntimeException] {
val e1 = intercept[SparkException] {
spark.range(10).map(n => { Thread.sleep(30.seconds.toMillis); n }).collect()
}
assert(e1.getMessage.contains("cancelled"), s"Unexpected exception: $e1")
val e2 = intercept[io.grpc.StatusRuntimeException] {
val e2 = intercept[SparkException] {
spark.range(10).map(n => { Thread.sleep(30.seconds.toMillis); n }).collect()
}
assert(e2.getMessage.contains("cancelled"), s"Unexpected exception: $e2")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,12 @@ import java.util.concurrent.TimeUnit
import scala.collection.JavaConverters._
import scala.collection.mutable

import io.grpc.{Server, StatusRuntimeException}
import io.grpc.Server
import io.grpc.netty.NettyServerBuilder
import io.grpc.stub.StreamObserver
import org.scalatest.BeforeAndAfterEach

import org.apache.spark.SparkException
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{AddArtifactsRequest, AddArtifactsResponse, AnalyzePlanRequest, AnalyzePlanResponse, ArtifactStatusesRequest, ArtifactStatusesResponse, ExecutePlanRequest, ExecutePlanResponse, SparkConnectServiceGrpc}
import org.apache.spark.sql.SparkSession
Expand Down Expand Up @@ -104,7 +105,7 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach {
val request = AnalyzePlanRequest.newBuilder().setSessionId("abc123").build()

// Failed the ssl handshake as the dummy server does not have any server credentials installed.
assertThrows[StatusRuntimeException] {
assertThrows[SparkException] {
client.analyze(request)
}
}
Expand Down

0 comments on commit e1a525e

Please sign in to comment.