Skip to content

Commit 6e53880

Browse files
committed
[SPARK-52321] Add SessionClosed/SqlConfNotFound/ParseSyntaxError to SparkConnectError
### What changes were proposed in this pull request? This PR aims to add `SessionClosed`, `SqlConfNotFound`, `ParseSyntaxError ` to `SparkConnectError`. ### Why are the changes needed? To provide a user can catch these exceptions easily instead of matching `internalError` with string patterns. ### Does this PR introduce _any_ user-facing change? Yes, but these are more specific exceptions than before. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #180 from dongjoon-hyun/SPARK-52321. Authored-by: Dongjoon Hyun <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
1 parent b0ba4a0 commit 6e53880

File tree

6 files changed

+14
-5
lines changed

6 files changed

+14
-5
lines changed

Sources/SparkConnect/DataFrame.swift

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -311,6 +311,8 @@ public actor DataFrame: Sendable {
311311
throw SparkConnectError.TableOrViewNotFound
312312
case let m where m.contains("UNRESOLVED_COLUMN.WITH_SUGGESTION"):
313313
throw SparkConnectError.ColumnNotFound
314+
case let m where m.contains("PARSE_SYNTAX_ERROR"):
315+
throw SparkConnectError.ParseSyntaxError
314316
default:
315317
throw error
316318
}

Sources/SparkConnect/SparkConnectClient.swift

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -131,6 +131,10 @@ public actor SparkConnectClient {
131131
return try await f(client)
132132
} catch let error as RPCError where error.code == .internalError {
133133
switch error.message {
134+
case let m where m.contains("INVALID_HANDLE.SESSION_CLOSED"):
135+
throw SparkConnectError.SessionClosed
136+
case let m where m.contains("SQL_CONF_NOT_FOUND"):
137+
throw SparkConnectError.SqlConfNotFound
134138
case let m where m.contains("TABLE_OR_VIEW_ALREADY_EXISTS"):
135139
throw SparkConnectError.TableOrViewAlreadyExists
136140
case let m where m.contains("TABLE_OR_VIEW_NOT_FOUND"):

Sources/SparkConnect/SparkConnectError.swift

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,10 @@ public enum SparkConnectError: Error {
2525
case InvalidSessionID
2626
case InvalidType
2727
case InvalidViewName
28+
case ParseSyntaxError
2829
case SchemaNotFound
30+
case SessionClosed
31+
case SqlConfNotFound
2932
case TableOrViewAlreadyExists
3033
case TableOrViewNotFound
3134
case UnsupportedOperation

Tests/SparkConnectTests/CatalogTests.swift

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -144,7 +144,7 @@ struct CatalogTests {
144144
})
145145
#expect(try await spark.catalog.tableExists(tableName) == false)
146146

147-
try await #require(throws: Error.self) {
147+
try await #require(throws: SparkConnectError.ParseSyntaxError) {
148148
try await spark.catalog.tableExists("invalid table name")
149149
}
150150
await spark.stop()
@@ -190,7 +190,7 @@ struct CatalogTests {
190190
#expect(try await spark.catalog.functionExists("base64"))
191191
#expect(try await spark.catalog.functionExists("non_exist_function") == false)
192192

193-
try await #require(throws: Error.self) {
193+
try await #require(throws: SparkConnectError.ParseSyntaxError) {
194194
try await spark.catalog.functionExists("invalid function name")
195195
}
196196
await spark.stop()

Tests/SparkConnectTests/RuntimeConfTests.swift

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ struct RuntimeConfTests {
3535

3636
#expect(try await !conf.get("spark.app.name").isEmpty)
3737

38-
try await #require(throws: Error.self) {
38+
try await #require(throws: SparkConnectError.SqlConfNotFound) {
3939
try await conf.get("spark.test.non-exist")
4040
}
4141

@@ -86,7 +86,7 @@ struct RuntimeConfTests {
8686
#expect(try await conf.get("spark.test.key1") == "value1")
8787

8888
try await conf.unset("spark.test.key1")
89-
try await #require(throws: Error.self) {
89+
try await #require(throws: SparkConnectError.SqlConfNotFound) {
9090
try await conf.get("spark.test.key1")
9191
}
9292

Tests/SparkConnectTests/SparkSessionTests.swift

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -72,7 +72,7 @@ struct SparkSessionTests {
7272
let sessionID = spark1.sessionID
7373
await spark1.stop()
7474
let remote = ProcessInfo.processInfo.environment["SPARK_REMOTE"] ?? "sc://localhost"
75-
try await #require(throws: Error.self) {
75+
try await #require(throws: SparkConnectError.SessionClosed) {
7676
try await SparkSession.builder.remote("\(remote)/;session_id=\(sessionID)").getOrCreate()
7777
}
7878
}

0 commit comments

Comments
 (0)