Skip to content

[SPARK-51472] Add gRPC SparkConnectClient actor #7

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 6 commits into from
Closed
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
67 changes: 67 additions & 0 deletions Sources/SparkConnect/Extension.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
//
// 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.
//

import Foundation

extension String {
/// Get a `Plan` instance from a string.
var toSparkConnectPlan: Plan {
var sql = Spark_Connect_SQL()
sql.query = self
var relation = Relation()
relation.sql = sql
var plan = Plan()
plan.opType = Plan.OneOf_OpType.root(relation)
return plan
}

/// Get a `UserContext` instance from a string.
var toUserContext: UserContext {
var context = UserContext()
context.userID = self
context.userName = self
return context
}

/// Get a `KeyValue` instance by using a string as the key.
var toKeyValue: KeyValue {
var keyValue = KeyValue()
keyValue.key = self
return keyValue
}
}

extension [String: String] {
/// Get an array of `KeyValue` from `[String: String]`.
var toSparkConnectKeyValue: [KeyValue] {
var array = [KeyValue]()
for keyValue in self {
var kv = KeyValue()
kv.key = keyValue.key
kv.value = keyValue.value
array.append(kv)
}
return array
}
}

extension Data {
/// Get an `Int32` value from unsafe 4 bytes.
var int32: Int32 { withUnsafeBytes({ $0.load(as: Int32.self) }) }
}
226 changes: 226 additions & 0 deletions Sources/SparkConnect/SparkConnectClient.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,226 @@
//
// 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.

import Foundation
import GRPCCore
import GRPCNIOTransportHTTP2
import GRPCProtobuf
import Synchronization

/// Conceptually the remote spark session that communicates with the server
public actor SparkConnectClient {
let clientType: String = "swift"
let url: URL
let host: String
let port: Int
let userContext: UserContext
var sessionID: String? = nil

/// Create a client to use GRPCClient.
/// - Parameters:
/// - remote: A string to connect `Spark Connect` server.
/// - user: A string for the user ID of this connection.
init(remote: String, user: String) {
self.url = URL(string: remote)!
self.host = url.host() ?? "localhost"
self.port = self.url.port ?? 15002
self.userContext = user.toUserContext
}
Comment on lines +38 to +43
Copy link
Member

Choose a reason for hiding this comment

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

In Scala Configuration, there are a lot configs other than remote and user, e.g., userAgent, isSslEnabled, etc. Are you going to add more configurations later?

Copy link
Member Author

Choose a reason for hiding this comment

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

Of course, this project aims to achieve the feature parity eventually. But, each language clients have different development speeds like we did in SparkR/PySpark/Scala Spark. For Spark Connect example, Spark Connect Go doesn't support isSslEnabled.


/// Stop the connection. Currently, this API is no-op because we don't reuse the connection yet.
func stop() {
}

/// Connect to the `Spark Connect` server with the given session ID string.
/// As a test connection, this sends the server `SparkVersion` request.
/// - Parameter sessionID: A string for the session ID.
/// - Returns: An `AnalyzePlanResponse` instance for `SparkVersion`
func connect(_ sessionID: String) async throws -> AnalyzePlanResponse {
try await withGRPCClient(
transport: .http2NIOPosix(
target: .dns(host: self.host, port: self.port),
transportSecurity: .plaintext
)
) { client in
// To prevent server-side `INVALID_HANDLE.FORMAT (SQLSTATE: HY000)` exception.
if UUID(uuidString: sessionID) == nil {
throw SparkConnectError.InvalidSessionIDException
}

self.sessionID = sessionID
let service = SparkConnectService.Client(wrapping: client)
let version = AnalyzePlanRequest.SparkVersion()
var request = AnalyzePlanRequest()
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
request.analyze = .sparkVersion(version)
let response = try await service.analyzePlan(request)
return response
}
}

/// Create a ``ConfigRequest`` instance for `Set` operation.
/// - Parameter map: A map of key-value string pairs.
/// - Returns: A ``ConfigRequest`` instance.
func getConfigRequestSet(map: [String: String]) -> ConfigRequest {
var request = ConfigRequest()
request.operation = ConfigRequest.Operation()
var set = ConfigRequest.Set()
set.pairs = map.toSparkConnectKeyValue
request.operation.opType = .set(set)
return request
}

/// Request the server to set a map of configurations for this session.
/// - Parameter map: A map of key-value pairs to set.
/// - Returns: Always return true.
func setConf(map: [String: String]) async throws -> Bool {
Copy link
Member

Choose a reason for hiding this comment

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

I wonder why don't return ConfigResponse like Scala but a Bool?

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Mar 12, 2025

Choose a reason for hiding this comment

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

This is a different implementation choice to encapsulate those details.

Technically, each language client maintains its SparkConnectClient in the private scope like private[sql].

try await withGRPCClient(
transport: .http2NIOPosix(
target: .dns(host: self.host, port: self.port),
transportSecurity: .plaintext
)
) { client in
let service = SparkConnectService.Client(wrapping: client)
var request = getConfigRequestSet(map: map)
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
let _ = try await service.config(request)
return true
}
}

/// Create a ``ConfigRequest`` instance for `Get` operation.
/// - Parameter keys: An array of keys to get.
/// - Returns: A `ConfigRequest` instance.
func getConfigRequestGet(keys: [String]) -> ConfigRequest {
var request = ConfigRequest()
request.operation = ConfigRequest.Operation()
var get = ConfigRequest.Get()
get.keys = keys
request.operation.opType = .get(get)
return request
}

/// Request the server to get a value of the given key.
/// - Parameter key: A string for key to look up.
/// - Returns: A string for the value of the key.
func getConf(_ key: String) async throws -> String {
try await withGRPCClient(
transport: .http2NIOPosix(
target: .dns(host: self.host, port: self.port),
transportSecurity: .plaintext
)
) { client in
let service = SparkConnectService.Client(wrapping: client)
var request = getConfigRequestGet(keys: [key])
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
let response = try await service.config(request)
return response.pairs[0].value
}
}

/// Create a ``ConfigRequest`` for `GetAll` operation.
/// - Returns: A `ConfigRequest` instance.
func getConfigRequestGetAll() -> ConfigRequest {
var request = ConfigRequest()
request.operation = ConfigRequest.Operation()
let getAll = ConfigRequest.GetAll()
request.operation.opType = .getAll(getAll)
return request
}

/// Request the server to get all configurations.
/// - Returns: A map of key-value pairs.
func getConfAll() async throws -> [String: String] {
try await withGRPCClient(
transport: .http2NIOPosix(
target: .dns(host: self.host, port: self.port),
transportSecurity: .plaintext
)
) { client in
let service = SparkConnectService.Client(wrapping: client)
var request = getConfigRequestGetAll()
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
let response = try await service.config(request)
var map = [String: String]()
for pair in response.pairs {
map[pair.key] = pair.value
}
return map
}
}

/// Create a `Plan` instance for `Range` relation.
/// - Parameters:
/// - start: A start of the range.
/// - end: A end (exclusive) of the range.
/// - step: A step value for the range from `start` to `end`.
/// - Returns: A `Plan` instance.
func getPlanRange(_ start: Int64, _ end: Int64, _ step: Int64) -> Plan {
var range = Range()
range.start = start
range.end = end
range.step = step
var relation = Relation()
relation.range = range
var plan = Plan()
plan.opType = .root(relation)
return plan
}

/// Create a ``ExecutePlanRequest`` instance with the given plan.
/// The operation ID is created by UUID.
/// - Parameters:
/// - plan: A plan to execute.
/// - Returns: An ``ExecutePlanRequest`` instance.
func getExecutePlanRequest(_ sessionID: String, _ plan: Plan) async
-> ExecutePlanRequest
{
var request = ExecutePlanRequest()
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
request.operationID = UUID().uuidString
request.plan = plan
return request
}

/// Create a ``AnalyzePlanRequest`` instance with the given plan.
/// - Parameters:
/// - plan: A plan to analyze.
/// - Returns: An ``AnalyzePlanRequest`` instance
func getAnalyzePlanRequest(_ sessionID: String, _ plan: Plan) async
Copy link
Member

Choose a reason for hiding this comment

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

getAnalyzePlanRequest and getExecutePlanRequest are not used in this PR, right? Will they be used in later PR?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes, it's going to used in SparkSession and DataFrame in the next PRs.

-> AnalyzePlanRequest
{
var request = AnalyzePlanRequest()
request.clientType = clientType
request.userContext = userContext
request.sessionID = self.sessionID!
var schema = AnalyzePlanRequest.Schema()
schema.plan = plan
request.analyze = .schema(schema)
return request
}
}
1 change: 1 addition & 0 deletions Sources/SparkConnect/SparkConnectError.swift
Original file line number Diff line number Diff line change
Expand Up @@ -20,4 +20,5 @@
/// A enum for ``SparkConnect`` package errors
enum SparkConnectError: Error {
case UnsupportedOperationException
case InvalidSessionIDException
}
29 changes: 29 additions & 0 deletions Sources/SparkConnect/TypeAliases.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
//
// 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.

typealias AnalyzePlanRequest = Spark_Connect_AnalyzePlanRequest
typealias AnalyzePlanResponse = Spark_Connect_AnalyzePlanResponse
typealias ConfigRequest = Spark_Connect_ConfigRequest
typealias DataType = Spark_Connect_DataType
typealias ExecutePlanRequest = Spark_Connect_ExecutePlanRequest
typealias Plan = Spark_Connect_Plan
typealias KeyValue = Spark_Connect_KeyValue
typealias Range = Spark_Connect_Range
typealias Relation = Spark_Connect_Relation
typealias SparkConnectService = Spark_Connect_SparkConnectService
typealias UserContext = Spark_Connect_UserContext
49 changes: 49 additions & 0 deletions Tests/SparkConnectTests/SparkConnectClientTests.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,49 @@
//
// 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.
//

import Foundation
import Testing

@testable import SparkConnect

/// A test suite for `SparkConnectClient`
@Suite(.serialized)
struct SparkConnectClientTests {
@Test
func createAndStop() async throws {
let client = SparkConnectClient(remote: "sc://localhost", user: "test")
await client.stop()
}

@Test
func connectWithInvalidUUID() async throws {
let client = SparkConnectClient(remote: "sc://localhost", user: "test")
try await #require(throws: SparkConnectError.InvalidSessionIDException) {
let _ = try await client.connect("not-a-uuid-format")
}
await client.stop()
}

@Test
func connect() async throws {
let client = SparkConnectClient(remote: "sc://localhost", user: "test")
let _ = try await client.connect(UUID().uuidString)
await client.stop()
}
}
Loading