├── Tests └── SparkConnectTests │ ├── Resources │ └── queries │ │ ├── select.sql │ │ ├── clear_cache.sql │ │ ├── show_tables.sql │ │ ├── array.sql │ │ ├── binary.sql │ │ ├── string.sql │ │ ├── cache.sql.answer │ │ ├── date.sql │ │ ├── show_databases.sql │ │ ├── uncache.sql.answer │ │ ├── clear_cache.sql.answer │ │ ├── map.sql │ │ ├── uncache.sql │ │ ├── describe_database.sql │ │ ├── describe_function.sql │ │ ├── describe_table.sql │ │ ├── posexplode.sql │ │ ├── create_scala_function.sql.answer │ │ ├── create_table_function.sql.answer │ │ ├── drop_scala_function.sql │ │ ├── drop_scala_function.sql.answer │ │ ├── drop_table_function.sql │ │ ├── drop_table_function.sql.answer │ │ ├── struct.sql │ │ ├── select.sql.answer │ │ ├── timestamp.sql │ │ ├── cast.sql │ │ ├── function.sql │ │ ├── pipesyntax.sql │ │ ├── cache.sql │ │ ├── string.sql.answer │ │ ├── explain.sql │ │ ├── binary.sql.answer │ │ ├── describe_query.sql │ │ ├── posexplode.sql.answer │ │ ├── show_databases.sql.answer │ │ ├── time.sql │ │ ├── create_scala_function.sql │ │ ├── array.sql.answer │ │ ├── pipesyntax.sql.answer │ │ ├── create_table_function.sql │ │ ├── decimal.sql │ │ ├── floating.sql │ │ ├── date.sql.answer │ │ ├── integral.sql │ │ ├── map.sql.answer │ │ ├── decimal.sql.answer │ │ ├── describe_table.sql.answer │ │ ├── struct.sql.answer │ │ ├── show_tables.sql.answer │ │ ├── timestamp.sql.answer │ │ ├── describe_query.sql.answer │ │ ├── variant.sql │ │ ├── floating.sql.answer │ │ ├── variant.sql.answer │ │ ├── function.sql.answer │ │ ├── cast.sql.answer │ │ ├── time.sql.answer │ │ ├── integral.sql.answer │ │ ├── describe_database.sql.answer │ │ ├── describe_function.sql.answer │ │ └── explain.sql.answer │ ├── CRC32Tests.swift │ ├── StreamingQueryTests.swift │ ├── BuilderTests.swift │ ├── DataStreamTests.swift │ ├── SparkFileUtilsTests.swift │ ├── CaseInsensitiveDictionaryTests.swift │ ├── DataFrameInternalTests.swift │ ├── SQLHelper.swift │ ├── RowTests.swift │ ├── ConstraintTests.swift │ ├── StreamingQueryManagerTests.swift │ ├── DataFrameWriterV2Tests.swift │ ├── MergeIntoWriterTests.swift │ ├── RuntimeConfTests.swift │ ├── SQLTests.swift │ ├── DataFrameReaderTests.swift │ └── DataFrameWriterTests.swift ├── NOTICE ├── .github ├── .licenserc.yaml ├── workflows │ └── publish_image.yml └── PULL_REQUEST_TEMPLATE ├── .markdownlint.yaml ├── .markdownlintignore ├── Sources └── SparkConnect │ ├── Documentation.docc │ ├── SparkConnect.md │ ├── SparkSession.md │ ├── Info.plist │ ├── GettingStarted.md │ └── Examples.md │ ├── MemoryAllocator.swift │ ├── ErrorUtils.swift │ ├── SparkConnectError.swift │ ├── BearerTokenInterceptor.swift │ ├── GroupedData.swift │ ├── ProtoUtils.swift │ ├── BitUtility.swift │ ├── FlightDescriptor.swift │ ├── FlightData.swift │ ├── ArrowSchema.swift │ ├── CaseInsensitiveDictionary.swift │ ├── ArrowData.swift │ ├── Row.swift │ ├── CRC32.swift │ ├── ArrowBuffer.swift │ ├── TypeAliases.swift │ ├── SparkFileUtils.swift │ ├── ProtoUtil.swift │ ├── RuntimeConf.swift │ ├── ChunkedArray.swift │ ├── StorageLevel.swift │ └── DataFrameWriterV2.swift ├── .spi.yml ├── Examples ├── pi │ ├── README.md │ ├── Sources │ │ └── main.swift │ ├── Package.swift │ └── Dockerfile ├── web │ ├── Sources │ │ └── SparkConnectSwiftWeb │ │ │ ├── configure.swift │ │ │ ├── routes.swift │ │ │ └── entrypoint.swift │ ├── Dockerfile │ ├── Package.swift │ └── README.md ├── app │ ├── Package.swift │ ├── Sources │ │ └── main.swift │ ├── README.md │ └── Dockerfile ├── spark-sql │ ├── Package.swift │ ├── Dockerfile │ ├── Sources │ │ └── main.swift │ └── README.md └── stream │ ├── Package.swift │ ├── Sources │ └── main.swift │ ├── Dockerfile │ └── README.md ├── .asf.yaml ├── .gitignore ├── Package.swift └── README.md /Tests/SparkConnectTests/Resources/queries/select.sql: -------------------------------------------------------------------------------- 1 | SELECT 1 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/clear_cache.sql: -------------------------------------------------------------------------------- 1 | CLEAR CACHE 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/show_tables.sql: -------------------------------------------------------------------------------- 1 | SHOW TABLES 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/array.sql: -------------------------------------------------------------------------------- 1 | SELECT array(1, 2, 3) 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/binary.sql: -------------------------------------------------------------------------------- 1 | SELECT binary('abc') 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/string.sql: -------------------------------------------------------------------------------- 1 | SELECT 'abc', 'def' 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/cache.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/date.sql: -------------------------------------------------------------------------------- 1 | SELECT DATE '2025-03-15' 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/show_databases.sql: -------------------------------------------------------------------------------- 1 | SHOW DATABASES 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/uncache.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/clear_cache.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/map.sql: -------------------------------------------------------------------------------- 1 | SELECT map(1.0, '2', 3.0, '4') 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/uncache.sql: -------------------------------------------------------------------------------- 1 | UNCACHE TABLE IF EXISTS t1 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_database.sql: -------------------------------------------------------------------------------- 1 | DESCRIBE DATABASE default 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_function.sql: -------------------------------------------------------------------------------- 1 | DESCRIBE FUNCTION abs 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_table.sql: -------------------------------------------------------------------------------- 1 | DESCRIBE TABLE testcache 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/posexplode.sql: -------------------------------------------------------------------------------- 1 | SELECT posexplode(array(10,20)) 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/create_scala_function.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/create_table_function.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/drop_scala_function.sql: -------------------------------------------------------------------------------- 1 | DROP FUNCTION IF EXISTS bar 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/drop_scala_function.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/drop_table_function.sql: -------------------------------------------------------------------------------- 1 | DROP FUNCTION IF EXISTS foo 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/drop_table_function.sql.answer: -------------------------------------------------------------------------------- 1 | ++ 2 | || 3 | ++ 4 | ++ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/struct.sql: -------------------------------------------------------------------------------- 1 | SELECT struct(1), struct(2, struct(3)) 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/select.sql.answer: -------------------------------------------------------------------------------- 1 | +---+ 2 | | 1| 3 | +---+ 4 | | 1| 5 | +---+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/timestamp.sql: -------------------------------------------------------------------------------- 1 | SELECT TIMESTAMP '2025-05-01 16:23:40.945304' 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/cast.sql: -------------------------------------------------------------------------------- 1 | SELECT 'true'::boolean, 1::string, '1'::int, '3.14'::float 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/function.sql: -------------------------------------------------------------------------------- 1 | SELECT length(java_method('java.util.UUID', 'randomUUID')) 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/pipesyntax.sql: -------------------------------------------------------------------------------- 1 | VALUES (0), (1) tab(col) 2 | |> EXTEND col * 2 AS result 3 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/cache.sql: -------------------------------------------------------------------------------- 1 | CACHE TABLE testCache OPTIONS ('storageLevel' 'DISK_ONLY') SELECT 1 col; 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/string.sql.answer: -------------------------------------------------------------------------------- 1 | +---+---+ 2 | |abc|def| 3 | +---+---+ 4 | |abc|def| 5 | +---+---+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/explain.sql: -------------------------------------------------------------------------------- 1 | EXPLAIN EXTENDED select k, sum(v) from values (1, 2), (1, 3) t(k, v) group by k 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/binary.sql.answer: -------------------------------------------------------------------------------- 1 | +----------+ 2 | | abc| 3 | +----------+ 4 | |[61 62 63]| 5 | +----------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_query.sql: -------------------------------------------------------------------------------- 1 | DESCRIBE QUERY VALUES(100, 'John', 10000.20D) AS employee(id, name, salary); 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/posexplode.sql.answer: -------------------------------------------------------------------------------- 1 | +---+---+ 2 | |pos|col| 3 | +---+---+ 4 | | 0| 10| 5 | | 1| 20| 6 | +---+---+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/show_databases.sql.answer: -------------------------------------------------------------------------------- 1 | +---------+ 2 | |namespace| 3 | +---------+ 4 | | default| 5 | +---------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/time.sql: -------------------------------------------------------------------------------- 1 | SELECT TIME '12:34:56.123456', to_time('12:34:56.123456'), make_time(12, 34, 56.123456) 2 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/create_scala_function.sql: -------------------------------------------------------------------------------- 1 | CREATE FUNCTION IF NOT EXISTS bar(x INT) 2 | RETURNS INT 3 | RETURN x + 1 4 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/array.sql.answer: -------------------------------------------------------------------------------- 1 | +--------------+ 2 | |array(1, 2, 3)| 3 | +--------------+ 4 | | [1, 2, 3]| 5 | +--------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/pipesyntax.sql.answer: -------------------------------------------------------------------------------- 1 | +---+------+ 2 | |col|result| 3 | +---+------+ 4 | | 0| 0| 5 | | 1| 2| 6 | +---+------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/create_table_function.sql: -------------------------------------------------------------------------------- 1 | CREATE FUNCTION IF NOT EXISTS foo(x INT) 2 | RETURNS TABLE(a INT) 3 | RETURN SELECT x + 1 AS x1 4 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/decimal.sql: -------------------------------------------------------------------------------- 1 | SELECT * FROM VALUES 2 | (1.0, 3.4, NULL::decimal, 0::decimal), 3 | (2.0, 34.56, 0::decimal, NULL::decimal) 4 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/floating.sql: -------------------------------------------------------------------------------- 1 | SELECT 2 | 1.0F, -2.0F, 3 | 3.0D, -4.0D, 4 | float('inf'), float('NaN'), 5 | double('inf'), double('NaN') 6 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/date.sql.answer: -------------------------------------------------------------------------------- 1 | +-----------------+ 2 | |DATE '2025-03-15'| 3 | +-----------------+ 4 | | 2025-03-15| 5 | +-----------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/integral.sql: -------------------------------------------------------------------------------- 1 | SELECT 2 | 127Y, -128Y, 3 | 32767S, -32768S, 4 | 2147483647, -2147483648, 5 | 9223372036854775807L, -9223372036854775808L 6 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/map.sql.answer: -------------------------------------------------------------------------------- 1 | +--------------------+ 2 | | map(1.0, 2, 3.0, 4)| 3 | +--------------------+ 4 | |{1.0 -> 2, 3.0 -> 4}| 5 | +--------------------+ -------------------------------------------------------------------------------- /NOTICE: -------------------------------------------------------------------------------- 1 | Apache Spark 2 | Copyright 2014 and onwards The Apache Software Foundation. 3 | 4 | This product includes software developed at 5 | The Apache Software Foundation (http://www.apache.org/). 6 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/decimal.sql.answer: -------------------------------------------------------------------------------- 1 | +----+-----+----+----+ 2 | |col1| col2|col3|col4| 3 | +----+-----+----+----+ 4 | | 1.0| 3.40|NULL| 0| 5 | | 2.0|34.56| 0|NULL| 6 | +----+-----+----+----+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_table.sql.answer: -------------------------------------------------------------------------------- 1 | +--------+---------+-------+ 2 | |col_name|data_type|comment| 3 | +--------+---------+-------+ 4 | | col| int| NULL| 5 | +--------+---------+-------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/struct.sql.answer: -------------------------------------------------------------------------------- 1 | +---------+--------------------+ 2 | |struct(1)|struct(2, struct(3))| 3 | +---------+--------------------+ 4 | | {1}| {2, {3}}| 5 | +---------+--------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/show_tables.sql.answer: -------------------------------------------------------------------------------- 1 | +---------+---------+-----------+ 2 | |namespace|tableName|isTemporary| 3 | +---------+---------+-----------+ 4 | | |testcache| true| 5 | +---------+---------+-----------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/timestamp.sql.answer: -------------------------------------------------------------------------------- 1 | +--------------------------------------+ 2 | |TIMESTAMP '2025-05-01 16:23:40.945304'| 3 | +--------------------------------------+ 4 | | 2025-05-01 16:23:40.945304| 5 | +--------------------------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_query.sql.answer: -------------------------------------------------------------------------------- 1 | +--------+---------+-------+ 2 | |col_name|data_type|comment| 3 | +--------+---------+-------+ 4 | | id| int| NULL| 5 | | name| string| NULL| 6 | | salary| double| NULL| 7 | +--------+---------+-------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/variant.sql: -------------------------------------------------------------------------------- 1 | FROM VALUES parse_json('{"a": true, "b": 1, "c": "swift"}') T(v) 2 | |> SELECT v, 3 | variant_get(v, '$.a', 'boolean') as a, 4 | variant_get(v, '$.b', 'int') as b, 5 | variant_get(v, '$.c', 'string') as c 6 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/floating.sql.answer: -------------------------------------------------------------------------------- 1 | +---+----+---+----+--------+---+--------+---+ 2 | |1.0|-2.0|3.0|-4.0| inf|NaN| inf|NaN| 3 | +---+----+---+----+--------+---+--------+---+ 4 | |1.0|-2.0|3.0|-4.0|Infinity|NaN|Infinity|NaN| 5 | +---+----+---+----+--------+---+--------+---+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/variant.sql.answer: -------------------------------------------------------------------------------- 1 | +----------------------------+----+---+-----+ 2 | | v| a| b| c| 3 | +----------------------------+----+---+-----+ 4 | |{"a":true,"b":1,"c":"swift"}|true| 1|swift| 5 | +----------------------------+----+---+-----+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/function.sql.answer: -------------------------------------------------------------------------------- 1 | +-----------------------------------------------+ 2 | |length(java_method(java.util.UUID, randomUUID))| 3 | +-----------------------------------------------+ 4 | | 36| 5 | +-----------------------------------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/cast.sql.answer: -------------------------------------------------------------------------------- 1 | +---------------------+-----------------+--------------+-------------------+ 2 | |CAST(true AS BOOLEAN)|CAST(1 AS STRING)|CAST(1 AS INT)|CAST(3.14 AS FLOAT)| 3 | +---------------------+-----------------+--------------+-------------------+ 4 | | true| 1| 1| 3.14| 5 | +---------------------+-----------------+--------------+-------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/time.sql.answer: -------------------------------------------------------------------------------- 1 | +----------------------+------------------------+----------------------------+ 2 | |TIME '12:34:56.123456'|to_time(12:34:56.123456)|make_time(12, 34, 56.123456)| 3 | +----------------------+------------------------+----------------------------+ 4 | | 12:34:56.123456| 12:34:56.123456| 12:34:56.123456| 5 | +----------------------+------------------------+----------------------------+ -------------------------------------------------------------------------------- /.github/.licenserc.yaml: -------------------------------------------------------------------------------- 1 | header: 2 | license: 3 | spdx-id: Apache-2.0 4 | copyright-owner: Apache Software Foundation 5 | 6 | paths: 7 | - '**' 8 | 9 | paths-ignore: 10 | - '**/*.md' 11 | - '**/.gitignore' 12 | - '.github/**' 13 | - 'LICENSE' 14 | - 'NOTICE' 15 | - '.asf.yaml' 16 | - '.nojekyll' 17 | - 'Package.swift' 18 | - '**/*pb.swift' 19 | - 'Tests/SparkConnectTests/Resources/queries/**' 20 | 21 | comment: on-failure 22 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/integral.sql.answer: -------------------------------------------------------------------------------- 1 | +---+----+-----+------+----------+-----------+-------------------+--------------------+ 2 | |127|-128|32767|-32768|2147483647|-2147483648|9223372036854775807|-9223372036854775808| 3 | +---+----+-----+------+----------+-----------+-------------------+--------------------+ 4 | |127|-128|32767|-32768|2147483647|-2147483648|9223372036854775807|-9223372036854775808| 5 | +---+----+-----+------+----------+-----------+-------------------+--------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_database.sql.answer: -------------------------------------------------------------------------------- 1 | +--------------+----------------------------------------+ 2 | | info_name| info_value| 3 | +--------------+----------------------------------------+ 4 | | Catalog Name| spark_catalog| 5 | |Namespace Name| default| 6 | | Comment| default database| 7 | | Location|*| 8 | | Owner| spark| 9 | +--------------+----------------------------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/describe_function.sql.answer: -------------------------------------------------------------------------------- 1 | +-------------------------------------------------------------------------------+ 2 | | function_desc| 3 | +-------------------------------------------------------------------------------+ 4 | | Function: abs| 5 | | Class: org.apache.spark.sql.catalyst.expressions.Abs| 6 | |Usage: abs(expr) - Returns the absolute value of the numeric or interval value.| 7 | +-------------------------------------------------------------------------------+ -------------------------------------------------------------------------------- /.markdownlint.yaml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | MD013: false 19 | -------------------------------------------------------------------------------- /.markdownlintignore: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one 2 | # or more contributor license agreements. See the NOTICE file 3 | # distributed with this work for additional information 4 | # regarding copyright ownership. The ASF licenses this file 5 | # to you under the Apache License, Version 2.0 (the 6 | # "License"); you may not use this file except in compliance 7 | # with the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, 12 | # software distributed under the License is distributed on an 13 | # "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | # KIND, either express or implied. See the License for the 15 | # specific language governing permissions and limitations 16 | # under the License. 17 | 18 | Sources/SparkConnect/Documentation.docc/Examples.md 19 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Documentation.docc/SparkConnect.md: -------------------------------------------------------------------------------- 1 | # ``SparkConnect`` 2 | 3 | Swift implementation of Apache Spark Connect client for distributed data processing. 4 | 5 | ## Overview 6 | 7 | SparkConnect is a modern Swift library that provides a native interface to Apache Spark clusters using the Spark Connect protocol. This library enables Swift developers to leverage the power of Apache Spark for distributed data processing, machine learning, and analytical workloads directly from their Swift applications. 8 | 9 | ### Key Features 10 | 11 | - Native Swift API for Apache Spark operations 12 | - Support for DataFrame and SQL operations 13 | - Support for grouped data operations and aggregations 14 | - Efficient data serialization using Arrow format 15 | 16 | ## Topics 17 | 18 | ### Getting Started 19 | 20 | - 21 | - ``SparkSession`` 22 | 23 | ### DataFrame Operations 24 | 25 | - ``DataFrame`` 26 | -------------------------------------------------------------------------------- /.spi.yml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | version: 1 16 | metadata: 17 | authors: "Apache Spark project" 18 | builder: 19 | configs: 20 | - documentation_targets: [SparkConnect] 21 | -------------------------------------------------------------------------------- /Examples/pi/README.md: -------------------------------------------------------------------------------- 1 | # A Swift Application computing an approximation to pi with Apache Spark Connect Swift Client 2 | 3 | This is an example Swift application to show how to use Apache Spark Connect Swift Client library. 4 | 5 | ## How to run 6 | 7 | Prepare `Spark Connect Server` via running Docker image. 8 | 9 | ```bash 10 | docker run --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait" 11 | ``` 12 | 13 | Build an application Docker image. 14 | 15 | ```bash 16 | $ docker build -t apache/spark-connect-swift:pi . 17 | $ docker images apache/spark-connect-swift:pi 18 | REPOSITORY TAG IMAGE ID CREATED SIZE 19 | apache/spark-connect-swift pi d03952577564 4 seconds ago 369MB 20 | ``` 21 | 22 | Run `pi` docker image. 23 | 24 | ```bash 25 | $ docker run --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:pi 26 | Pi is roughly 3.1412831412831412 27 | ``` 28 | 29 | Run from source code. 30 | 31 | ```bash 32 | $ swift run 33 | ... 34 | Pi is roughly 3.1423711423711422 35 | ``` 36 | -------------------------------------------------------------------------------- /Examples/web/Sources/SparkConnectSwiftWeb/configure.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Vapor 21 | 22 | // configures your application 23 | public func configure(_ app: Application) async throws { 24 | app.http.server.configuration.hostname = "0.0.0.0" 25 | // register routes 26 | try routes(app) 27 | } 28 | -------------------------------------------------------------------------------- /Sources/SparkConnect/MemoryAllocator.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class MemoryAllocator { 22 | let alignment: Int 23 | init(_ alignment: Int) { 24 | self.alignment = alignment 25 | } 26 | 27 | func allocateArray(_ byteCount: Int) -> UnsafeMutableRawPointer { 28 | return UnsafeMutableRawPointer.allocate( 29 | byteCount: byteCount, 30 | alignment: self.alignment) 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /Examples/pi/Sources/main.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import SparkConnect 21 | 22 | let spark = try await SparkSession.builder.getOrCreate() 23 | 24 | let n: Int64 = CommandLine.arguments.count > 1 ? Int64(CommandLine.arguments[1])! : 1_000_000 25 | 26 | let count = 27 | try await spark 28 | .range(n) 29 | .selectExpr("(pow(rand() * 2 - 1, 2) + pow(rand() * 2 - 1, 2)) as v") 30 | .where("v <= 1") 31 | .count() 32 | 33 | print("Pi is roughly \(4.0 * Double(count) / (Double(n) - 1))") 34 | 35 | await spark.stop() 36 | -------------------------------------------------------------------------------- /Examples/pi/Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // 3 | // Licensed to the Apache Software Foundation (ASF) under one 4 | // or more contributor license agreements. See the NOTICE file 5 | // distributed with this work for additional information 6 | // regarding copyright ownership. The ASF licenses this file 7 | // to you under the Apache License, Version 2.0 (the 8 | // "License"); you may not use this file except in compliance 9 | // with the License. You may obtain a copy of the License at 10 | // 11 | // http://www.apache.org/licenses/LICENSE-2.0 12 | // 13 | // Unless required by applicable law or agreed to in writing, 14 | // software distributed under the License is distributed on an 15 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | // KIND, either express or implied. See the License for the 17 | // specific language governing permissions and limitations 18 | // under the License. 19 | // 20 | 21 | import PackageDescription 22 | 23 | let package = Package( 24 | name: "SparkConnectSwiftPi", 25 | platforms: [ 26 | .macOS(.v15) 27 | ], 28 | dependencies: [ 29 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main") 30 | ], 31 | targets: [ 32 | .executableTarget( 33 | name: "SparkConnectSwiftPi", 34 | dependencies: [.product(name: "SparkConnect", package: "spark-connect-swift")] 35 | ) 36 | ] 37 | ) 38 | -------------------------------------------------------------------------------- /Examples/app/Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // 3 | // Licensed to the Apache Software Foundation (ASF) under one 4 | // or more contributor license agreements. See the NOTICE file 5 | // distributed with this work for additional information 6 | // regarding copyright ownership. The ASF licenses this file 7 | // to you under the Apache License, Version 2.0 (the 8 | // "License"); you may not use this file except in compliance 9 | // with the License. You may obtain a copy of the License at 10 | // 11 | // http://www.apache.org/licenses/LICENSE-2.0 12 | // 13 | // Unless required by applicable law or agreed to in writing, 14 | // software distributed under the License is distributed on an 15 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | // KIND, either express or implied. See the License for the 17 | // specific language governing permissions and limitations 18 | // under the License. 19 | // 20 | 21 | import PackageDescription 22 | 23 | let package = Package( 24 | name: "SparkConnectSwiftApp", 25 | platforms: [ 26 | .macOS(.v15) 27 | ], 28 | dependencies: [ 29 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main") 30 | ], 31 | targets: [ 32 | .executableTarget( 33 | name: "SparkConnectSwiftApp", 34 | dependencies: [.product(name: "SparkConnect", package: "spark-connect-swift")] 35 | ) 36 | ] 37 | ) 38 | -------------------------------------------------------------------------------- /Examples/spark-sql/Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // 3 | // Licensed to the Apache Software Foundation (ASF) under one 4 | // or more contributor license agreements. See the NOTICE file 5 | // distributed with this work for additional information 6 | // regarding copyright ownership. The ASF licenses this file 7 | // to you under the Apache License, Version 2.0 (the 8 | // "License"); you may not use this file except in compliance 9 | // with the License. You may obtain a copy of the License at 10 | // 11 | // http://www.apache.org/licenses/LICENSE-2.0 12 | // 13 | // Unless required by applicable law or agreed to in writing, 14 | // software distributed under the License is distributed on an 15 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | // KIND, either express or implied. See the License for the 17 | // specific language governing permissions and limitations 18 | // under the License. 19 | // 20 | 21 | import PackageDescription 22 | 23 | let package = Package( 24 | name: "SparkConnectSwiftSQLRepl", 25 | platforms: [ 26 | .macOS(.v15) 27 | ], 28 | dependencies: [ 29 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main") 30 | ], 31 | targets: [ 32 | .executableTarget( 33 | name: "SparkConnectSwiftSQLRepl", 34 | dependencies: [.product(name: "SparkConnect", package: "spark-connect-swift")] 35 | ) 36 | ] 37 | ) 38 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Documentation.docc/SparkSession.md: -------------------------------------------------------------------------------- 1 | # ``SparkConnect/SparkSession`` 2 | 3 | The entry point for SparkConnect functionality. 4 | 5 | ## Overview 6 | 7 | `SparkSession` is the primary interaction point with Apache Spark. It provides an interface to create DataFrames, execute SQL queries, and manage cluster configurations. 8 | 9 | ### Creating a SparkSession 10 | 11 | ```swift 12 | let spark = try await SparkSession 13 | .builder 14 | .appName("MySwiftApp") 15 | .remote("sc://localhost:15002") 16 | .getOrCreate() 17 | ``` 18 | 19 | ### Basic Usage 20 | 21 | ```swift 22 | // Create a DataFrame from a range 23 | let df = spark.range(1, 10) 24 | 25 | // Execute SQL query 26 | let result = spark.sql("SELECT * FROM table") 27 | 28 | // Read data from files 29 | let csvDf = spark.read.csv("path/to/file.csv") 30 | ``` 31 | 32 | ## Topics 33 | 34 | ### Creating Sessions 35 | 36 | - ``builder`` 37 | - ``stop()`` 38 | 39 | ### DataFrame Operations 40 | 41 | - ``emptyDataFrame`` 42 | - ``range(_:_:_:)`` 43 | - ``sql(_:)`` 44 | 45 | ### Data I/O 46 | 47 | - ``read`` 48 | 49 | ### Configuration 50 | 51 | - ``conf`` 52 | 53 | ### Catalog Operations 54 | 55 | - ``catalog`` 56 | 57 | ### Managing Operations 58 | 59 | - ``addTag(_:)`` 60 | - ``removeTag(_:)`` 61 | - ``getTags()`` 62 | - ``clearTags()`` 63 | - ``interruptAll()`` 64 | - ``interruptTag(_:)`` 65 | - ``interruptOperation(_:)`` 66 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ErrorUtils.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | /// Utility functions like `org.apache.spark.util.SparkErrorUtils`. 22 | public enum ErrorUtils { 23 | public static func tryWithSafeFinally( 24 | _ block: () async throws -> T, _ finallyBlock: () async throws -> Void 25 | ) async rethrows -> T { 26 | let result: T 27 | do { 28 | result = try await block() 29 | try await finallyBlock() 30 | } catch { 31 | try? await finallyBlock() 32 | throw error 33 | } 34 | return result 35 | } 36 | } 37 | -------------------------------------------------------------------------------- /Examples/stream/Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // 3 | // Licensed to the Apache Software Foundation (ASF) under one 4 | // or more contributor license agreements. See the NOTICE file 5 | // distributed with this work for additional information 6 | // regarding copyright ownership. The ASF licenses this file 7 | // to you under the Apache License, Version 2.0 (the 8 | // "License"); you may not use this file except in compliance 9 | // with the License. You may obtain a copy of the License at 10 | // 11 | // http://www.apache.org/licenses/LICENSE-2.0 12 | // 13 | // Unless required by applicable law or agreed to in writing, 14 | // software distributed under the License is distributed on an 15 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | // KIND, either express or implied. See the License for the 17 | // specific language governing permissions and limitations 18 | // under the License. 19 | // 20 | 21 | import PackageDescription 22 | 23 | let package = Package( 24 | name: "SparkConnectSwiftNetworkWordCount", 25 | platforms: [ 26 | .macOS(.v15) 27 | ], 28 | dependencies: [ 29 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main") 30 | ], 31 | targets: [ 32 | .executableTarget( 33 | name: "SparkConnectSwiftNetworkWordCount", 34 | dependencies: [.product(name: "SparkConnect", package: "spark-connect-swift")] 35 | ) 36 | ] 37 | ) 38 | -------------------------------------------------------------------------------- /Sources/SparkConnect/SparkConnectError.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | /// A enum for ``SparkConnect`` package errors 21 | public enum SparkConnectError: Error { 22 | case CatalogNotFound 23 | case ColumnNotFound 24 | case DataSourceNotFound 25 | case InvalidArgument 26 | case InvalidSessionID 27 | case InvalidType 28 | case InvalidViewName 29 | case OutputTypeUnspecified 30 | case ParseSyntaxError 31 | case SchemaNotFound 32 | case SessionClosed 33 | case SqlConfNotFound 34 | case TableOrViewAlreadyExists 35 | case TableOrViewNotFound 36 | case UnsupportedOperation 37 | } 38 | -------------------------------------------------------------------------------- /.asf.yaml: -------------------------------------------------------------------------------- 1 | # Licensed to the Apache Software Foundation (ASF) under one or more 2 | # contributor license agreements. See the NOTICE file distributed with 3 | # this work for additional information regarding copyright ownership. 4 | # The ASF licenses this file to You under the Apache License, Version 2.0 5 | # (the "License"); you may not use this file except in compliance with 6 | # the License. You may obtain a copy of the License at 7 | # 8 | # http://www.apache.org/licenses/LICENSE-2.0 9 | # 10 | # Unless required by applicable law or agreed to in writing, software 11 | # distributed under the License is distributed on an "AS IS" BASIS, 12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 | # See the License for the specific language governing permissions and 14 | # limitations under the License. 15 | 16 | # https://github.com/apache/infrastructure-asfyaml/blob/main/README.md 17 | --- 18 | github: 19 | description: "Apache Spark Connect Client for Swift" 20 | homepage: https://apache.github.io/spark-connect-swift/ 21 | labels: 22 | - swift 23 | - big-data 24 | - sql 25 | - spark 26 | - streaming 27 | enabled_merge_buttons: 28 | merge: false 29 | squash: true 30 | rebase: true 31 | ghp_branch: gh-pages 32 | ghp_path: / 33 | autolink_jira: SPARK 34 | 35 | notifications: 36 | pullrequests: reviews@spark.apache.org 37 | issues: reviews@spark.apache.org 38 | commits: commits@spark.apache.org 39 | jira_options: link label 40 | -------------------------------------------------------------------------------- /Examples/web/Sources/SparkConnectSwiftWeb/routes.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import SparkConnect 21 | import Vapor 22 | 23 | func routes(_ app: Application) throws { 24 | app.get { req async in 25 | "Welcome to the Swift world. Say hello!" 26 | } 27 | 28 | app.get("hello") { req async -> String in 29 | return await Task { 30 | do { 31 | let spark = try await SparkSession.builder.getOrCreate() 32 | let response = "Hi, this is powered by the Apache Spark \(await spark.version)." 33 | await spark.stop() 34 | return response 35 | } catch { 36 | return "Fail to connect: \(error)" 37 | } 38 | }.value 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Documentation.docc/Info.plist: -------------------------------------------------------------------------------- 1 | 2 | 18 | 19 | 20 | 21 | CFBundleDisplayName 22 | SparkConnect 23 | CFBundleIdentifier 24 | org.apache.spark.connect.swift.SparkConnect 25 | CFBundleName 26 | SparkConnect 27 | CFBundleVersion 28 | 0.1.0 29 | NSHumanReadableCopyright 30 | © 2025 Apache Software Foundation 31 | 32 | 33 | -------------------------------------------------------------------------------- /Examples/app/Sources/main.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import SparkConnect 21 | 22 | let spark = try await SparkSession.builder.getOrCreate() 23 | print("Connected to Apache Spark \(await spark.version) Server") 24 | 25 | let statements = [ 26 | "DROP TABLE IF EXISTS t", 27 | "CREATE TABLE IF NOT EXISTS t(a INT) USING ORC", 28 | "INSERT INTO t VALUES (1), (2), (3)", 29 | ] 30 | 31 | for s in statements { 32 | print("EXECUTE: \(s)") 33 | _ = try await spark.sql(s).count() 34 | } 35 | print("SELECT * FROM t") 36 | try await spark.sql("SELECT * FROM t").cache().show() 37 | 38 | try await spark.range(10).filter("id % 2 == 0").write.mode("overwrite").orc("/tmp/orc") 39 | try await spark.read.orc("/tmp/orc").show() 40 | 41 | await spark.stop() 42 | -------------------------------------------------------------------------------- /.github/workflows/publish_image.yml: -------------------------------------------------------------------------------- 1 | name: Publish Image 2 | 3 | on: 4 | workflow_dispatch: 5 | inputs: 6 | example: 7 | description: 'list of examples to publish (JSON)' 8 | required: true 9 | # keep in sync with default value of strategy matrix 'example' 10 | default: '["pi", "web"]' 11 | 12 | jobs: 13 | publish-image: 14 | if: ${{ startsWith(github.repository, 'apache/') }} 15 | runs-on: ubuntu-latest 16 | strategy: 17 | fail-fast: false 18 | matrix: 19 | # keep in sync with default value of workflow_dispatch input 'example' 20 | example: ${{ fromJSON( inputs.example || '["pi", "web"]' ) }} 21 | steps: 22 | - name: Set up QEMU 23 | uses: docker/setup-qemu-action@v3 24 | - name: Set up Docker Buildx 25 | uses: docker/setup-buildx-action@v3 26 | - name: Login to Docker Hub 27 | uses: docker/login-action@v3 28 | with: 29 | username: ${{ secrets.DOCKERHUB_USER }} 30 | password: ${{ secrets.DOCKERHUB_TOKEN }} 31 | - name: Checkout repository 32 | uses: actions/checkout@v6 33 | with: 34 | ref: main 35 | - name: Build and push 36 | uses: docker/build-push-action@v6 37 | with: 38 | # build cache on Github Actions, See: https://docs.docker.com/build/cache/backends/gha/#using-dockerbuild-push-action 39 | cache-from: type=gha 40 | cache-to: type=gha,mode=max 41 | context: Examples/${{ matrix.example }} 42 | file: Examples/${{ matrix.example }}/Dockerfile 43 | platforms: linux/amd64,linux/arm64 44 | push: true 45 | tags: apache/spark-connect-swift:${{ matrix.example }} 46 | -------------------------------------------------------------------------------- /Examples/stream/Sources/main.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | 23 | let spark = try await SparkSession.builder.getOrCreate() 24 | print("Connected to Apache Spark \(await spark.version) Server") 25 | 26 | let host = ProcessInfo.processInfo.environment["TARGET_HOST"] ?? "localhost" 27 | 28 | let lines = 29 | await spark 30 | .readStream 31 | .format("socket") 32 | .option("host", host) 33 | .option("port", 9999) 34 | .load() 35 | 36 | let word = 37 | await lines 38 | .selectExpr("explode(split(value, ' ')) as word") 39 | 40 | let wordCounts = 41 | await word 42 | .groupBy("word") 43 | .agg("count(*)") 44 | 45 | let query = 46 | try await wordCounts 47 | .writeStream 48 | .outputMode("complete") 49 | .format("console") 50 | .start() 51 | 52 | _ = try await query.awaitTermination() 53 | -------------------------------------------------------------------------------- /Sources/SparkConnect/BearerTokenInterceptor.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import GRPCCore 22 | 23 | struct BearerTokenInterceptor: ClientInterceptor { 24 | let token: String 25 | 26 | init(token: String) { 27 | self.token = token 28 | } 29 | 30 | func intercept( 31 | request: StreamingClientRequest, 32 | context: ClientContext, 33 | next: ( 34 | _ request: StreamingClientRequest, 35 | _ context: ClientContext 36 | ) async throws -> StreamingClientResponse 37 | ) async throws -> StreamingClientResponse { 38 | var request = request 39 | request.metadata.addString("Bearer \(self.token)", forKey: "Authorization") 40 | 41 | // Forward the request to the next interceptor. 42 | return try await next(request, context) 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /Examples/app/README.md: -------------------------------------------------------------------------------- 1 | # A Swift Application with Apache Spark Connect Swift Client 2 | 3 | This is an example Swift application to show how to use Apache Spark Connect Swift Client library. 4 | 5 | ## How to run 6 | 7 | Prepare `Spark Connect Server` via running Docker image. 8 | 9 | ```bash 10 | docker run --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait" 11 | ``` 12 | 13 | Build an application Docker image. 14 | 15 | ```bash 16 | $ docker build -t apache/spark-connect-swift:app . 17 | $ docker images apache/spark-connect-swift:app 18 | REPOSITORY TAG IMAGE ID CREATED SIZE 19 | apache/spark-connect-swift app e132e1b38348 5 seconds ago 368MB 20 | ``` 21 | 22 | Run `app` docker image. 23 | 24 | ```bash 25 | $ docker run --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:app 26 | Connected to Apache Spark 4.0.1 Server 27 | EXECUTE: DROP TABLE IF EXISTS t 28 | EXECUTE: CREATE TABLE IF NOT EXISTS t(a INT) USING ORC 29 | EXECUTE: INSERT INTO t VALUES (1), (2), (3) 30 | SELECT * FROM t 31 | +---+ 32 | | a| 33 | +---+ 34 | | 2| 35 | | 1| 36 | | 3| 37 | +---+ 38 | 39 | +---+ 40 | | id| 41 | +---+ 42 | | 0| 43 | | 8| 44 | | 6| 45 | | 2| 46 | | 4| 47 | +---+ 48 | ``` 49 | 50 | Run from source code. 51 | 52 | ```bash 53 | $ swift run 54 | ... 55 | Connected to Apache Spark 4.0.1 Server 56 | EXECUTE: DROP TABLE IF EXISTS t 57 | EXECUTE: CREATE TABLE IF NOT EXISTS t(a INT) USING ORC 58 | EXECUTE: INSERT INTO t VALUES (1), (2), (3) 59 | SELECT * FROM t 60 | +---+ 61 | | a | 62 | +---+ 63 | | 2 | 64 | | 1 | 65 | | 3 | 66 | +---+ 67 | +----+ 68 | | id | 69 | +----+ 70 | | 2 | 71 | | 6 | 72 | | 0 | 73 | | 8 | 74 | | 4 | 75 | +----+ 76 | ``` 77 | -------------------------------------------------------------------------------- /Examples/pi/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | FROM swift:6.2 AS builder 18 | 19 | WORKDIR /app 20 | 21 | COPY . . 22 | 23 | RUN swift build -c release 24 | 25 | FROM swift:6.2-slim 26 | 27 | ARG SPARK_UID=185 28 | 29 | LABEL org.opencontainers.image.authors="Apache Spark project " 30 | LABEL org.opencontainers.image.licenses="Apache-2.0" 31 | LABEL org.opencontainers.image.ref.name="Apache Spark Connect for Swift" 32 | 33 | ENV SPARK_SWIFT_HOME=/opt/spark-swift 34 | ENV SPARK_SWIFT_APP=SparkConnectSwiftPi 35 | 36 | WORKDIR $SPARK_SWIFT_HOME 37 | 38 | RUN groupadd --system --gid=$SPARK_UID spark && \ 39 | useradd --system --home-dir $SPARK_SWIFT_HOME --uid=$SPARK_UID --gid=spark spark && \ 40 | chown -R spark:spark $SPARK_SWIFT_HOME 41 | 42 | COPY --from=builder --chown=spark:spark /app/.build/*-unknown-linux-gnu/release/$SPARK_SWIFT_APP . 43 | 44 | USER spark 45 | 46 | ENTRYPOINT ["/bin/sh", "-c", "$SPARK_SWIFT_HOME/$SPARK_SWIFT_APP"] 47 | -------------------------------------------------------------------------------- /Examples/app/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | FROM swift:6.2 AS builder 18 | 19 | WORKDIR /app 20 | 21 | COPY . . 22 | 23 | RUN swift build -c release 24 | 25 | FROM swift:6.2-slim 26 | 27 | ARG SPARK_UID=185 28 | 29 | LABEL org.opencontainers.image.authors="Apache Spark project " 30 | LABEL org.opencontainers.image.licenses="Apache-2.0" 31 | LABEL org.opencontainers.image.ref.name="Apache Spark Connect for Swift" 32 | 33 | ENV SPARK_SWIFT_HOME=/opt/spark-swift 34 | ENV SPARK_SWIFT_APP=SparkConnectSwiftApp 35 | 36 | WORKDIR $SPARK_SWIFT_HOME 37 | 38 | RUN groupadd --system --gid=$SPARK_UID spark && \ 39 | useradd --system --home-dir $SPARK_SWIFT_HOME --uid=$SPARK_UID --gid=spark spark && \ 40 | chown -R spark:spark $SPARK_SWIFT_HOME 41 | 42 | COPY --from=builder --chown=spark:spark /app/.build/*-unknown-linux-gnu/release/$SPARK_SWIFT_APP . 43 | 44 | USER spark 45 | 46 | ENTRYPOINT ["/bin/sh", "-c", "$SPARK_SWIFT_HOME/$SPARK_SWIFT_APP"] 47 | -------------------------------------------------------------------------------- /Examples/spark-sql/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | FROM swift:6.2 AS builder 18 | 19 | WORKDIR /app 20 | 21 | COPY . . 22 | 23 | RUN swift build -c release 24 | 25 | FROM swift:6.2-slim 26 | 27 | ARG SPARK_UID=185 28 | 29 | LABEL org.opencontainers.image.authors="Apache Spark project " 30 | LABEL org.opencontainers.image.licenses="Apache-2.0" 31 | LABEL org.opencontainers.image.ref.name="Apache Spark Connect for Swift" 32 | 33 | ENV SPARK_SWIFT_HOME=/opt/spark-swift 34 | ENV SPARK_SWIFT_APP=SparkConnectSwiftSQLRepl 35 | 36 | WORKDIR $SPARK_SWIFT_HOME 37 | 38 | RUN groupadd --system --gid=$SPARK_UID spark && \ 39 | useradd --system --home-dir $SPARK_SWIFT_HOME --uid=$SPARK_UID --gid=spark spark && \ 40 | chown -R spark:spark $SPARK_SWIFT_HOME 41 | 42 | COPY --from=builder --chown=spark:spark /app/.build/*-unknown-linux-gnu/release/$SPARK_SWIFT_APP . 43 | 44 | USER spark 45 | 46 | ENTRYPOINT ["/bin/sh", "-c", "$SPARK_SWIFT_HOME/$SPARK_SWIFT_APP"] 47 | -------------------------------------------------------------------------------- /Sources/SparkConnect/GroupedData.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | public actor GroupedData { 21 | let df: DataFrame 22 | let groupType: GroupType 23 | let groupingCols: [String] 24 | 25 | init(_ df: DataFrame, _ groupType: GroupType, _ groupingCols: [String]) { 26 | self.df = df 27 | self.groupType = groupType 28 | self.groupingCols = groupingCols 29 | } 30 | 31 | public func agg(_ exprs: String...) async -> DataFrame { 32 | var aggregate = Aggregate() 33 | aggregate.input = await (self.df.getPlan() as! Plan).root 34 | aggregate.groupType = self.groupType 35 | aggregate.groupingExpressions = self.groupingCols.map { $0.toExpression } 36 | aggregate.aggregateExpressions = exprs.map { $0.toExpression } 37 | var relation = Relation() 38 | relation.aggregate = aggregate 39 | var plan = Plan() 40 | plan.opType = .root(relation) 41 | return await DataFrame(spark: df.spark, plan: plan) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /Examples/stream/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | FROM swift:6.2 AS builder 18 | 19 | WORKDIR /app 20 | 21 | COPY . . 22 | 23 | RUN swift build -c release 24 | 25 | FROM swift:6.2-slim 26 | 27 | ARG SPARK_UID=185 28 | 29 | LABEL org.opencontainers.image.authors="Apache Spark project " 30 | LABEL org.opencontainers.image.licenses="Apache-2.0" 31 | LABEL org.opencontainers.image.ref.name="Apache Spark Connect for Swift" 32 | 33 | ENV SPARK_SWIFT_HOME=/opt/spark-swift 34 | ENV SPARK_SWIFT_APP=SparkConnectSwiftNetworkWordCount 35 | 36 | WORKDIR $SPARK_SWIFT_HOME 37 | 38 | RUN groupadd --system --gid=$SPARK_UID spark && \ 39 | useradd --system --home-dir $SPARK_SWIFT_HOME --uid=$SPARK_UID --gid=spark spark && \ 40 | chown -R spark:spark $SPARK_SWIFT_HOME 41 | 42 | COPY --from=builder --chown=spark:spark /app/.build/*-unknown-linux-gnu/release/$SPARK_SWIFT_APP . 43 | 44 | USER spark 45 | 46 | ENTRYPOINT ["/bin/sh", "-c", "$SPARK_SWIFT_HOME/$SPARK_SWIFT_APP"] 47 | -------------------------------------------------------------------------------- /Examples/web/Dockerfile: -------------------------------------------------------------------------------- 1 | # 2 | # Licensed to the Apache Software Foundation (ASF) under one or more 3 | # contributor license agreements. See the NOTICE file distributed with 4 | # this work for additional information regarding copyright ownership. 5 | # The ASF licenses this file to You under the Apache License, Version 2.0 6 | # (the "License"); you may not use this file except in compliance with 7 | # the License. You may obtain a copy of the License at 8 | # 9 | # http://www.apache.org/licenses/LICENSE-2.0 10 | # 11 | # Unless required by applicable law or agreed to in writing, software 12 | # distributed under the License is distributed on an "AS IS" BASIS, 13 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | # See the License for the specific language governing permissions and 15 | # limitations under the License. 16 | # 17 | FROM swift:6.2 AS builder 18 | 19 | WORKDIR /app 20 | 21 | COPY . . 22 | 23 | RUN swift build -c release 24 | 25 | FROM swift:6.2-slim 26 | 27 | ARG SPARK_UID=185 28 | 29 | LABEL org.opencontainers.image.authors="Apache Spark project " 30 | LABEL org.opencontainers.image.licenses="Apache-2.0" 31 | LABEL org.opencontainers.image.ref.name="Apache Spark Connect for Swift" 32 | 33 | ENV SPARK_SWIFT_HOME=/opt/spark-swift 34 | ENV SPARK_SWIFT_APP=SparkConnectSwiftWeb 35 | 36 | WORKDIR $SPARK_SWIFT_HOME 37 | 38 | RUN groupadd --system --gid=$SPARK_UID spark && \ 39 | useradd --system --home-dir $SPARK_SWIFT_HOME --uid=$SPARK_UID --gid=spark spark && \ 40 | chown -R spark:spark $SPARK_SWIFT_HOME 41 | 42 | COPY --from=builder --chown=spark:spark /app/.build/*-unknown-linux-gnu/release/$SPARK_SWIFT_APP . 43 | 44 | USER spark 45 | 46 | EXPOSE 8080 47 | 48 | ENTRYPOINT ["/bin/sh", "-c", "$SPARK_SWIFT_HOME/$SPARK_SWIFT_APP"] 49 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ProtoUtils.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | /// Utility functions like `org.apache.spark.sql.connect.common.ProtoUtils`. 22 | public enum ProtoUtils { 23 | 24 | private static let SPARK_JOB_TAGS_SEP = "," // SparkContext.SPARK_JOB_TAGS_SEP 25 | 26 | /// Validate if a tag for ExecutePlanRequest.tags is valid. Throw IllegalArgumentException if not. 27 | /// - Parameter tag: A tag string. 28 | public static func throwIfInvalidTag(_ tag: String) throws { 29 | // Same format rules apply to Spark Connect execution tags as to SparkContext job tags, 30 | // because the Spark Connect job tag is also used as part of SparkContext job tag. 31 | // See SparkContext.throwIfInvalidTag and ExecuteHolderSessionTag 32 | if tag.isEmpty { 33 | throw SparkConnectError.InvalidArgument 34 | } 35 | if tag.contains(SPARK_JOB_TAGS_SEP) { 36 | throw SparkConnectError.InvalidArgument 37 | } 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /Examples/spark-sql/Sources/main.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | 23 | let statement = /([^;]*);/ 24 | 25 | let spark = try await SparkSession.builder.getOrCreate() 26 | print("Connected to Apache Spark \(await spark.version) Server") 27 | 28 | var isRunning = true 29 | var lines = "" 30 | while isRunning { 31 | if lines.isEmpty { 32 | print("spark-sql (\(try await spark.catalog.currentDatabase()))> ", terminator: "") 33 | } 34 | guard let input = readLine() else { 35 | isRunning = false 36 | break 37 | } 38 | lines += input + " " 39 | 40 | let matches = lines.matches(of: statement) 41 | for match in matches { 42 | lines = "" 43 | switch match.1 { 44 | case "exit": 45 | isRunning = false 46 | break 47 | default: 48 | do { 49 | try await spark.time({ try await spark.sql(String(match.1)).show(10000, false) }) 50 | } catch { 51 | print("Error: \(error)") 52 | } 53 | } 54 | } 55 | } 56 | 57 | await spark.stop() 58 | -------------------------------------------------------------------------------- /Sources/SparkConnect/BitUtility.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | class BitUtility { 21 | static func isSet(_ bit: UInt, buffer: ArrowBuffer) -> Bool { 22 | let byteIndex = UInt(bit / 8) 23 | let theByte = buffer.rawPointer.load(fromByteOffset: Int(byteIndex), as: UInt8.self) 24 | return theByte & UInt8(1 << (bit % 8)) > 0 25 | } 26 | 27 | static func setBit(_ bit: UInt, buffer: ArrowBuffer) { 28 | let byteIndex = UInt(bit / 8) 29 | var theByte = buffer.rawPointer.load(fromByteOffset: Int(byteIndex), as: UInt8.self) 30 | theByte |= UInt8(1 << (bit % 8)) 31 | buffer.rawPointer.storeBytes(of: theByte, toByteOffset: Int(byteIndex), as: UInt8.self) 32 | } 33 | 34 | static func clearBit(_ bit: UInt, buffer: ArrowBuffer) { 35 | let byteIndex = UInt(bit / 8) 36 | var theByte = buffer.rawPointer.load(fromByteOffset: Int(byteIndex), as: UInt8.self) 37 | theByte &= ~(UInt8(1 << (bit % 8))) 38 | buffer.rawPointer.storeBytes(of: theByte, toByteOffset: Int(byteIndex), as: UInt8.self) 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | # Xcode 2 | # 3 | # gitignore contributors: remember to update Global/Xcode.gitignore, Objective-C.gitignore & Swift.gitignore 4 | 5 | ## Swift Package Manager 6 | Package.resolved 7 | .swiftpm/ 8 | 9 | ## User settings 10 | xcuserdata/ 11 | 12 | ## Obj-C/Swift specific 13 | *.hmap 14 | 15 | ## App packaging 16 | *.ipa 17 | *.dSYM.zip 18 | *.dSYM 19 | 20 | ## Playgrounds 21 | timeline.xctimeline 22 | playground.xcworkspace 23 | 24 | # Swift Package Manager 25 | # 26 | # Add this line if you want to avoid checking in source code from Swift Package Manager dependencies. 27 | # Packages/ 28 | # Package.pins 29 | # Package.resolved 30 | # *.xcodeproj 31 | # 32 | # Xcode automatically generates this directory with a .xcworkspacedata file and xcuserdata 33 | # hence it is not needed unless you have added a package configuration file to your project 34 | # .swiftpm 35 | 36 | .build/ 37 | 38 | # CocoaPods 39 | # 40 | # We recommend against adding the Pods directory to your .gitignore. However 41 | # you should judge for yourself, the pros and cons are mentioned at: 42 | # https://guides.cocoapods.org/using/using-cocoapods.html#should-i-check-the-pods-directory-into-source-control 43 | # 44 | # Pods/ 45 | # 46 | # Add this line if you want to avoid checking in source code from the Xcode workspace 47 | # *.xcworkspace 48 | 49 | # Carthage 50 | # 51 | # Add this line if you want to avoid checking in source code from Carthage dependencies. 52 | # Carthage/Checkouts 53 | 54 | Carthage/Build/ 55 | 56 | # fastlane 57 | # 58 | # It is recommended to not store the screenshots in the git repo. 59 | # Instead, use fastlane to re-generate the screenshots whenever they are needed. 60 | # For more information about the recommended setup visit: 61 | # https://docs.fastlane.tools/best-practices/source-control/#source-control 62 | 63 | fastlane/report.xml 64 | fastlane/Preview.html 65 | fastlane/screenshots/**/*.png 66 | fastlane/test_output 67 | -------------------------------------------------------------------------------- /Sources/SparkConnect/FlightDescriptor.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class FlightDescriptor { 22 | public enum DescriptorType { 23 | case unknown 24 | case path 25 | case cmd 26 | } 27 | 28 | public let type: FlightDescriptor.DescriptorType 29 | public let cmd: Data 30 | public let paths: [String] 31 | 32 | init(_ descriptor: Arrow_Flight_Protocol_FlightDescriptor) { 33 | self.type = descriptor.type == .cmd ? .cmd : .path 34 | self.cmd = descriptor.cmd 35 | self.paths = descriptor.path 36 | } 37 | 38 | public init(cmd: Data) { 39 | self.type = .cmd 40 | self.cmd = cmd 41 | self.paths = [String]() 42 | } 43 | 44 | public init(paths: [String]) { 45 | self.type = .path 46 | self.cmd = Data() 47 | self.paths = paths 48 | } 49 | 50 | func toProtocol() -> Arrow_Flight_Protocol_FlightDescriptor { 51 | var descriptor = Arrow_Flight_Protocol_FlightDescriptor() 52 | descriptor.type = self.type == .cmd ? .cmd : .path 53 | descriptor.cmd = self.cmd 54 | descriptor.path = self.paths 55 | return descriptor 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/CRC32Tests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | struct CRC32Tests { 25 | @Test 26 | func testChecksumWithEmptyData() async throws { 27 | #expect(CRC32.checksum(data: Data()) == 0) 28 | #expect(CRC32.checksum(string: "") == 0) 29 | #expect(CRC32.checksum(bytes: []) == 0) 30 | } 31 | 32 | @Test 33 | func testChecksum() async throws { 34 | let str = "Apache Spark Connect Client for Swift" 35 | #expect(CRC32.checksum(string: str, encoding: .ascii) == 2_736_908_745) 36 | #expect(CRC32.checksum(data: str.data(using: .ascii)!) == 2_736_908_745) 37 | #expect(CRC32.checksum(bytes: [UInt8](str.data(using: .ascii)!)) == 2_736_908_745) 38 | } 39 | 40 | @Test 41 | func testLongChecksum() async throws { 42 | let str = String(repeating: "Apache Spark Connect Client for Swift", count: 1000) 43 | #expect(CRC32.checksum(string: str, encoding: .ascii) == 1_985_943_888) 44 | #expect(CRC32.checksum(data: str.data(using: .ascii)!) == 1_985_943_888) 45 | #expect(CRC32.checksum(bytes: [UInt8](str.data(using: .ascii)!)) == 1_985_943_888) 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /Sources/SparkConnect/FlightData.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class FlightData { 22 | let flightData: Arrow_Flight_Protocol_FlightData 23 | public var flightDescriptor: FlightDescriptor? { 24 | return flightData.hasFlightDescriptor ? FlightDescriptor(flightData.flightDescriptor) : nil 25 | } 26 | 27 | public var dataHeader: Data { flightData.dataHeader } 28 | 29 | public var dataBody: Data { flightData.dataBody } 30 | 31 | init(_ flightData: Arrow_Flight_Protocol_FlightData) { 32 | self.flightData = flightData 33 | } 34 | 35 | public init(_ dataHeader: Data, dataBody: Data, flightDescriptor: FlightDescriptor? = nil) { 36 | if flightDescriptor != nil { 37 | self.flightData = Arrow_Flight_Protocol_FlightData.with { 38 | $0.dataHeader = dataHeader 39 | $0.dataBody = dataBody 40 | $0.flightDescriptor = flightDescriptor!.toProtocol() 41 | } 42 | } else { 43 | self.flightData = Arrow_Flight_Protocol_FlightData.with { 44 | $0.dataBody = dataBody 45 | } 46 | } 47 | } 48 | 49 | func toProtocol() -> Arrow_Flight_Protocol_FlightData { self.flightData } 50 | } 51 | -------------------------------------------------------------------------------- /Examples/web/Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // 3 | // Licensed to the Apache Software Foundation (ASF) under one 4 | // or more contributor license agreements. See the NOTICE file 5 | // distributed with this work for additional information 6 | // regarding copyright ownership. The ASF licenses this file 7 | // to you under the Apache License, Version 2.0 (the 8 | // "License"); you may not use this file except in compliance 9 | // with the License. You may obtain a copy of the License at 10 | // 11 | // http://www.apache.org/licenses/LICENSE-2.0 12 | // 13 | // Unless required by applicable law or agreed to in writing, 14 | // software distributed under the License is distributed on an 15 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 16 | // KIND, either express or implied. See the License for the 17 | // specific language governing permissions and limitations 18 | // under the License. 19 | // 20 | import PackageDescription 21 | 22 | let package = Package( 23 | name: "SparkConnectSwiftWeb", 24 | platforms: [ 25 | .macOS(.v15) 26 | ], 27 | dependencies: [ 28 | // 💧 A server-side Swift web framework. 29 | .package(url: "https://github.com/vapor/vapor.git", from: "4.115.0"), 30 | // 🔵 Non-blocking, event-driven networking for Swift. Used for custom executors 31 | .package(url: "https://github.com/apple/swift-nio.git", from: "2.65.0"), 32 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main"), 33 | ], 34 | targets: [ 35 | .executableTarget( 36 | name: "SparkConnectSwiftWeb", 37 | dependencies: [ 38 | .product(name: "Vapor", package: "vapor"), 39 | .product(name: "NIOCore", package: "swift-nio"), 40 | .product(name: "NIOPosix", package: "swift-nio"), 41 | .product(name: "SparkConnect", package: "spark-connect-swift"), 42 | ], 43 | swiftSettings: swiftSettings 44 | ) 45 | ] 46 | ) 47 | 48 | var swiftSettings: [SwiftSetting] { 49 | [ 50 | .enableUpcomingFeature("ExistentialAny") 51 | ] 52 | } 53 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/StreamingQueryTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for `StreamingQuery` 26 | @Suite(.serialized) 27 | struct StreamingQueryTests { 28 | 29 | @Test 30 | func create() async throws { 31 | let spark = try await SparkSession.builder.getOrCreate() 32 | let id = UUID() 33 | let runId = UUID() 34 | let query = StreamingQuery(id, runId, "name", spark) 35 | #expect(await query.id == id) 36 | #expect(await query.runId == runId) 37 | #expect(await query.name == "name") 38 | 39 | // Streaming query xxx is not found 40 | try await #require(throws: Error.self) { 41 | try await query.isActive 42 | } 43 | try await #require(throws: Error.self) { 44 | try await query.recentProgress 45 | } 46 | try await #require(throws: Error.self) { 47 | try await query.lastProgress 48 | } 49 | try await #require(throws: Error.self) { 50 | try await query.awaitTermination() 51 | } 52 | try await #require(throws: Error.self) { 53 | try await query.awaitTermination(1000) 54 | } 55 | await spark.stop() 56 | } 57 | } 58 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/BuilderTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for `SparkSession.Builder` 26 | @Suite(.serialized) 27 | struct BuilderTests { 28 | let TEST_REMOTE = ProcessInfo.processInfo.environment["SPARK_REMOTE"] ?? "sc://localhost:15002" 29 | 30 | @Test 31 | func builderDefault() async throws { 32 | let url = URL(string: self.TEST_REMOTE)! 33 | let spark = try await SparkSession.builder.getOrCreate() 34 | #expect(await spark.client.clientType == "swift") 35 | #expect(await spark.client.url.host() == url.host()) 36 | #expect(await spark.client.url.port == url.port) 37 | await spark.stop() 38 | } 39 | 40 | @Test 41 | func remote() async throws { 42 | // Don't try to connect 43 | let builder = await SparkSession.builder.remote("sc://spark:1234") 44 | #expect(await builder.sparkConf["spark.remote"] == "sc://spark:1234") 45 | await builder.clear() 46 | } 47 | 48 | @Test 49 | func appName() async throws { 50 | let builder = await SparkSession.builder.appName("TestApp") 51 | #expect(await builder.sparkConf["spark.app.name"] == "TestApp") 52 | try await builder.getOrCreate().stop() 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /Examples/web/Sources/SparkConnectSwiftWeb/entrypoint.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Logging 21 | import NIOCore 22 | import NIOPosix 23 | import Vapor 24 | 25 | @main 26 | enum Entrypoint { 27 | static func main() async throws { 28 | var env = try Environment.detect() 29 | try LoggingSystem.bootstrap(from: &env) 30 | 31 | let app = try await Application.make(env) 32 | 33 | // This attempts to install NIO as the Swift Concurrency global executor. 34 | // You can enable it if you'd like to reduce the amount of context switching between NIO and Swift Concurrency. 35 | // Note: this has caused issues with some libraries that use `.wait()` and cleanly shutting down. 36 | // If enabled, you should be careful about calling async functions before this point as it can cause assertion failures. 37 | // let executorTakeoverSuccess = NIOSingletons.unsafeTryInstallSingletonPosixEventLoopGroupAsConcurrencyGlobalExecutor() 38 | // app.logger.debug("Tried to install SwiftNIO's EventLoopGroup as Swift's global concurrency executor", metadata: ["success": .stringConvertible(executorTakeoverSuccess)]) 39 | 40 | do { 41 | try await configure(app) 42 | try await app.execute() 43 | } catch { 44 | app.logger.report(error: error) 45 | try? await app.asyncShutdown() 46 | throw error 47 | } 48 | try await app.asyncShutdown() 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Documentation.docc/GettingStarted.md: -------------------------------------------------------------------------------- 1 | # Getting Started with SparkConnect 2 | 3 | A step-by-step guide to get started with SparkConnect for Swift. 4 | 5 | ## Installation 6 | 7 | Add SparkConnect to your Swift package dependencies: 8 | 9 | ```swift 10 | dependencies: [ 11 | .package(url: "https://github.com/apache/spark-connect-swift.git", from: "main") 12 | ] 13 | ``` 14 | 15 | Then, add it to your target dependencies: 16 | 17 | ```swift 18 | targets: [ 19 | .target( 20 | name: "YourApp", 21 | dependencies: ["SparkConnect"] 22 | ) 23 | ] 24 | ``` 25 | 26 | ## Prerequisites 27 | 28 | - Swift 6.2 or later 29 | - macOS 15+, iOS 18+, watchOS 11+, or tvOS 18+ 30 | - A running Apache Spark cluster with Spark Connect enabled 31 | 32 | ## Basic Usage 33 | 34 | ### 1. Create a SparkSession 35 | 36 | ```swift 37 | import SparkConnect 38 | 39 | let spark = try await SparkSession 40 | .builder 41 | .appName("MySwiftApp") 42 | .remote("sc://localhost:15002") 43 | .getOrCreate() 44 | ``` 45 | 46 | ### 2. DataFrame Operations 47 | 48 | ```swift 49 | // From a range 50 | let df1 = try await spark.range(1, 10) 51 | 52 | // Show data 53 | try await df1.show() 54 | 55 | // Select columns 56 | try await df1.select("id").show() 57 | 58 | let df2 = await df1.selectExpr("id", "id % 3 as value") 59 | try await df2.show() 60 | 61 | // Filter data 62 | try await df2.filter("value == 0").show() 63 | 64 | // Group and aggregate 65 | try await df2 66 | .groupBy("value") 67 | .agg("count(*)", "sum(value)") 68 | .show() 69 | ``` 70 | 71 | ### 3. SQL Queries 72 | 73 | ```swift 74 | // Register a temporary view 75 | try await df2.createOrReplaceTempView("v1") 76 | 77 | // Run SQL Queries 78 | let result = try await spark.sql(""" 79 | SELECT id, sum(value) as value_sum 80 | FROM v1 81 | GROUP BY id 82 | ORDER BY value_sum DESC 83 | """) 84 | 85 | result.show() 86 | ``` 87 | 88 | ### 4. Reading and Writing Data 89 | 90 | ```swift 91 | // Read CSV 92 | let csvDf = spark.read 93 | .option("header", "true") 94 | .option("inferSchema", "true") 95 | .csv("path/to/data.csv") 96 | 97 | // Write ORC 98 | csvDf.write 99 | .mode("overwrite") 100 | .orc("path/to/output") 101 | ``` 102 | -------------------------------------------------------------------------------- /Package.swift: -------------------------------------------------------------------------------- 1 | // swift-tools-version: 6.2 2 | // The swift-tools-version declares the minimum version of Swift required to build this package. 3 | // 4 | // Licensed to the Apache Software Foundation (ASF) under one 5 | // or more contributor license agreements. See the NOTICE file 6 | // distributed with this work for additional information 7 | // regarding copyright ownership. The ASF licenses this file 8 | // to you under the Apache License, Version 2.0 (the 9 | // "License"); you may not use this file except in compliance 10 | // with the License. You may obtain a copy of the License at 11 | // 12 | // http://www.apache.org/licenses/LICENSE-2.0 13 | // 14 | // Unless required by applicable law or agreed to in writing, 15 | // software distributed under the License is distributed on an 16 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 17 | // KIND, either express or implied. See the License for the 18 | // specific language governing permissions and limitations 19 | // under the License. 20 | // 21 | import PackageDescription 22 | 23 | let package = Package( 24 | name: "SparkConnect", 25 | platforms: [ 26 | .macOS(.v15), 27 | .iOS(.v18), 28 | .watchOS(.v11), 29 | .tvOS(.v18), 30 | ], 31 | products: [ 32 | .library( 33 | name: "SparkConnect", 34 | targets: ["SparkConnect"]) 35 | ], 36 | dependencies: [ 37 | .package(url: "https://github.com/grpc/grpc-swift-2.git", exact: "2.2.0"), 38 | .package(url: "https://github.com/grpc/grpc-swift-protobuf.git", exact: "2.1.1"), 39 | .package(url: "https://github.com/grpc/grpc-swift-nio-transport.git", exact: "2.3.0"), 40 | .package(url: "https://github.com/google/flatbuffers.git", branch: "v25.9.23"), 41 | ], 42 | targets: [ 43 | .target( 44 | name: "SparkConnect", 45 | dependencies: [ 46 | .product(name: "GRPCCore", package: "grpc-swift-2"), 47 | .product(name: "GRPCProtobuf", package: "grpc-swift-protobuf"), 48 | .product(name: "GRPCNIOTransportHTTP2", package: "grpc-swift-nio-transport"), 49 | .product(name: "FlatBuffers", package: "flatbuffers"), 50 | ], 51 | resources: [ 52 | .process("Documentation.docc") 53 | ] 54 | ), 55 | .testTarget( 56 | name: "SparkConnectTests", 57 | dependencies: ["SparkConnect"], 58 | resources: [ 59 | .copy("Resources/queries") 60 | ] 61 | ), 62 | ] 63 | ) 64 | -------------------------------------------------------------------------------- /Examples/stream/README.md: -------------------------------------------------------------------------------- 1 | # A Swift Network Word Count Application with Apache Spark Connect Swift Client 2 | 3 | This is an example Swift stream processing application to show how to count words with Apache Spark Connect Swift Client library. 4 | 5 | ## Run `Spark Connect Server` 6 | 7 | ```bash 8 | docker run --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait -c spark.log.level=ERROR" 9 | ``` 10 | 11 | ## Run `Netcat` as a streaming input server 12 | 13 | You will first need to run Netcat (a small utility found in most Unix-like systems) as a data server by using 14 | 15 | ```bash 16 | nc -lk 9999 17 | ``` 18 | 19 | ## Build and run from docker image 20 | 21 | Build an application Docker image. 22 | 23 | ```bash 24 | $ docker build -t apache/spark-connect-swift:stream . 25 | $ docker images apache/spark-connect-swift:stream 26 | REPOSITORY TAG IMAGE ID CREATED SIZE 27 | apache/spark-connect-swift stream a4daa10ad9c5 7 seconds ago 369MB 28 | ``` 29 | 30 | Run `stream` docker image. 31 | 32 | ```bash 33 | docker run --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 -e TARGET_HOST=host.docker.internal apache/spark-connect-swift:stream 34 | ``` 35 | 36 | ## Send input and check output 37 | 38 | Then, any lines typed in the terminal running the `Netcat` server will be counted and printed on screen every second. 39 | 40 | ```bash 41 | $ nc -lk 9999 42 | apache spark 43 | apache hadoop 44 | ``` 45 | 46 | `Spark Connect Server` output will look something like the following. 47 | 48 | ```bash 49 | ------------------------------------------- 50 | Batch: 0 51 | ------------------------------------------- 52 | +----+--------+ 53 | |word|count(1)| 54 | +----+--------+ 55 | +----+--------+ 56 | 57 | ------------------------------------------- 58 | Batch: 1 59 | ------------------------------------------- 60 | +------+--------+ 61 | | word|count(1)| 62 | +------+--------+ 63 | |apache| 1| 64 | | spark| 1| 65 | +------+--------+ 66 | 67 | ------------------------------------------- 68 | Batch: 2 69 | ------------------------------------------- 70 | +------+--------+ 71 | | word|count(1)| 72 | +------+--------+ 73 | |apache| 2| 74 | | spark| 1| 75 | |hadoop| 1| 76 | +------+--------+ 77 | ``` 78 | 79 | ## Run from source code 80 | 81 | ```bash 82 | $ swift run 83 | ... 84 | Connected to Apache Spark 4.0.1 Server 85 | ``` 86 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ArrowSchema.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class ArrowField { 22 | public let type: ArrowType 23 | public let name: String 24 | public let isNullable: Bool 25 | 26 | init(_ name: String, type: ArrowType, isNullable: Bool) { 27 | self.name = name 28 | self.type = type 29 | self.isNullable = isNullable 30 | } 31 | } 32 | 33 | /// @nodoc 34 | public class ArrowSchema { 35 | public let fields: [ArrowField] 36 | public let fieldLookup: [String: Int] 37 | init(_ fields: [ArrowField]) { 38 | var fieldLookup = [String: Int]() 39 | for (index, field) in fields.enumerated() { 40 | fieldLookup[field.name] = index 41 | } 42 | 43 | self.fields = fields 44 | self.fieldLookup = fieldLookup 45 | } 46 | 47 | public func field(_ index: Int) -> ArrowField { 48 | return self.fields[index] 49 | } 50 | 51 | public func fieldIndex(_ name: String) -> Int? { 52 | return self.fieldLookup[name] 53 | } 54 | 55 | public class Builder { 56 | private var fields: [ArrowField] = [] 57 | 58 | public init() {} 59 | 60 | @discardableResult 61 | public func addField(_ field: ArrowField) -> Builder { 62 | fields.append(field) 63 | return self 64 | } 65 | 66 | @discardableResult 67 | public func addField(_ name: String, type: ArrowType, isNullable: Bool) -> Builder { 68 | fields.append(ArrowField(name, type: type, isNullable: isNullable)) 69 | return self 70 | } 71 | 72 | public func finish() -> ArrowSchema { 73 | return ArrowSchema(fields) 74 | } 75 | } 76 | } 77 | -------------------------------------------------------------------------------- /Sources/SparkConnect/CaseInsensitiveDictionary.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | /// A dictionary in which keys are case insensitive. The input dictionary can be 22 | /// accessed for cases where case-sensitive information is required. 23 | public struct CaseInsensitiveDictionary: Sendable { 24 | public var originalDictionary: [String: Sendable] 25 | private var keyLowerCasedDictionary: [String: Sendable] = [:] 26 | 27 | init(_ originalDictionary: [String: Sendable] = [:]) { 28 | self.originalDictionary = originalDictionary 29 | for (key, value) in originalDictionary { 30 | keyLowerCasedDictionary[key.lowercased()] = value 31 | } 32 | } 33 | 34 | subscript(key: String) -> Sendable? { 35 | get { 36 | return keyLowerCasedDictionary[key.lowercased()] 37 | } 38 | set { 39 | let lowerKey = key.lowercased() 40 | if let newValue = newValue { 41 | keyLowerCasedDictionary[lowerKey] = newValue 42 | } else { 43 | keyLowerCasedDictionary.removeValue(forKey: lowerKey) 44 | } 45 | originalDictionary = originalDictionary.filter { $0.key.lowercased() != lowerKey } 46 | if let newValue = newValue { 47 | originalDictionary[key] = newValue 48 | } 49 | } 50 | } 51 | 52 | public func toDictionary() -> [String: Sendable] { 53 | return originalDictionary 54 | } 55 | 56 | public func toStringDictionary() -> [String: String] { 57 | return originalDictionary.mapValues { String(describing: $0) } 58 | } 59 | 60 | public var count: Int { 61 | return keyLowerCasedDictionary.count 62 | } 63 | } 64 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/DataStreamTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `DataStreamReader` and `DataStreamWriter` 25 | @Suite(.serialized) 26 | struct DataStreamTests { 27 | @Test 28 | func query() async throws { 29 | let spark = try await SparkSession.builder.getOrCreate() 30 | 31 | // Prepare directories 32 | let input = "/tmp/input-" + UUID().uuidString 33 | let checkpoint = "/tmp/checkpoint-" + UUID().uuidString 34 | let output = "/tmp/output-" + UUID().uuidString 35 | try await spark.range(2025).write.orc(input) 36 | 37 | // Create a streaming dataframe. 38 | let df = 39 | try await spark 40 | .readStream 41 | .schema("id LONG") 42 | .orc(input) 43 | #expect(try await df.isStreaming()) 44 | 45 | // Processing 46 | let df2 = await df.selectExpr("id", "id * 10 as value") 47 | 48 | // Start a streaming query 49 | let query = 50 | try await df2 51 | .writeStream 52 | .option("checkpointLocation", checkpoint) 53 | .outputMode("append") 54 | .format("orc") 55 | .trigger(Trigger.ProcessingTime(1000)) 56 | .start(output) 57 | #expect(try await query.isActive) 58 | // Wait for processing 59 | try await Task.sleep(nanoseconds: 2_000_000_000) 60 | 61 | try await query.stop() 62 | #expect(try await query.isActive == false) 63 | 64 | let df3 = await spark.read.orc(output) 65 | #expect(try await df3.dtypes.count == 2) 66 | #expect(try await df3.count() == 2025) 67 | await spark.stop() 68 | } 69 | } 70 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ArrowData.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class ArrowData { 22 | public let type: ArrowType 23 | public let buffers: [ArrowBuffer] 24 | public let children: [ArrowData] 25 | public let nullCount: UInt 26 | public let length: UInt 27 | public let stride: Int 28 | 29 | convenience init(_ arrowType: ArrowType, buffers: [ArrowBuffer], nullCount: UInt) throws { 30 | try self.init( 31 | arrowType, buffers: buffers, 32 | children: [ArrowData](), nullCount: nullCount, 33 | length: buffers[1].length) 34 | } 35 | 36 | init( 37 | _ arrowType: ArrowType, buffers: [ArrowBuffer], children: [ArrowData], nullCount: UInt, 38 | length: UInt 39 | ) throws { 40 | let infoType = arrowType.info 41 | switch infoType { 42 | case .primitiveInfo(let typeId): 43 | if typeId == ArrowTypeId.unknown { 44 | throw ArrowError.unknownType("Unknown primitive type for data") 45 | } 46 | case .variableInfo(let typeId): 47 | if typeId == ArrowTypeId.unknown { 48 | throw ArrowError.unknownType("Unknown variable type for data") 49 | } 50 | case .timeInfo(let typeId): 51 | if typeId == ArrowTypeId.unknown { 52 | throw ArrowError.unknownType("Unknown time type for data") 53 | } 54 | case .complexInfo(let typeId): 55 | if typeId == ArrowTypeId.unknown { 56 | throw ArrowError.unknownType("Unknown complex type for data") 57 | } 58 | } 59 | 60 | self.type = arrowType 61 | self.buffers = buffers 62 | self.children = children 63 | self.nullCount = nullCount 64 | self.length = length 65 | self.stride = arrowType.getStride() 66 | } 67 | 68 | public func isNull(_ at: UInt) -> Bool { 69 | let nullBuffer = buffers[0] 70 | return nullBuffer.length > 0 && !BitUtility.isSet(at, buffer: nullBuffer) 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/SparkFileUtilsTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for `SparkFileUtils` 26 | @Suite(.serialized) 27 | struct SparkFileUtilsTests { 28 | let fm = FileManager.default 29 | 30 | @Test 31 | func resolveURI() async throws { 32 | let fileNameURL = SparkFileUtils.resolveURL("jar1") 33 | #expect(fileNameURL!.absoluteString == "file://\(fm.currentDirectoryPath)/jar1") 34 | 35 | let homeUrl = SparkFileUtils.resolveURL("~/jar1") 36 | #expect(homeUrl!.absoluteString == "\(fm.homeDirectoryForCurrentUser.absoluteString)jar1") 37 | 38 | let absolutePath = SparkFileUtils.resolveURL("file:/jar1") 39 | #expect(absolutePath!.absoluteString == "file:/jar1") 40 | 41 | let hdfsPath = SparkFileUtils.resolveURL("hdfs:/root/spark.jar") 42 | #expect(hdfsPath!.absoluteString == "hdfs:/root/spark.jar") 43 | 44 | let s3aPath = SparkFileUtils.resolveURL("s3a:/bucket/spark.jar") 45 | #expect(s3aPath!.absoluteString == "s3a:/bucket/spark.jar") 46 | } 47 | 48 | @Test 49 | func directory() async throws { 50 | // This tests three functions. 51 | // createTempDir -> createDirectory(root: String, namePrefix: String = "spark") 52 | // -> createDirectory(at: URL) 53 | let dir = SparkFileUtils.createTempDir() 54 | 55 | var isDir: ObjCBool = false 56 | let exists = fm.fileExists(atPath: dir.path(), isDirectory: &isDir) 57 | #expect(exists && isDir.boolValue) 58 | 59 | #expect(SparkFileUtils.recursiveList(directory: dir).isEmpty) 60 | 61 | let emptyData = Data() 62 | try emptyData.write(to: URL(string: dir.absoluteString + "/1")!) 63 | 64 | #expect(SparkFileUtils.recursiveList(directory: dir).count == 1) 65 | 66 | try SparkFileUtils.deleteRecursively(dir) 67 | 68 | #expect(!fm.fileExists(atPath: dir.path(), isDirectory: &isDir)) 69 | } 70 | } 71 | -------------------------------------------------------------------------------- /Examples/spark-sql/README.md: -------------------------------------------------------------------------------- 1 | # A `Spark SQL REPL` Application with Apache Spark Connect Swift Client 2 | 3 | This is an example Swift application to show how to develop a Spark SQL REPL(Read-eval-print Loop) with Apache Spark Connect Swift Client library. 4 | 5 | ## How to run 6 | 7 | Prepare `Spark Connect Server` via running Docker image. 8 | 9 | ```bash 10 | docker run -it --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait" 11 | ``` 12 | 13 | Build an application Docker image. 14 | 15 | ```bash 16 | $ docker build -t apache/spark-connect-swift:spark-sql . 17 | $ docker images apache/spark-connect-swift:spark-sql 18 | REPOSITORY TAG IMAGE ID CREATED SIZE 19 | apache/spark-connect-swift spark-sql 265ddfec650d 7 seconds ago 390MB 20 | ``` 21 | 22 | Run `spark-sql` docker image. 23 | 24 | ```bash 25 | $ docker run -it --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:spark-sql 26 | Connected to Apache Spark 4.0.1 Server 27 | spark-sql (default)> SHOW DATABASES; 28 | +---------+ 29 | |namespace| 30 | +---------+ 31 | |default | 32 | +---------+ 33 | 34 | Time taken: 30 ms 35 | spark-sql (default)> CREATE DATABASE db1; 36 | ++ 37 | || 38 | ++ 39 | ++ 40 | 41 | Time taken: 31 ms 42 | spark-sql (default)> USE db1; 43 | ++ 44 | || 45 | ++ 46 | ++ 47 | 48 | Time taken: 27 ms 49 | spark-sql (db1)> CREATE TABLE t1 AS SELECT * FROM RANGE(10); 50 | ++ 51 | || 52 | ++ 53 | ++ 54 | 55 | Time taken: 99 ms 56 | spark-sql (db1)> SELECT * FROM t1; 57 | +---+ 58 | | id| 59 | +---+ 60 | | 1| 61 | | 5| 62 | | 3| 63 | | 0| 64 | | 6| 65 | | 9| 66 | | 4| 67 | | 8| 68 | | 7| 69 | | 2| 70 | +---+ 71 | 72 | Time taken: 80 ms 73 | spark-sql (db1)> USE default; 74 | ++ 75 | || 76 | ++ 77 | ++ 78 | 79 | Time taken: 26 ms 80 | spark-sql (default)> DROP DATABASE db1 CASCADE; 81 | ++ 82 | || 83 | ++ 84 | ++ 85 | spark-sql (default)> exit; 86 | ``` 87 | 88 | Apache Spark 4 supports [SQL Pipe Syntax](https://spark.apache.org/docs/4.0.1/sql-pipe-syntax.html). 89 | 90 | ```bash 91 | $ swift run 92 | ... 93 | Build of product 'SparkSQLRepl' complete! (2.33s) 94 | Connected to Apache Spark 4.0.1 Server 95 | spark-sql (default)> 96 | FROM ORC.`/opt/spark/examples/src/main/resources/users.orc` 97 | |> AGGREGATE COUNT(*) cnt 98 | GROUP BY name 99 | |> ORDER BY cnt DESC, name ASC 100 | ; 101 | +------+---+ 102 | | name|cnt| 103 | +------+---+ 104 | |Alyssa| 1| 105 | | Ben| 1| 106 | +------+---+ 107 | 108 | Time taken: 159 ms 109 | ``` 110 | 111 | Run from source code. 112 | 113 | ```bash 114 | $ swift run 115 | ... 116 | Connected to Apache Spark 4.0.1 Server 117 | spark-sql (default)> 118 | ``` 119 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/CaseInsensitiveDictionaryTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for `CaseInsensitiveDictionary` 26 | @Suite(.serialized) 27 | struct CaseInsensitiveDictionaryTests { 28 | @Test 29 | func empty() async throws { 30 | let dict = CaseInsensitiveDictionary([:]) 31 | #expect(dict.count == 0) 32 | } 33 | 34 | @Test 35 | func originalDictionary() async throws { 36 | let dict = CaseInsensitiveDictionary([ 37 | "key1": "value1", 38 | "KEY1": "VALUE1", 39 | ]) 40 | #expect(dict.count == 1) 41 | #expect(dict.originalDictionary.count == 2) 42 | } 43 | 44 | @Test 45 | func toDictionary() async throws { 46 | let dict = CaseInsensitiveDictionary([ 47 | "key1": "value1", 48 | "KEY1": "VALUE1", 49 | ]) 50 | #expect(dict.toDictionary().count == 2) 51 | } 52 | 53 | @Test 54 | func `subscript`() async throws { 55 | var dict = CaseInsensitiveDictionary([:]) 56 | #expect(dict.count == 0) 57 | 58 | dict["KEY1"] = "value1" 59 | #expect(dict.count == 1) 60 | #expect(dict["key1"] as! String == "value1") 61 | #expect(dict["KEY1"] as! String == "value1") 62 | #expect(dict["KeY1"] as! String == "value1") 63 | 64 | dict["key2"] = false 65 | #expect(dict.count == 2) 66 | #expect(dict["kEy2"] as! Bool == false) 67 | 68 | dict["key3"] = 2025 69 | #expect(dict.count == 3) 70 | #expect(dict["key3"] as! Int == 2025) 71 | } 72 | 73 | @Test 74 | func updatedOriginalDictionary() async throws { 75 | var dict = CaseInsensitiveDictionary([ 76 | "key1": "value1", 77 | "KEY1": "VALUE1", 78 | ]) 79 | #expect(dict.count == 1) 80 | #expect(dict.originalDictionary.count == 2) 81 | 82 | dict["KEY1"] = "Swift" 83 | #expect(dict["KEY1"] as! String == "Swift") 84 | #expect(dict.count == 1) 85 | #expect(dict.originalDictionary.count == 1) 86 | #expect(dict.toDictionary().count == 1) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/DataFrameInternalTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Testing 21 | 22 | @testable import SparkConnect 23 | 24 | /// A test suite for `DataFrame` internal APIs 25 | @Suite(.serialized) 26 | struct DataFrameInternalTests { 27 | 28 | @Test 29 | func showString() async throws { 30 | let spark = try await SparkSession.builder.getOrCreate() 31 | let rows = try await spark.range(10).showString(2, 0, false).collect() 32 | #expect(rows.count == 1) 33 | #expect(rows[0].length == 1) 34 | #expect( 35 | try (rows[0].get(0) as! String).trimmingCharacters(in: .whitespacesAndNewlines) == """ 36 | +---+ 37 | |id | 38 | +---+ 39 | |0 | 40 | |1 | 41 | +---+ 42 | only showing top 2 rows 43 | """) 44 | await spark.stop() 45 | } 46 | 47 | @Test 48 | func showStringTruncate() async throws { 49 | let spark = try await SparkSession.builder.getOrCreate() 50 | let rows = try await spark.sql("SELECT * FROM VALUES ('abc', 'def'), ('ghi', 'jkl')") 51 | .showString(2, 2, false).collect() 52 | #expect(rows.count == 1) 53 | #expect(rows[0].length == 1) 54 | print(try rows[0].get(0) as! String) 55 | #expect( 56 | try rows[0].get(0) as! String == """ 57 | +----+----+ 58 | |col1|col2| 59 | +----+----+ 60 | | ab| de| 61 | | gh| jk| 62 | +----+----+ 63 | 64 | """) 65 | await spark.stop() 66 | } 67 | 68 | @Test 69 | func showStringVertical() async throws { 70 | let spark = try await SparkSession.builder.getOrCreate() 71 | let rows = try await spark.range(10).showString(2, 0, true).collect() 72 | #expect(rows.count == 1) 73 | #expect(rows[0].length == 1) 74 | print(try rows[0].get(0) as! String) 75 | #expect( 76 | try (rows[0].get(0) as! String).trimmingCharacters(in: .whitespacesAndNewlines) == """ 77 | -RECORD 0-- 78 | id | 0 79 | -RECORD 1-- 80 | id | 1 81 | only showing top 2 rows 82 | """) 83 | await spark.stop() 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Row.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | public struct Row: Sendable, Equatable { 22 | let values: [Sendable?] 23 | 24 | public init(_ values: Sendable?...) { 25 | self.values = values 26 | } 27 | 28 | public init(valueArray: [Sendable?]) { 29 | self.values = valueArray 30 | } 31 | 32 | public static var empty: Row { 33 | return Row() 34 | } 35 | 36 | public var size: Int { return length } 37 | 38 | public var length: Int { return values.count } 39 | 40 | subscript(index: Int) -> Sendable { 41 | get throws { 42 | return try get(index) 43 | } 44 | } 45 | 46 | public func get(_ i: Int) throws -> Sendable { 47 | if i < 0 || i >= self.length { 48 | throw SparkConnectError.InvalidArgument 49 | } 50 | return values[i] 51 | } 52 | 53 | public func getAsBool(_ i: Int) throws -> Bool { 54 | return try get(i) as! Bool 55 | } 56 | 57 | public static func == (lhs: Row, rhs: Row) -> Bool { 58 | if lhs.values.count != rhs.values.count { 59 | return false 60 | } 61 | return lhs.values.elementsEqual(rhs.values) { (x, y) in 62 | if x == nil && y == nil { 63 | return true 64 | } else if let a = x as? Bool, let b = y as? Bool { 65 | return a == b 66 | } else if let a = x as? any FixedWidthInteger, let b = y as? any FixedWidthInteger { 67 | return Int64(a) == Int64(b) 68 | } else if let a = x as? Float, let b = y as? Float { 69 | return a == b 70 | } else if let a = x as? Double, let b = y as? Double { 71 | return a == b 72 | } else if let a = x as? Decimal, let b = y as? Decimal { 73 | return a == b 74 | } else if let a = x as? Date, let b = y as? Date { 75 | return a == b 76 | } else if let a = x as? String, let b = y as? String { 77 | return a == b 78 | } else { 79 | return false 80 | } 81 | } 82 | } 83 | 84 | public func toString() -> String { 85 | return "[\(self.values.map { "\($0 ?? "null")" }.joined(separator: ","))]" 86 | } 87 | } 88 | 89 | extension Row { 90 | } 91 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/SQLHelper.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test utility 26 | struct SQLHelper { 27 | public static func withDatabase(_ spark: SparkSession, _ dbNames: String...) -> ( 28 | () async throws -> Void 29 | ) async throws -> Void { 30 | func body(_ f: () async throws -> Void) async throws { 31 | try await ErrorUtils.tryWithSafeFinally( 32 | f, 33 | { 34 | for name in dbNames { 35 | try await spark.sql("DROP DATABASE IF EXISTS \(name) CASCADE").count() 36 | } 37 | }) 38 | } 39 | return body 40 | } 41 | 42 | public static func withTable(_ spark: SparkSession, _ tableNames: String...) -> ( 43 | () async throws -> Void 44 | ) async throws -> Void { 45 | func body(_ f: () async throws -> Void) async throws { 46 | try await ErrorUtils.tryWithSafeFinally( 47 | f, 48 | { 49 | for name in tableNames { 50 | try await spark.sql("DROP TABLE IF EXISTS \(name)").count() 51 | } 52 | }) 53 | } 54 | return body 55 | } 56 | 57 | public static func withTempView(_ spark: SparkSession, _ viewNames: String...) -> ( 58 | () async throws -> Void 59 | ) async throws -> Void { 60 | func body(_ f: () async throws -> Void) async throws { 61 | try await ErrorUtils.tryWithSafeFinally( 62 | f, 63 | { 64 | for name in viewNames { 65 | try await spark.catalog.dropTempView(name) 66 | } 67 | }) 68 | } 69 | return body 70 | } 71 | 72 | public static func withGlobalTempView(_ spark: SparkSession, _ viewNames: String...) -> ( 73 | () async throws -> Void 74 | ) async throws -> Void { 75 | func body(_ f: () async throws -> Void) async throws { 76 | try await ErrorUtils.tryWithSafeFinally( 77 | f, 78 | { 79 | for name in viewNames { 80 | try await spark.catalog.dropGlobalTempView(name) 81 | } 82 | }) 83 | } 84 | return body 85 | } 86 | } 87 | -------------------------------------------------------------------------------- /Sources/SparkConnect/CRC32.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | 19 | import Foundation 20 | 21 | public struct CRC32 { 22 | 23 | /// Pre-computed CRC32 table 24 | private static let crcTable: [UInt32] = { 25 | var table = [UInt32](repeating: 0, count: 256) 26 | let polynomial: UInt32 = 0xEDB8_8320 // IEEE 802.3 polynomial 27 | 28 | for i in 0..<256 { 29 | var c = UInt32(i) 30 | for _ in 0..<8 { 31 | if (c & 1) == 1 { 32 | c = polynomial ^ (c >> 1) 33 | } else { 34 | c = c >> 1 35 | } 36 | } 37 | table[i] = c 38 | } 39 | return table 40 | }() 41 | 42 | /// Calculates the CRC32 checksum for the given Data. 43 | /// 44 | /// - Parameter data: The Data object for which to calculate the checksum. 45 | /// - Returns: The calculated CRC32 checksum as a UInt32. 46 | public static func checksum(data: Data) -> UInt32 { 47 | var crc: UInt32 = 0xFFFF_FFFF 48 | 49 | data.withUnsafeBytes { (pointer: UnsafeRawBufferPointer) in 50 | for byte in pointer.bindMemory(to: UInt8.self) { 51 | crc = (crc >> 8) ^ crcTable[Int((crc ^ UInt32(byte)) & 0xFF)] 52 | } 53 | } 54 | return ~crc 55 | } 56 | 57 | /// Calculates the CRC32 checksum for the given String. 58 | /// 59 | /// - Parameter string: The String object for which to calculate the checksum. 60 | /// - Parameter encoding: The encoding to use when converting the string to Data (defaults to .utf8). 61 | /// - Returns: The calculated CRC32 checksum as a UInt32. Returns nil if the string cannot be converted to Data. 62 | public static func checksum(string: String, encoding: String.Encoding = .utf8) -> UInt32? { 63 | guard let data = string.data(using: encoding) else { 64 | return nil 65 | } 66 | return checksum(data: data) 67 | } 68 | 69 | /// Calculates the CRC32 checksum for the given array of bytes. 70 | /// 71 | /// - Parameter bytes: The [UInt8] array for which to calculate the checksum. 72 | /// - Returns: The calculated CRC32 checksum as a UInt32. 73 | public static func checksum(bytes: [UInt8]) -> UInt32 { 74 | var crc: UInt32 = 0xFFFF_FFFF 75 | 76 | for byte in bytes { 77 | crc = (crc >> 8) ^ crcTable[Int((crc ^ UInt32(byte)) & 0xFF)] 78 | } 79 | return ~crc 80 | } 81 | } 82 | -------------------------------------------------------------------------------- /.github/PULL_REQUEST_TEMPLATE: -------------------------------------------------------------------------------- 1 | 10 | 11 | ### What changes were proposed in this pull request? 12 | 20 | 21 | 22 | ### Why are the changes needed? 23 | 28 | 29 | 30 | ### Does this PR introduce _any_ user-facing change? 31 | 38 | 39 | 40 | ### How was this patch tested? 41 | 46 | 47 | 48 | ### Was this patch authored or co-authored using generative AI tooling? 49 | 55 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ArrowBuffer.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public class ArrowBuffer { 22 | static let minLength: UInt = 1 << 5 23 | static let maxLength = UInt.max 24 | fileprivate(set) var length: UInt 25 | let capacity: UInt 26 | public let rawPointer: UnsafeMutableRawPointer 27 | let isMemoryOwner: Bool 28 | 29 | init( 30 | length: UInt, capacity: UInt, rawPointer: UnsafeMutableRawPointer, isMemoryOwner: Bool = true 31 | ) { 32 | self.length = length 33 | self.capacity = capacity 34 | self.rawPointer = rawPointer 35 | self.isMemoryOwner = isMemoryOwner 36 | } 37 | 38 | deinit { 39 | if isMemoryOwner { 40 | self.rawPointer.deallocate() 41 | } 42 | } 43 | 44 | func append(to data: inout Data) { 45 | let ptr = UnsafePointer(rawPointer.assumingMemoryBound(to: UInt8.self)) 46 | data.append(ptr, count: Int(capacity)) 47 | } 48 | 49 | static func createEmptyBuffer() -> ArrowBuffer { 50 | return ArrowBuffer( 51 | length: 0, 52 | capacity: 0, 53 | rawPointer: UnsafeMutableRawPointer.allocate(byteCount: 0, alignment: .zero)) 54 | } 55 | 56 | static func createBuffer(_ data: [UInt8], length: UInt) -> ArrowBuffer { 57 | let byteCount = UInt(data.count) 58 | let capacity = alignTo64(byteCount) 59 | let memory = MemoryAllocator(64) 60 | let rawPointer = memory.allocateArray(Int(capacity)) 61 | rawPointer.copyMemory(from: data, byteCount: data.count) 62 | return ArrowBuffer(length: length, capacity: capacity, rawPointer: rawPointer) 63 | } 64 | 65 | static func createBuffer(_ length: UInt, size: UInt, doAlign: Bool = true) -> ArrowBuffer { 66 | let actualLen = max(length, ArrowBuffer.minLength) 67 | let byteCount = size * actualLen 68 | var capacity = byteCount 69 | if doAlign { 70 | capacity = alignTo64(byteCount) 71 | } 72 | 73 | let memory = MemoryAllocator(64) 74 | let rawPointer = memory.allocateArray(Int(capacity)) 75 | rawPointer.initializeMemory(as: UInt8.self, repeating: 0, count: Int(capacity)) 76 | return ArrowBuffer(length: length, capacity: capacity, rawPointer: rawPointer) 77 | } 78 | 79 | static func copyCurrent(_ from: ArrowBuffer, to: inout ArrowBuffer, len: UInt) { 80 | to.rawPointer.copyMemory(from: from.rawPointer, byteCount: Int(len)) 81 | } 82 | 83 | private static func alignTo64(_ length: UInt) -> UInt { 84 | let bufAlignment = length % 64 85 | if bufAlignment != 0 { 86 | return length + (64 - bufAlignment) + 8 87 | } 88 | 89 | return length + 8 90 | } 91 | } 92 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/RowTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `Row` 25 | @Suite(.serialized) 26 | struct RowTests { 27 | @Test 28 | func empty() { 29 | #expect(Row.empty.size == 0) 30 | #expect(Row.empty.length == 0) 31 | #expect(throws: SparkConnectError.InvalidArgument) { 32 | try Row.empty.get(0) 33 | } 34 | } 35 | 36 | @Test 37 | func create() { 38 | #expect(Row(nil).size == 1) 39 | #expect(Row(1).size == 1) 40 | #expect(Row(1.1).size == 1) 41 | #expect(Row(Decimal(1.1)).size == 1) 42 | #expect(Row("a").size == 1) 43 | #expect(Row(nil, 1, 1.1, "a", true).size == 5) 44 | #expect(Row(valueArray: [nil, 1, 1.1, "a", true]).size == 5) 45 | } 46 | 47 | @Test 48 | func string() async throws { 49 | #expect(Row(nil, 1, 1.1, "a", true).toString() == "[null,1,1.1,a,true]") 50 | } 51 | 52 | @Test 53 | func get() throws { 54 | let row = Row(1, 1.1, "a", true, Decimal(1.2)) 55 | #expect(try row.get(0) as! Int == 1) 56 | #expect(try row.get(1) as! Double == 1.1) 57 | #expect(try row.get(2) as! String == "a") 58 | #expect(try row.get(3) as! Bool == true) 59 | #expect(try row.get(4) as! Decimal == Decimal(1.2)) 60 | #expect(throws: SparkConnectError.InvalidArgument) { 61 | try Row.empty.get(-1) 62 | } 63 | } 64 | 65 | @Test 66 | func compare() { 67 | #expect(Row(nil) != Row()) 68 | #expect(Row(nil) == Row(nil)) 69 | 70 | #expect(Row(1) == Row(1)) 71 | #expect(Row(1) != Row(2)) 72 | #expect(Row(1, 2, 3) == Row(1, 2, 3)) 73 | #expect(Row(1, 2, 3) != Row(1, 2, 4)) 74 | 75 | #expect(Row(1.0) == Row(1.0)) 76 | #expect(Row(1.0) != Row(2.0)) 77 | 78 | #expect(Row(Decimal(1.0)) == Row(Decimal(1.0))) 79 | #expect(Row(Decimal(1.0)) != Row(Decimal(2.0))) 80 | 81 | #expect(Row("a") == Row("a")) 82 | #expect(Row("a") != Row("b")) 83 | 84 | #expect(Row(true) == Row(true)) 85 | #expect(Row(true) != Row(false)) 86 | 87 | #expect(Row(1, "a") == Row(1, "a")) 88 | #expect(Row(1, "a") != Row(2, "a")) 89 | #expect(Row(1, "a") != Row(1, "b")) 90 | 91 | #expect(Row(0, 1, 2) == Row(valueArray: [0, 1, 2])) 92 | 93 | #expect(Row(0) == Row(Optional(0))) 94 | #expect(Row(Optional(0)) == Row(Optional(0))) 95 | 96 | #expect([Row(1)] == [Row(1)]) 97 | #expect([Row(1), Row(2)] == [Row(1), Row(2)]) 98 | #expect([Row(1), Row(2)] != [Row(1), Row(3)]) 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/ConstraintTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for new syntaxes from SPARK-51207 (SPIP: Constraints in DSv2) 25 | /// For now, only syntax test is here because Apache Spark 4.1 and the corresponding Apache Iceberg is not released yet. 26 | @Suite(.serialized) 27 | struct ConstraintTests { 28 | 29 | @Test 30 | func primary_key() async throws { 31 | let spark = try await SparkSession.builder.getOrCreate() 32 | if await spark.version.starts(with: "4.1") { 33 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 34 | try await SQLHelper.withTable(spark, tableName)({ 35 | try await spark.sql("CREATE TABLE \(tableName)(a INT, PRIMARY KEY(a)) USING ORC").count() 36 | try await spark.sql("INSERT INTO \(tableName) VALUES (1), (2)").count() 37 | }) 38 | } 39 | await spark.stop() 40 | } 41 | 42 | @Test 43 | func foreign_key() async throws { 44 | let spark = try await SparkSession.builder.getOrCreate() 45 | if await spark.version.starts(with: "4.1") { 46 | let tableName1 = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 47 | let tableName2 = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 48 | try await SQLHelper.withTable(spark, tableName1, tableName2)({ 49 | try await spark.sql("CREATE TABLE \(tableName1)(id INT) USING ORC").count() 50 | try await spark.sql( 51 | "CREATE TABLE \(tableName2)(fk INT, FOREIGN KEY(fk) REFERENCES \(tableName2)(id)) USING ORC" 52 | ).count() 53 | }) 54 | } 55 | await spark.stop() 56 | } 57 | 58 | @Test 59 | func unique() async throws { 60 | let spark = try await SparkSession.builder.getOrCreate() 61 | if await spark.version.starts(with: "4.1") { 62 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 63 | try await SQLHelper.withTable(spark, tableName)({ 64 | try await spark.sql("CREATE TABLE \(tableName)(a INT UNIQUE) USING ORC").count() 65 | try await spark.sql("INSERT INTO \(tableName) VALUES (1), (2)").count() 66 | }) 67 | } 68 | await spark.stop() 69 | } 70 | 71 | @Test 72 | func check() async throws { 73 | let spark = try await SparkSession.builder.getOrCreate() 74 | if await spark.version.starts(with: "4.1") { 75 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 76 | try await SQLHelper.withTable(spark, tableName)({ 77 | try await spark.sql( 78 | "CREATE TABLE \(tableName)(a INT, CONSTRAINT c1 CHECK (a > 0)) USING ORC" 79 | ).count() 80 | try await spark.sql("INSERT INTO \(tableName) VALUES (-1)").count() 81 | }) 82 | } 83 | await spark.stop() 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/Resources/queries/explain.sql.answer: -------------------------------------------------------------------------------- 1 | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 2 | | plan| 3 | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ 4 | |== Parsed Logical Plan ==\n'Aggregate ['k], ['k, unresolvedalias('sum('v))]\n+- SubqueryAlias t\n +- LocalRelation [k#, v#]\n\n== Analyzed Logical Plan ==\nk: int, sum(v): bigint\nAggregate [k#], [k#, sum(v#) AS sum(v)#]\n+- SubqueryAlias t\n +- LocalRelation [k#, v#]\n\n== Optimized Logical Plan ==\nAggregate [k#], [k#, sum(v#) AS sum(v)#]\n+- LocalRelation [k#, v#]\n\n== Physical Plan ==\nAdaptiveSparkPlan isFinalPlan=false\n+- HashAggregate(keys=[k#], functions=[sum(v#)], output=[k#, sum(v)#])\n +- Exchange hashpartitioning(k#, 200), ENSURE_REQUIREMENTS, [plan_id=]\n +- HashAggregate(keys=[k#], functions=[partial_sum(v#)], output=[k#, sum#])\n +- LocalTableScan [k#, v#]\n| 5 | +----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+ -------------------------------------------------------------------------------- /Tests/SparkConnectTests/StreamingQueryManagerTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `StreamingQueryManager` 25 | @Suite(.serialized) 26 | struct StreamingQueryManagerTests { 27 | 28 | @Test 29 | func active() async throws { 30 | let spark = try await SparkSession.builder.getOrCreate() 31 | #expect(try await spark.streams.active.count == 0) 32 | await spark.stop() 33 | } 34 | 35 | @Test 36 | func get() async throws { 37 | let spark = try await SparkSession.builder.getOrCreate() 38 | await #expect(throws: SparkConnectError.InvalidArgument) { 39 | try await spark.streams.get(UUID()) 40 | } 41 | await #expect(throws: SparkConnectError.InvalidArgument) { 42 | try await spark.streams.get(UUID().uuidString) 43 | } 44 | await spark.stop() 45 | } 46 | 47 | @Test 48 | func awaitAnyTermination() async throws { 49 | let spark = try await SparkSession.builder.getOrCreate() 50 | try await spark.streams.awaitAnyTermination(1) 51 | await #expect(throws: SparkConnectError.InvalidArgument) { 52 | try await spark.streams.awaitAnyTermination(-1) 53 | } 54 | await spark.stop() 55 | } 56 | 57 | @Test 58 | func resetTerminated() async throws { 59 | let spark = try await SparkSession.builder.getOrCreate() 60 | try await spark.streams.resetTerminated() 61 | await spark.stop() 62 | } 63 | 64 | @Test 65 | func query() async throws { 66 | let spark = try await SparkSession.builder.getOrCreate() 67 | 68 | // Prepare directories 69 | let input = "/tmp/input-" + UUID().uuidString 70 | let checkpoint = "/tmp/checkpoint-" + UUID().uuidString 71 | let output = "/tmp/output-" + UUID().uuidString 72 | try await spark.range(2025).write.orc(input) 73 | 74 | // Create a streaming dataframe. 75 | let df = 76 | try await spark 77 | .readStream 78 | .schema("id LONG") 79 | .orc(input) 80 | #expect(try await df.isStreaming()) 81 | #expect(try await spark.streams.active.count == 0) 82 | 83 | // Start a streaming query 84 | let query = 85 | try await df 86 | .selectExpr("id", "id * 10 as value") 87 | .writeStream 88 | .option("checkpointLocation", checkpoint) 89 | .outputMode("append") 90 | .format("orc") 91 | .trigger(Trigger.ProcessingTime(1000)) 92 | .start(output) 93 | #expect(try await query.isActive) 94 | #expect(try await spark.streams.active.count == 1) 95 | #expect(try await spark.streams.get(query.id).isActive) 96 | #expect(try await spark.streams.get(query.id.uuidString).isActive) 97 | // Wait for processing 98 | try await Task.sleep(nanoseconds: 2_000_000_000) 99 | 100 | try await query.stop() 101 | #expect(try await spark.streams.active.count == 0) 102 | #expect(try await query.isActive == false) 103 | 104 | await spark.stop() 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/DataFrameWriterV2Tests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `DataFrameWriterV2` 25 | @Suite(.serialized) 26 | struct DataFrameWriterV2Tests { 27 | let icebergEnabled = ProcessInfo.processInfo.environment["SPARK_ICEBERG_TEST_ENABLED"] != nil 28 | 29 | @Test 30 | func create() async throws { 31 | let spark = try await SparkSession.builder.getOrCreate() 32 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 33 | try await SQLHelper.withTable(spark, tableName)({ 34 | let write = try await spark.range(2).writeTo(tableName).using("orc") 35 | try await write.create() 36 | #expect(try await spark.table(tableName).count() == 2) 37 | try await #require(throws: SparkConnectError.TableOrViewAlreadyExists) { 38 | try await write.create() 39 | } 40 | }) 41 | await spark.stop() 42 | } 43 | 44 | @Test 45 | func createOrReplace() async throws { 46 | let spark = try await SparkSession.builder.getOrCreate() 47 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 48 | try await SQLHelper.withTable(spark, tableName)({ 49 | let write = try await spark.range(2).writeTo(tableName).using("orc") 50 | try await write.create() 51 | #expect(try await spark.table(tableName).count() == 2) 52 | if icebergEnabled { 53 | try await write.createOrReplace() 54 | } else { 55 | try await #require(throws: Error.self) { 56 | try await write.createOrReplace() 57 | } 58 | } 59 | }) 60 | await spark.stop() 61 | } 62 | 63 | @Test 64 | func replace() async throws { 65 | let spark = try await SparkSession.builder.getOrCreate() 66 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 67 | try await SQLHelper.withTable(spark, tableName)({ 68 | let write = try await spark.range(2).writeTo(tableName).using("orc") 69 | try await write.create() 70 | #expect(try await spark.table(tableName).count() == 2) 71 | if icebergEnabled { 72 | try await write.replace() 73 | } else { 74 | try await #require(throws: Error.self) { 75 | try await write.replace() 76 | } 77 | } 78 | }) 79 | await spark.stop() 80 | } 81 | 82 | @Test 83 | func append() async throws { 84 | let spark = try await SparkSession.builder.getOrCreate() 85 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 86 | try await SQLHelper.withTable(spark, tableName)({ 87 | let write = try await spark.range(2).writeTo(tableName).using("orc") 88 | try await write.create() 89 | #expect(try await spark.table(tableName).count() == 2) 90 | if icebergEnabled { 91 | try await write.append() 92 | } else { 93 | try await #require(throws: Error.self) { 94 | try await write.append() 95 | } 96 | } 97 | }) 98 | await spark.stop() 99 | } 100 | } 101 | -------------------------------------------------------------------------------- /Sources/SparkConnect/TypeAliases.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | 19 | typealias ActionType = Spark_Connect_MergeAction.ActionType 20 | typealias Aggregate = Spark_Connect_Aggregate 21 | typealias AnalyzePlanRequest = Spark_Connect_AnalyzePlanRequest 22 | typealias AnalyzePlanResponse = Spark_Connect_AnalyzePlanResponse 23 | typealias Command = Spark_Connect_Command 24 | typealias ConfigRequest = Spark_Connect_ConfigRequest 25 | typealias DataSource = Spark_Connect_Read.DataSource 26 | typealias DataType = Spark_Connect_DataType 27 | typealias DayTimeInterval = Spark_Connect_DataType.DayTimeInterval 28 | typealias Drop = Spark_Connect_Drop 29 | typealias ExecutePlanRequest = Spark_Connect_ExecutePlanRequest 30 | typealias ExecutePlanResponse = Spark_Connect_ExecutePlanResponse 31 | typealias ExplainMode = AnalyzePlanRequest.Explain.ExplainMode 32 | typealias ExpressionLiteral = Spark_Connect_Expression.Literal 33 | typealias ExpressionString = Spark_Connect_Expression.ExpressionString 34 | typealias Filter = Spark_Connect_Filter 35 | typealias GroupType = Spark_Connect_Aggregate.GroupType 36 | typealias Join = Spark_Connect_Join 37 | typealias JoinType = Spark_Connect_Join.JoinType 38 | typealias KeyValue = Spark_Connect_KeyValue 39 | typealias LateralJoin = Spark_Connect_LateralJoin 40 | typealias Limit = Spark_Connect_Limit 41 | typealias MapType = Spark_Connect_DataType.Map 42 | typealias MergeAction = Spark_Connect_MergeAction 43 | typealias MergeIntoTableCommand = Spark_Connect_MergeIntoTableCommand 44 | typealias NamedTable = Spark_Connect_Read.NamedTable 45 | typealias OneOf_Analyze = AnalyzePlanRequest.OneOf_Analyze 46 | typealias OneOf_CatType = Spark_Connect_Catalog.OneOf_CatType 47 | typealias OutputType = Spark_Connect_OutputType 48 | typealias Plan = Spark_Connect_Plan 49 | typealias Project = Spark_Connect_Project 50 | typealias Range = Spark_Connect_Range 51 | typealias Read = Spark_Connect_Read 52 | typealias Relation = Spark_Connect_Relation 53 | typealias Repartition = Spark_Connect_Repartition 54 | typealias RepartitionByExpression = Spark_Connect_RepartitionByExpression 55 | typealias Sample = Spark_Connect_Sample 56 | typealias SaveMode = Spark_Connect_WriteOperation.SaveMode 57 | typealias SetOperation = Spark_Connect_SetOperation 58 | typealias SetOpType = SetOperation.SetOpType 59 | typealias ShowString = Spark_Connect_ShowString 60 | typealias SparkConnectService = Spark_Connect_SparkConnectService 61 | typealias Sort = Spark_Connect_Sort 62 | typealias StreamingQueryCommand = Spark_Connect_StreamingQueryCommand 63 | typealias StreamingQueryInstanceId = Spark_Connect_StreamingQueryInstanceId 64 | typealias StreamingQueryManagerCommand = Spark_Connect_StreamingQueryManagerCommand 65 | typealias StructType = Spark_Connect_DataType.Struct 66 | typealias Tail = Spark_Connect_Tail 67 | typealias UserContext = Spark_Connect_UserContext 68 | typealias UnresolvedAttribute = Spark_Connect_Expression.UnresolvedAttribute 69 | typealias WithColumnsRenamed = Spark_Connect_WithColumnsRenamed 70 | typealias WriteOperation = Spark_Connect_WriteOperation 71 | typealias WriteOperationV2 = Spark_Connect_WriteOperationV2 72 | typealias WriteStreamOperationStart = Spark_Connect_WriteStreamOperationStart 73 | typealias YearMonthInterval = Spark_Connect_DataType.YearMonthInterval 74 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/MergeIntoWriterTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `MergeIntoWriter` 25 | /// Since this requires Apache Spark 4 with Iceberg support (SPARK-48794), this suite only tests syntaxes. 26 | @Suite(.serialized) 27 | struct MergeIntoWriterTests { 28 | @Test 29 | func whenMatched() async throws { 30 | let spark = try await SparkSession.builder.getOrCreate() 31 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 32 | try await SQLHelper.withTable(spark, tableName)({ 33 | let mergeInto = try await spark.range(1).mergeInto(tableName, "true") 34 | if await spark.version >= "4.0.0" { 35 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 36 | try await mergeInto.whenMatched().delete().merge() 37 | } 38 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 39 | try await mergeInto.whenMatched("true").delete().merge() 40 | } 41 | } else { 42 | try await #require(throws: Error.self) { 43 | try await mergeInto.whenMatched().delete().merge() 44 | } 45 | try await #require(throws: Error.self) { 46 | try await mergeInto.whenMatched("true").delete().merge() 47 | } 48 | } 49 | }) 50 | await spark.stop() 51 | } 52 | 53 | @Test 54 | func whenNotMatched() async throws { 55 | let spark = try await SparkSession.builder.getOrCreate() 56 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 57 | try await SQLHelper.withTable(spark, tableName)({ 58 | let mergeInto = try await spark.range(1).mergeInto(tableName, "true") 59 | if await spark.version >= "4.0.0" { 60 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 61 | try await mergeInto.whenNotMatched().insertAll().merge() 62 | } 63 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 64 | try await mergeInto.whenNotMatched("true").insertAll().merge() 65 | } 66 | } else { 67 | try await #require(throws: Error.self) { 68 | try await mergeInto.whenNotMatched().insertAll().merge() 69 | } 70 | try await #require(throws: Error.self) { 71 | try await mergeInto.whenNotMatched("true").insertAll().merge() 72 | } 73 | } 74 | }) 75 | await spark.stop() 76 | } 77 | 78 | @Test 79 | func whenNotMatchedBySource() async throws { 80 | let spark = try await SparkSession.builder.getOrCreate() 81 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 82 | try await SQLHelper.withTable(spark, tableName)({ 83 | let mergeInto = try await spark.range(1).mergeInto(tableName, "true") 84 | if await spark.version >= "4.0.0" { 85 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 86 | try await mergeInto.whenNotMatchedBySource().delete().merge() 87 | } 88 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 89 | try await mergeInto.whenNotMatchedBySource("true").delete().merge() 90 | } 91 | } else { 92 | try await #require(throws: Error.self) { 93 | try await mergeInto.whenNotMatchedBySource().delete().merge() 94 | } 95 | try await #require(throws: Error.self) { 96 | try await mergeInto.whenNotMatchedBySource("true").delete().merge() 97 | } 98 | } 99 | }) 100 | await spark.stop() 101 | } 102 | } 103 | -------------------------------------------------------------------------------- /Examples/web/README.md: -------------------------------------------------------------------------------- 1 | # A Swift Application with Apache Spark Connect Swift Client 2 | 3 | This project is designed to illustrate a Swift-based HTTP WebServer with Apache Spark Connect. 4 | 5 | - 6 | 7 | ## Create a Swift project 8 | 9 | ```bash 10 | brew install vapor 11 | vapor new spark-connect-swift-web -n 12 | ``` 13 | 14 | ## Use `Apache Spark Connect Swift Client` package 15 | 16 | ```bash 17 | $ git diff HEAD 18 | diff --git a/Package.swift b/Package.swift 19 | index 2edcc8f..dd918a9 100644 20 | --- a/Package.swift 21 | +++ b/Package.swift 22 | @@ -4,13 +4,14 @@ import PackageDescription 23 | let package = Package( 24 | name: "SparkConnectSwiftWebapp", 25 | platforms: [ 26 | - .macOS(.v13) 27 | + .macOS(.v15) 28 | ], 29 | dependencies: [ 30 | // 💧 A server-side Swift web framework. 31 | .package(url: "https://github.com/vapor/vapor.git", from: "4.115.0"), 32 | // 🔵 Non-blocking, event-driven networking for Swift. Used for custom executors 33 | .package(url: "https://github.com/apple/swift-nio.git", from: "2.65.0"), 34 | + .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main"), 35 | ], 36 | targets: [ 37 | .executableTarget( 38 | @@ -19,6 +20,7 @@ let package = Package( 39 | .product(name: "Vapor", package: "vapor"), 40 | .product(name: "NIOCore", package: "swift-nio"), 41 | .product(name: "NIOPosix", package: "swift-nio"), 42 | + .product(name: "SparkConnect", package: "spark-connect-swift"), 43 | ], 44 | swiftSettings: swiftSettings 45 | ), 46 | diff --git a/Sources/SparkConnectSwiftWeb/configure.swift b/Sources/SparkConnectSwiftWeb/configure.swift 47 | index 7715d7c..eea2f95 100644 48 | --- a/Sources/SparkConnectSwiftWeb/configure.swift 49 | +++ b/Sources/SparkConnectSwiftWeb/configure.swift 50 | @@ -2,6 +2,7 @@ import Vapor 51 | 52 | // configures your application 53 | public func configure(_ app: Application) async throws { 54 | + app.http.server.configuration.hostname = "0.0.0.0" 55 | // uncomment to serve files from /Public folder 56 | // app.middleware.use(FileMiddleware(publicDirectory: app.directory.publicDirectory)) 57 | 58 | diff --git a/Sources/SparkConnectSwiftWeb/routes.swift b/Sources/SparkConnectSwiftWeb/routes.swift 59 | index 2edcc8f..dd918a9 100644 60 | --- a/Sources/SparkConnectSwiftWeb/routes.swift 61 | +++ b/Sources/SparkConnectSwiftWeb/routes.swift 62 | @@ -1,11 +1,21 @@ 63 | import Vapor 64 | +import SparkConnect 65 | 66 | func routes(_ app: Application) throws { 67 | app.get { req async in 68 | - "It works!" 69 | + "Welcome to the Swift world. Say hello!" 70 | } 71 | 72 | app.get("hello") { req async -> String in 73 | - "Hello, world!" 74 | + return await Task { 75 | + do { 76 | + let spark = try await SparkSession.builder.getOrCreate() 77 | + let response = "Hi, this is powered by the Apache Spark \(await spark.version)." 78 | + await spark.stop() 79 | + return response 80 | + } catch { 81 | + return "Fail to connect: \(error)" 82 | + } 83 | + }.value 84 | } 85 | } 86 | ``` 87 | 88 | ## How to run 89 | 90 | Prepare `Spark Connect Server` via running Docker image. 91 | 92 | ```bash 93 | docker run --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait" 94 | ``` 95 | 96 | Build an application Docker image. 97 | 98 | ```bash 99 | $ docker build -t apache/spark-connect-swift:web . 100 | $ docker images apache/spark-connect-swift:web 101 | REPOSITORY TAG IMAGE ID CREATED SIZE 102 | apache/spark-connect-swift web 3fd2422fdbee 27 seconds ago 417MB 103 | ``` 104 | 105 | Run `web` docker image 106 | 107 | ```bash 108 | $ docker run -it --rm -p 8080:8080 -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:web 109 | [ NOTICE ] Server started on http://127.0.0.1:8080 110 | ``` 111 | 112 | Connect to the Swift Web Server to talk with `Apache Spark`. 113 | 114 | ```bash 115 | $ curl http://127.0.0.1:8080/ 116 | Welcome to the Swift world. Say hello!% 117 | 118 | $ curl http://127.0.0.1:8080/hello 119 | Hi, this is powered by the Apache Spark 4.0.1.% 120 | ``` 121 | 122 | Run from source code. 123 | 124 | ```bash 125 | swift run 126 | ``` 127 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | # Apache Spark Connect Client for Swift 2 | 3 | [![GitHub Actions Build](https://github.com/apache/spark-connect-swift/actions/workflows/build_and_test.yml/badge.svg)](https://github.com/apache/spark-connect-swift/blob/main/.github/workflows/build_and_test.yml) 4 | [![Swift Version Compatibility](https://img.shields.io/endpoint?url=https%3A%2F%2Fswiftpackageindex.com%2Fapi%2Fpackages%2Fapache%2Fspark-connect-swift%2Fbadge%3Ftype%3Dswift-versions)](https://swiftpackageindex.com/apache/spark-connect-swift) 5 | [![Platform Compatibility](https://img.shields.io/endpoint?url=https%3A%2F%2Fswiftpackageindex.com%2Fapi%2Fpackages%2Fapache%2Fspark-connect-swift%2Fbadge%3Ftype%3Dplatforms)](https://swiftpackageindex.com/apache/spark-connect-swift) 6 | 7 | This is an experimental Swift library to show how to connect to a remote Apache Spark Connect Server and run SQL statements to manipulate remote data. 8 | 9 | So far, this library project is tracking the upstream changes of [Apache Arrow](https://arrow.apache.org) project's Swift-support. 10 | 11 | ## Resources 12 | 13 | - [Homepage](https://apache.github.io/spark-connect-swift/) 14 | - [Swift Package Index](https://swiftpackageindex.com/apache/spark-connect-swift/) 15 | - [Library Documentation](https://swiftpackageindex.com/apache/spark-connect-swift/main/documentation/sparkconnect) 16 | 17 | ## Requirement 18 | 19 | - [Apache Spark 4.0.1 (September 2025)](https://github.com/apache/spark/releases/tag/v4.0.1) 20 | - [Swift 6.2 (September 2025)](https://swift.org) 21 | - [gRPC Swift 2.2 (November 2025)](https://github.com/grpc/grpc-swift-2/releases/tag/2.2.0) 22 | - [gRPC Swift Protobuf 2.1 (August 2025)](https://github.com/grpc/grpc-swift-protobuf/releases/tag/2.1.1) 23 | - [gRPC Swift NIO Transport 2.3 (November 2025)](https://github.com/grpc/grpc-swift-nio-transport/releases/tag/2.3.0) 24 | - [FlatBuffers v25.9.23 (September 2025)](https://github.com/google/flatbuffers/releases/tag/v25.9.23) 25 | - [Apache Arrow Swift](https://github.com/apache/arrow-swift) 26 | 27 | ## How to use in your apps 28 | 29 | Create a Swift project. 30 | 31 | ```bash 32 | mkdir SparkConnectSwiftApp 33 | cd SparkConnectSwiftApp 34 | swift package init --name SparkConnectSwiftApp --type executable 35 | ``` 36 | 37 | Add `SparkConnect` package to the dependency like the following 38 | 39 | ```bash 40 | $ cat Package.swift 41 | import PackageDescription 42 | 43 | let package = Package( 44 | name: "SparkConnectSwiftApp", 45 | platforms: [ 46 | .macOS(.v15) 47 | ], 48 | dependencies: [ 49 | .package(url: "https://github.com/apache/spark-connect-swift.git", branch: "main") 50 | ], 51 | targets: [ 52 | .executableTarget( 53 | name: "SparkConnectSwiftApp", 54 | dependencies: [.product(name: "SparkConnect", package: "spark-connect-swift")] 55 | ) 56 | ] 57 | ) 58 | ``` 59 | 60 | Use `SparkSession` of `SparkConnect` module in Swift. 61 | 62 | ```bash 63 | $ cat Sources/main.swift 64 | 65 | import SparkConnect 66 | 67 | let spark = try await SparkSession.builder.getOrCreate() 68 | print("Connected to Apache Spark \(await spark.version) Server") 69 | 70 | let statements = [ 71 | "DROP TABLE IF EXISTS t", 72 | "CREATE TABLE IF NOT EXISTS t(a INT) USING ORC", 73 | "INSERT INTO t VALUES (1), (2), (3)", 74 | ] 75 | 76 | for s in statements { 77 | print("EXECUTE: \(s)") 78 | _ = try await spark.sql(s).count() 79 | } 80 | print("SELECT * FROM t") 81 | try await spark.sql("SELECT * FROM t").cache().show() 82 | 83 | try await spark.range(10).filter("id % 2 == 0").write.mode("overwrite").orc("/tmp/orc") 84 | try await spark.read.orc("/tmp/orc").show() 85 | 86 | await spark.stop() 87 | ``` 88 | 89 | Run your Swift application. 90 | 91 | ```bash 92 | $ swift run 93 | ... 94 | Connected to Apache Spark 4.0.1 Server 95 | EXECUTE: DROP TABLE IF EXISTS t 96 | EXECUTE: CREATE TABLE IF NOT EXISTS t(a INT) 97 | EXECUTE: INSERT INTO t VALUES (1), (2), (3) 98 | SELECT * FROM t 99 | +---+ 100 | | a | 101 | +---+ 102 | | 2 | 103 | | 1 | 104 | | 3 | 105 | +---+ 106 | +----+ 107 | | id | 108 | +----+ 109 | | 2 | 110 | | 6 | 111 | | 0 | 112 | | 8 | 113 | | 4 | 114 | +----+ 115 | ``` 116 | 117 | You can find more complete examples including `Spark SQL REPL`, `Web Server` and `Streaming` applications in the [Examples](https://github.com/apache/spark-connect-swift/tree/main/Examples) directory. 118 | 119 | This library also supports `SPARK_REMOTE` environment variable to specify the [Spark Connect connection string](https://spark.apache.org/docs/latest/spark-connect-overview.html#set-sparkremote-environment-variable) in order to provide more options. 120 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/RuntimeConfTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for `RuntimeConf` 26 | @Suite(.serialized) 27 | struct RuntimeConfTests { 28 | let TEST_REMOTE = ProcessInfo.processInfo.environment["SPARK_REMOTE"] ?? "sc://localhost" 29 | 30 | @Test 31 | func get() async throws { 32 | let client = SparkConnectClient(remote: TEST_REMOTE) 33 | try await client.connect(UUID().uuidString) 34 | let conf = RuntimeConf(client) 35 | 36 | #expect(try await !conf.get("spark.app.name").isEmpty) 37 | 38 | try await #require(throws: SparkConnectError.SqlConfNotFound) { 39 | try await conf.get("spark.test.non-exist") 40 | } 41 | 42 | await client.stop() 43 | } 44 | 45 | @Test 46 | func getWithDefault() async throws { 47 | let client = SparkConnectClient(remote: TEST_REMOTE) 48 | try await client.connect(UUID().uuidString) 49 | let conf = RuntimeConf(client) 50 | #expect(try await conf.get("spark.sql.adaptive.customCostEvaluatorClass", "XYZ") == "XYZ") 51 | #expect(try await conf.get("spark.test.non-exist", "my_default") == "my_default") 52 | await client.stop() 53 | } 54 | 55 | @Test 56 | func getOption() async throws { 57 | let client = SparkConnectClient(remote: TEST_REMOTE) 58 | try await client.connect(UUID().uuidString) 59 | let conf = RuntimeConf(client) 60 | #expect(try await conf.getOption("spark.app.name") != nil) 61 | #expect(try await conf.getOption("spark.test.non-exist") == nil) 62 | await client.stop() 63 | } 64 | 65 | @Test 66 | func set() async throws { 67 | let client = SparkConnectClient(remote: TEST_REMOTE) 68 | try await client.connect(UUID().uuidString) 69 | let conf = RuntimeConf(client) 70 | try await conf.set("spark.test.key1", "value1") 71 | #expect(try await conf.get("spark.test.key1") == "value1") 72 | await client.stop() 73 | } 74 | 75 | @Test 76 | func reset() async throws { 77 | let client = SparkConnectClient(remote: TEST_REMOTE) 78 | try await client.connect(UUID().uuidString) 79 | let conf = RuntimeConf(client) 80 | 81 | // Success with a key that doesn't exist 82 | try await conf.unset("spark.test.key1") 83 | 84 | // Make it sure that `spark.test.key1` exists before testing `reset`. 85 | try await conf.set("spark.test.key1", "value1") 86 | #expect(try await conf.get("spark.test.key1") == "value1") 87 | 88 | try await conf.unset("spark.test.key1") 89 | try await #require(throws: SparkConnectError.SqlConfNotFound) { 90 | try await conf.get("spark.test.key1") 91 | } 92 | 93 | await client.stop() 94 | } 95 | 96 | @Test 97 | func getAll() async throws { 98 | let client = SparkConnectClient(remote: TEST_REMOTE) 99 | try await client.connect(UUID().uuidString) 100 | let conf = RuntimeConf(client) 101 | let map = try await conf.getAll() 102 | #expect(map.count > 0) 103 | #expect(map["spark.app.id"] != nil) 104 | #expect(map["spark.app.startTime"] != nil) 105 | #expect(map["spark.executor.id"] == "driver") 106 | #expect(map["spark.master"] != nil) 107 | await client.stop() 108 | } 109 | 110 | @Test 111 | func isModifiable() async throws { 112 | let client = SparkConnectClient(remote: TEST_REMOTE) 113 | try await client.connect(UUID().uuidString) 114 | let conf = RuntimeConf(client) 115 | #expect(try await conf.isModifiable("spark.sql.adaptive.customCostEvaluatorClass")) 116 | #expect(try await conf.isModifiable("spark.sql.warehouse.dir") == false) 117 | #expect(try await conf.isModifiable("spark.test.non-exist") == false) 118 | await client.stop() 119 | } 120 | } 121 | -------------------------------------------------------------------------------- /Sources/SparkConnect/SparkFileUtils.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | /// Utility functions like `org.apache.spark.util.SparkFileUtils`. 22 | public enum SparkFileUtils { 23 | 24 | /// Return a well-formed URL for the file described by a user input string. 25 | /// 26 | /// If the supplied path does not contain a scheme, or is a relative path, it will be 27 | /// converted into an absolute path with a file:// scheme. 28 | /// 29 | /// - Parameter path: A path string. 30 | /// - Returns: An URL 31 | static func resolveURL(_ path: String) -> URL? { 32 | if let url = URL(string: path) { 33 | if url.scheme != nil { 34 | return url.absoluteURL 35 | } 36 | 37 | // make sure to handle if the path has a fragment (applies to yarn 38 | // distributed cache) 39 | if let fragment = url.fragment { 40 | var components = URLComponents() 41 | components.scheme = "file" 42 | components.path = (path as NSString).expandingTildeInPath 43 | components.fragment = fragment 44 | return components.url?.absoluteURL 45 | } 46 | } 47 | return URL(fileURLWithPath: (path as NSString).expandingTildeInPath).absoluteURL 48 | } 49 | 50 | /// Lists files recursively. 51 | /// - Parameter directory: <#directory description#> 52 | /// - Returns: <#description#> 53 | static func recursiveList(directory: URL) -> [URL] { 54 | let fileManager = FileManager.default 55 | var results: [URL] = [] 56 | if let enumerator = fileManager.enumerator(at: directory, includingPropertiesForKeys: nil) { 57 | for case let fileURL as URL in enumerator { 58 | results.append(fileURL) 59 | } 60 | } 61 | return results 62 | } 63 | 64 | /// Create a directory given the abstract pathname 65 | /// - Parameter url: An URL location. 66 | /// - Returns: Return true if the directory is successfully created; otherwise, return false. 67 | @discardableResult 68 | static func createDirectory(at url: URL) -> Bool { 69 | let fileManager = FileManager.default 70 | do { 71 | try fileManager.createDirectory(at: url, withIntermediateDirectories: true) 72 | var isDir: ObjCBool = false 73 | let exists = fileManager.fileExists(atPath: url.path, isDirectory: &isDir) 74 | return exists && isDir.boolValue 75 | } catch { 76 | print("Failed to create directory: \(url.path), error: \(error)") 77 | return false 78 | } 79 | } 80 | 81 | /// Create a temporary directory inside the given parent directory. 82 | /// - Parameters: 83 | /// - root: A parent directory. 84 | /// - namePrefix: A prefix for a new directory name. 85 | /// - Returns: An URL for the created directory 86 | static func createDirectory(root: String, namePrefix: String = "spark") -> URL { 87 | let tempDir = URL(fileURLWithPath: root).appendingPathComponent( 88 | "\(namePrefix)-\(UUID().uuidString)") 89 | createDirectory(at: tempDir) 90 | return tempDir 91 | } 92 | 93 | /// Create a new temporary directory prefixed with `spark` inside ``NSTemporaryDirectory``. 94 | /// - Returns: An URL for the created directory 95 | static func createTempDir() -> URL { 96 | let dir = createDirectory(root: NSTemporaryDirectory(), namePrefix: "spark") 97 | 98 | return dir 99 | } 100 | 101 | /// Delete a file or directory and its contents recursively. 102 | /// Throws an exception if deletion is unsuccessful. 103 | /// - Parameter url: An URL location. 104 | static func deleteRecursively(_ url: URL) throws { 105 | let fileManager = FileManager.default 106 | if fileManager.fileExists(atPath: url.path) { 107 | try fileManager.removeItem(at: url) 108 | } else { 109 | throw SparkConnectError.InvalidArgument 110 | } 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ProtoUtil.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | func fromProto( // swiftlint:disable:this cyclomatic_complexity function_body_length 21 | field: org_apache_arrow_flatbuf_Field 22 | ) -> ArrowField { 23 | let type = field.typeType 24 | var arrowType = ArrowType(ArrowType.ArrowUnknown) 25 | switch type { 26 | case .int: 27 | let intType = field.type(type: org_apache_arrow_flatbuf_Int.self)! 28 | let bitWidth = intType.bitWidth 29 | if bitWidth == 8 { 30 | arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt8 : ArrowType.ArrowUInt8) 31 | } else if bitWidth == 16 { 32 | arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt16 : ArrowType.ArrowUInt16) 33 | } else if bitWidth == 32 { 34 | arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt32 : ArrowType.ArrowUInt32) 35 | } else if bitWidth == 64 { 36 | arrowType = ArrowType(intType.isSigned ? ArrowType.ArrowInt64 : ArrowType.ArrowUInt64) 37 | } 38 | case .bool: 39 | arrowType = ArrowType(ArrowType.ArrowBool) 40 | case .floatingpoint: 41 | let floatType = field.type(type: org_apache_arrow_flatbuf_FloatingPoint.self)! 42 | if floatType.precision == .single { 43 | arrowType = ArrowType(ArrowType.ArrowFloat) 44 | } else if floatType.precision == .double { 45 | arrowType = ArrowType(ArrowType.ArrowDouble) 46 | } 47 | case .decimal: 48 | let decimalType = field.type(type: org_apache_arrow_flatbuf_Decimal.self)! 49 | if decimalType.bitWidth == 128 && decimalType.precision <= 38 { 50 | let arrowDecimal128 = ArrowTypeId.decimal128(decimalType.precision, decimalType.scale) 51 | arrowType = ArrowType(ArrowType.Info.primitiveInfo(arrowDecimal128)) 52 | } else { 53 | // Unsupport yet 54 | arrowType = ArrowType(ArrowType.ArrowUnknown) 55 | } 56 | case .utf8: 57 | arrowType = ArrowType(ArrowType.ArrowString) 58 | case .binary: 59 | arrowType = ArrowType(ArrowType.ArrowBinary) 60 | case .date: 61 | let dateType = field.type(type: org_apache_arrow_flatbuf_Date.self)! 62 | if dateType.unit == .day { 63 | arrowType = ArrowType(ArrowType.ArrowDate32) 64 | } else { 65 | arrowType = ArrowType(ArrowType.ArrowDate64) 66 | } 67 | case .time: 68 | let timeType = field.type(type: org_apache_arrow_flatbuf_Time.self)! 69 | if timeType.unit == .second || timeType.unit == .millisecond { 70 | let arrowUnit: ArrowTime32Unit = timeType.unit == .second ? .seconds : .milliseconds 71 | arrowType = ArrowTypeTime32(arrowUnit) 72 | } else { 73 | let arrowUnit: ArrowTime64Unit = timeType.unit == .microsecond ? .microseconds : .nanoseconds 74 | arrowType = ArrowTypeTime64(arrowUnit) 75 | } 76 | case .timestamp: 77 | let timestampType = field.type(type: org_apache_arrow_flatbuf_Timestamp.self)! 78 | let arrowUnit: ArrowTimestampUnit 79 | switch timestampType.unit { 80 | case .second: 81 | arrowUnit = .seconds 82 | case .millisecond: 83 | arrowUnit = .milliseconds 84 | case .microsecond: 85 | arrowUnit = .microseconds 86 | case .nanosecond: 87 | arrowUnit = .nanoseconds 88 | } 89 | 90 | let timezone = timestampType.timezone 91 | arrowType = ArrowTypeTimestamp(arrowUnit, timezone: timezone?.isEmpty == true ? nil : timezone) 92 | case .struct_: 93 | var children = [ArrowField]() 94 | for index in 0.. String { 67 | return try await client.getConf(key) 68 | } 69 | 70 | /// Returns the value of Spark runtime configuration property for the given key. If the key is 71 | /// not set yet, return the user given `value`. This is useful when its default value defined 72 | /// by Apache Spark is not the desired one. 73 | /// - Parameters: 74 | /// - key: A string for the configuration key. 75 | /// - value: A default string value for the configuration. 76 | public func get(_ key: String, _ value: String) async throws -> String { 77 | return try await client.getConfWithDefault(key, value) 78 | } 79 | 80 | /// Get all configurations. 81 | /// - Returns: A map of configuration key-values. 82 | public func getAll() async throws -> [String: String] { 83 | return try await client.getConfAll() 84 | } 85 | 86 | /// Returns the value of Spark runtime configuration property for the given key. If the key is 87 | /// not set yet, return its default value if possible, otherwise `nil` will be returned. 88 | /// - Parameter key: A string for the configuration look-up. 89 | /// - Returns: A string for the configuration or nil. 90 | public func getOption(_ key: String) async throws -> String? { 91 | return try await client.getConfOption(key) 92 | } 93 | 94 | /// Indicates whether the configuration property with the given key is modifiable in the current 95 | /// session. 96 | /// - Parameter key: A string for the configuration look-up. 97 | /// - Returns: `true` if the configuration property is modifiable. For static SQL, Spark Core, invalid 98 | /// (not existing) and other non-modifiable configuration properties, the returned value is 99 | /// `false`. 100 | public func isModifiable(_ key: String) async throws -> Bool { 101 | return try await client.isModifiable(key) 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /Sources/SparkConnect/ChunkedArray.swift: -------------------------------------------------------------------------------- 1 | // Licensed to the Apache Software Foundation (ASF) under one 2 | // or more contributor license agreements. See the NOTICE file 3 | // distributed with this work for additional information 4 | // regarding copyright ownership. The ASF licenses this file 5 | // to you under the Apache License, Version 2.0 (the 6 | // "License"); you may not use this file except in compliance 7 | // with the License. You may obtain a copy of the License at 8 | // 9 | // http://www.apache.org/licenses/LICENSE-2.0 10 | // 11 | // Unless required by applicable law or agreed to in writing, 12 | // software distributed under the License is distributed on an 13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 14 | // KIND, either express or implied. See the License for the 15 | // specific language governing permissions and limitations 16 | // under the License. 17 | 18 | import Foundation 19 | 20 | /// @nodoc 21 | public protocol AnyArray { 22 | var arrowData: ArrowData { get } 23 | func asAny(_ index: UInt) -> Any? 24 | var length: UInt { get } 25 | } 26 | 27 | /// @nodoc 28 | public protocol AsString { 29 | func asString(_ index: UInt) -> String 30 | } 31 | 32 | /// @nodoc 33 | public class ChunkedArrayHolder { 34 | public let type: ArrowType 35 | public let length: UInt 36 | public let nullCount: UInt 37 | public let holder: Any 38 | 39 | public let getBufferData: () -> Result<[Data], ArrowError> 40 | public let getBufferDataSizes: () -> Result<[Int], ArrowError> 41 | public init(_ chunked: ChunkedArray) { // swiftlint:disable:this cyclomatic_complexity 42 | self.holder = chunked 43 | self.length = chunked.length 44 | self.type = chunked.type 45 | self.nullCount = chunked.nullCount 46 | self.getBufferData = { () -> Result<[Data], ArrowError> in 47 | var bufferData = [Data]() 48 | var numBuffers = 2 49 | switch toFBTypeEnum(chunked.type) { 50 | case .success(let fbType): 51 | if !isFixedPrimitive(fbType) { 52 | numBuffers = 3 53 | } 54 | case .failure(let error): 55 | return .failure(error) 56 | } 57 | 58 | for _ in 0.. Result<[Int], ArrowError> in 72 | var bufferDataSizes = [Int]() 73 | var numBuffers = 2 74 | 75 | switch toFBTypeEnum(chunked.type) { 76 | case .success(let fbType): 77 | if !isFixedPrimitive(fbType) { 78 | numBuffers = 3 79 | } 80 | case .failure(let error): 81 | return .failure(error) 82 | } 83 | 84 | for _ in 0..: AsString { 101 | public let arrays: [ArrowArray] 102 | public let type: ArrowType 103 | public let nullCount: UInt 104 | public let length: UInt 105 | public var arrayCount: UInt { return UInt(self.arrays.count) } 106 | 107 | public init(_ arrays: [ArrowArray]) throws { 108 | if arrays.count == 0 { 109 | throw ArrowError.arrayHasNoElements 110 | } 111 | 112 | self.type = arrays[0].arrowData.type 113 | var len: UInt = 0 114 | var nullCount: UInt = 0 115 | for array in arrays { 116 | len += array.length 117 | nullCount += array.nullCount 118 | } 119 | 120 | self.arrays = arrays 121 | self.length = len 122 | self.nullCount = nullCount 123 | } 124 | 125 | public subscript(_ index: UInt) -> T? { 126 | if arrays.count == 0 { 127 | return nil 128 | } 129 | 130 | var localIndex = index 131 | var arrayIndex = 0 132 | var len: UInt = arrays[arrayIndex].length 133 | while localIndex > (len - 1) { 134 | arrayIndex += 1 135 | if arrayIndex > arrays.count { 136 | return nil 137 | } 138 | 139 | localIndex -= len 140 | len = arrays[arrayIndex].length 141 | } 142 | 143 | return arrays[arrayIndex][localIndex] 144 | } 145 | 146 | public func asString(_ index: UInt) -> String { 147 | if self[index] == nil { 148 | return "" 149 | } 150 | 151 | return "\(self[index]!)" 152 | } 153 | } 154 | -------------------------------------------------------------------------------- /Sources/SparkConnect/StorageLevel.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | /// Flags for controlling the storage of an `RDD`. Each ``StorageLevel`` records whether to use memory, 21 | /// or `ExternalBlockStore`, whether to drop the `RDD` to disk if it falls out of memory or 22 | /// `ExternalBlockStore`, whether to keep the data in memory in a serialized format, and whether 23 | /// to replicate the `RDD` partitions on multiple nodes. 24 | public struct StorageLevel: Sendable { 25 | /// Whether the cache should use disk or not. 26 | public var useDisk: Bool 27 | 28 | /// Whether the cache should use memory or not. 29 | public var useMemory: Bool 30 | 31 | /// Whether the cache should use off-heap or not. 32 | public var useOffHeap: Bool 33 | 34 | /// Whether the cached data is deserialized or not. 35 | public var deserialized: Bool 36 | 37 | /// The number of replicas. 38 | public var replication: Int32 39 | 40 | init(useDisk: Bool, useMemory: Bool, useOffHeap: Bool, deserialized: Bool, replication: Int32 = 1) 41 | { 42 | self.useDisk = useDisk 43 | self.useMemory = useMemory 44 | self.useOffHeap = useOffHeap 45 | self.deserialized = deserialized 46 | self.replication = replication 47 | } 48 | 49 | public static let NONE = StorageLevel( 50 | useDisk: false, useMemory: false, useOffHeap: false, deserialized: false) 51 | public static let DISK_ONLY = StorageLevel( 52 | useDisk: true, useMemory: false, useOffHeap: false, deserialized: false) 53 | public static let DISK_ONLY_2 = StorageLevel( 54 | useDisk: true, useMemory: false, useOffHeap: false, deserialized: false, replication: 2) 55 | public static let DISK_ONLY_3 = StorageLevel( 56 | useDisk: true, useMemory: false, useOffHeap: false, deserialized: false, replication: 3) 57 | public static let MEMORY_ONLY = StorageLevel( 58 | useDisk: false, useMemory: true, useOffHeap: false, deserialized: false) 59 | public static let MEMORY_ONLY_2 = StorageLevel( 60 | useDisk: false, useMemory: true, useOffHeap: false, deserialized: false, replication: 2) 61 | public static let MEMORY_AND_DISK = StorageLevel( 62 | useDisk: true, useMemory: true, useOffHeap: false, deserialized: false) 63 | public static let MEMORY_AND_DISK_2 = StorageLevel( 64 | useDisk: true, useMemory: true, useOffHeap: false, deserialized: false, replication: 2) 65 | public static let OFF_HEAP = StorageLevel( 66 | useDisk: true, useMemory: true, useOffHeap: true, deserialized: false) 67 | public static let MEMORY_AND_DISK_DESER = StorageLevel( 68 | useDisk: true, useMemory: true, useOffHeap: false, deserialized: true) 69 | } 70 | 71 | extension StorageLevel { 72 | var toSparkConnectStorageLevel: Spark_Connect_StorageLevel { 73 | var level = Spark_Connect_StorageLevel() 74 | level.useDisk = self.useDisk 75 | level.useMemory = self.useMemory 76 | level.useOffHeap = self.useOffHeap 77 | level.deserialized = self.deserialized 78 | level.replication = self.replication 79 | return level 80 | } 81 | 82 | public static func == (lhs: StorageLevel, rhs: StorageLevel) -> Bool { 83 | return lhs.useDisk == rhs.useDisk && lhs.useMemory == rhs.useMemory 84 | && lhs.useOffHeap == rhs.useOffHeap && lhs.deserialized == rhs.deserialized 85 | && lhs.replication == rhs.replication 86 | } 87 | } 88 | 89 | extension StorageLevel: CustomStringConvertible { 90 | public var description: String { 91 | return 92 | "StorageLevel(useDisk: \(useDisk), useMemory: \(useMemory), useOffHeap: \(useOffHeap), deserialized: \(deserialized), replication: \(replication))" 93 | } 94 | } 95 | 96 | extension Spark_Connect_StorageLevel { 97 | var toStorageLevel: StorageLevel { 98 | return StorageLevel( 99 | useDisk: self.useDisk, 100 | useMemory: self.useMemory, 101 | useOffHeap: self.useOffHeap, 102 | deserialized: self.deserialized, 103 | replication: self.replication 104 | ) 105 | } 106 | } 107 | -------------------------------------------------------------------------------- /Sources/SparkConnect/Documentation.docc/Examples.md: -------------------------------------------------------------------------------- 1 | # Spark Connect Swift Examples 2 | 3 | This document provides an overview of the example applications inside [Examples](https://github.com/apache/spark-connect-swift/tree/main/Examples) directory. These examples demonstrate how to use Spark Connect Swift to interact with Apache Spark in different contexts. 4 | 5 | ## Prepare Spark Connect Server 6 | 7 | Start a Spark Connect Server: 8 | 9 | ```bash 10 | docker run -it --rm -p 15002:15002 apache/spark:4.0.1 bash -c "/opt/spark/sbin/start-connect-server.sh --wait -c spark.log.level=ERROR" 11 | ``` 12 | 13 | ## Basic Application Example 14 | 15 | The basic application example demonstrates fundamental operations with Apache Spark Connect, including: 16 | 17 | - Connecting to a Spark server 18 | - Creating and manipulating tables with SQL 19 | - Using DataFrame operations 20 | - Reading and writing data in the ORC format 21 | 22 | ### Key Features 23 | 24 | - SQL execution for table operations 25 | - DataFrame transformations with filter operations 26 | - Data persistence with ORC format 27 | - Basic session management 28 | 29 | ### How to Run 30 | 31 | Build and run the application: 32 | 33 | ```bash 34 | # Using Docker 35 | docker build -t apache/spark-connect-swift:app . 36 | docker run -it --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:app 37 | 38 | # From source code 39 | swift run 40 | ``` 41 | 42 | ## Spark SQL REPL(Read-Eval-Print Loop) Example 43 | 44 | The Spark SQL REPL application example demonstrates interactive operations with ad-hoc Spark SQL queries with Apache Spark Connect, including: 45 | 46 | - Connecting to a Spark server 47 | - Receiving ad-hoc Spark SQL queries from users 48 | - Show the SQL results interactively 49 | 50 | ### Key Features 51 | 52 | - Spark SQL execution for table operations 53 | - User interactions 54 | 55 | ### How to Run 56 | 57 | Build and run the application: 58 | 59 | ```bash 60 | # Using Docker 61 | docker build -t apache/spark-connect-swift:spark-sql . 62 | docker run -it --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:spark-sql 63 | 64 | # From source code 65 | swift run 66 | ``` 67 | 68 | ## Pi Calculation Example 69 | 70 | The Pi calculation example shows how to use Spark Connect Swift for computational tasks by calculating an approximation of π (pi) using the Monte Carlo method. 71 | 72 | ### Key Features 73 | 74 | - Command-line argument handling 75 | - Mathematical computations with Spark 76 | - Random number generation 77 | - Filtering and counting operations 78 | 79 | ### How to Run 80 | 81 | Build and run the application: 82 | 83 | ```bash 84 | # Using Docker 85 | docker build -t apache/spark-connect-swift:pi . 86 | docker run --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:pi 87 | 88 | # From source code 89 | swift run 90 | ``` 91 | 92 | ## Streaming Example 93 | 94 | The streaming example demonstrates how to process streaming data using Spark Connect Swift client, specifically for counting words from a network socket stream. 95 | 96 | ### Key Features 97 | 98 | - Stream processing with Spark Connect 99 | - Network socket data source 100 | - Word counting with string operations 101 | - Real-time console output 102 | 103 | ### How to Run 104 | 105 | Start a Netcat server as the data source: 106 | 107 | ```bash 108 | nc -lk 9999 109 | ``` 110 | 111 | Build and run the application: 112 | 113 | ```bash 114 | # Using Docker 115 | docker build -t apache/spark-connect-swift:stream . 116 | docker run --rm -e SPARK_REMOTE=sc://host.docker.internal:15002 -e TARGET_HOST=host.docker.internal apache/spark-connect-swift:stream 117 | 118 | # From source code 119 | swift run 120 | ``` 121 | 122 | Type text into the Netcat terminal to see real-time word counting from `Spark Connect Server` container. 123 | 124 | ## Web Application Example 125 | 126 | The web application example showcases how to integrate Spark Connect Swift with a web server using the Vapor framework. 127 | 128 | ### Key Features 129 | 130 | - HTTP server integration with Vapor 131 | - REST API endpoints 132 | - Spark session management within web requests 133 | - Version information retrieval 134 | 135 | ### How to Run 136 | 137 | Build and run the application: 138 | 139 | ```bash 140 | # Using Docker 141 | docker build -t apache/spark-connect-swift:web . 142 | docker run -it --rm -p 8080:8080 -e SPARK_REMOTE=sc://host.docker.internal:15002 apache/spark-connect-swift:web 143 | 144 | # From source code 145 | swift run 146 | ``` 147 | 148 | Access the web application: 149 | 150 | ```bash 151 | # Root endpoint 152 | $ curl http://127.0.0.1:8080/ 153 | Welcome to the Swift world. Say hello!% 154 | 155 | # Spark-powered endpoint 156 | curl http://127.0.0.1:8080/hello 157 | Hi, this is powered by the Apache Spark 4.0.1.% 158 | ``` 159 | 160 | ## Development Environment 161 | 162 | All examples include: 163 | 164 | - A Dockerfile for containerized execution 165 | - A Package.swift file for Swift Package Manager configuration 166 | - A README.md with detailed instructions 167 | - Source code in the Sources directory 168 | 169 | These examples are designed to be used with Apache Spark 4.0 or newer, using the Spark Connect protocol for client-server interaction. 170 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/SQLTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import Testing 22 | 23 | @testable import SparkConnect 24 | 25 | /// A test suite for various SQL statements. 26 | @Suite(.serialized) 27 | struct SQLTests { 28 | let fm = FileManager.default 29 | let path = Bundle.module.path(forResource: "queries", ofType: "")! 30 | let regenerateGoldenFiles = 31 | ProcessInfo.processInfo.environment["SPARK_GENERATE_GOLDEN_FILES"] == "1" 32 | 33 | let regexID = /#\d+L?/ 34 | let regexPlanId = /plan_id=\d+/ 35 | let regexLocation = /file:[a-zA-Z0-9\.\-\/\\]+/ 36 | let regexOwner = /(runner|185)/ 37 | 38 | private func cleanUp(_ str: String) -> String { 39 | return removeOwner(removeID(removeLocation(replaceUserName(str)))) 40 | } 41 | 42 | private func removeID(_ str: String) -> String { 43 | return str.replacing(regexPlanId, with: "plan_id=").replacing(regexID, with: "#") 44 | } 45 | 46 | private func removeLocation(_ str: String) -> String { 47 | return str.replacing(regexLocation, with: "*") 48 | } 49 | 50 | private func removeOwner(_ str: String) -> String { 51 | return str.replacing(regexOwner, with: "*") 52 | } 53 | 54 | private func replaceUserName(_ str: String) -> String { 55 | #if os(macOS) || os(Linux) 56 | return str.replacing(ProcessInfo.processInfo.userName, with: "spark") 57 | #else 58 | return str 59 | #endif 60 | } 61 | 62 | private func normalize(_ str: String) -> String { 63 | return str.replacing(/[-]+/, with: "-").replacing(/[ ]+/, with: " ") 64 | } 65 | 66 | @Test 67 | func testRemoveID() { 68 | #expect(removeID("123") == "123") 69 | #expect(removeID("123L") == "123L") 70 | #expect(removeID("#123") == "#") 71 | #expect(removeID("#123L") == "#") 72 | #expect(removeID("plan_id=123") == "plan_id=") 73 | } 74 | 75 | @Test 76 | func removeLocation() { 77 | #expect(removeLocation("file:/abc") == "*") 78 | } 79 | 80 | @Test 81 | func removeOwner() { 82 | #expect(removeOwner("runner") == "*") 83 | #expect(removeOwner("185") == "*") 84 | } 85 | 86 | @Test 87 | func testNormalize() { 88 | #expect(normalize("+------+------------------+") == "+-+-+") 89 | #expect(normalize("+ + +") == "+ + +") 90 | } 91 | 92 | let queriesForSpark4Only: [String] = [ 93 | "create_scala_function.sql", 94 | "create_table_function.sql", 95 | "cast.sql", 96 | "decimal.sql", 97 | "pipesyntax.sql", 98 | "explain.sql", 99 | "variant.sql", 100 | ] 101 | 102 | let queriesForSpark41Only: [String] = [ 103 | "time.sql" 104 | ] 105 | 106 | @Test 107 | func runAll() async throws { 108 | let spark = try await SparkSession.builder.getOrCreate() 109 | let MAX = Int32.max 110 | for name in try! fm.contentsOfDirectory(atPath: path).sorted() { 111 | guard name.hasSuffix(".sql") else { continue } 112 | print(name) 113 | if await !spark.version.starts(with: "4.") && queriesForSpark4Only.contains(name) { 114 | print("Skip query \(name) due to the difference between Spark 3 and 4.") 115 | continue 116 | } 117 | if await !spark.version.starts(with: "4.1") && queriesForSpark41Only.contains(name) { 118 | print("Skip query \(name) due to the difference between Spark 4.0 and 4.1") 119 | continue 120 | } 121 | 122 | let sql = try String(contentsOf: URL(fileURLWithPath: "\(path)/\(name)"), encoding: .utf8) 123 | let result = 124 | try await spark.sql(sql).showString(MAX, MAX, false).collect()[0].get(0) as! String 125 | let answer = cleanUp(result.trimmingCharacters(in: .whitespacesAndNewlines)) 126 | if regenerateGoldenFiles { 127 | let path = 128 | "\(FileManager.default.currentDirectoryPath)/Tests/SparkConnectTests/Resources/queries/\(name).answer" 129 | fm.createFile(atPath: path, contents: answer.data(using: .utf8)!, attributes: nil) 130 | } else { 131 | let expected = cleanUp( 132 | try String(contentsOf: URL(fileURLWithPath: "\(path)/\(name).answer"), encoding: .utf8) 133 | ) 134 | .trimmingCharacters(in: .whitespacesAndNewlines) 135 | if answer != expected { 136 | print("Try to compare normalized result.") 137 | #expect(normalize(answer) == normalize(expected)) 138 | } 139 | } 140 | } 141 | await spark.stop() 142 | } 143 | } 144 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/DataFrameReaderTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `DataFrameReader` 25 | @Suite(.serialized) 26 | struct DataFrameReaderTests { 27 | 28 | @Test 29 | func csv() async throws { 30 | let spark = try await SparkSession.builder.getOrCreate() 31 | let path = "../examples/src/main/resources/people.csv" 32 | #expect(try await spark.read.format("csv").load(path).count() == 3) 33 | #expect(try await spark.read.csv(path).count() == 3) 34 | #expect(try await spark.read.csv(path, path).count() == 6) 35 | await spark.stop() 36 | } 37 | 38 | @Test 39 | func json() async throws { 40 | let spark = try await SparkSession.builder.getOrCreate() 41 | let path = "../examples/src/main/resources/people.json" 42 | #expect(try await spark.read.format("json").load(path).count() == 3) 43 | #expect(try await spark.read.json(path).count() == 3) 44 | #expect(try await spark.read.json(path, path).count() == 6) 45 | await spark.stop() 46 | } 47 | 48 | @Test 49 | func xml() async throws { 50 | let spark = try await SparkSession.builder.getOrCreate() 51 | if await spark.version >= "4.0.0" { 52 | let path = "../examples/src/main/resources/people.xml" 53 | #expect(try await spark.read.option("rowTag", "person").format("xml").load(path).count() == 3) 54 | #expect(try await spark.read.option("rowTag", "person").xml(path).count() == 3) 55 | #expect(try await spark.read.option("rowTag", "person").xml(path, path).count() == 6) 56 | } 57 | await spark.stop() 58 | } 59 | 60 | @Test 61 | func orc() async throws { 62 | let spark = try await SparkSession.builder.getOrCreate() 63 | let path = "../examples/src/main/resources/users.orc" 64 | #expect(try await spark.read.format("orc").load(path).count() == 2) 65 | #expect(try await spark.read.orc(path).count() == 2) 66 | #expect(try await spark.read.orc(path, path).count() == 4) 67 | await spark.stop() 68 | } 69 | 70 | @Test 71 | func parquet() async throws { 72 | let spark = try await SparkSession.builder.getOrCreate() 73 | let path = "../examples/src/main/resources/users.parquet" 74 | #expect(try await spark.read.format("parquet").load(path).count() == 2) 75 | #expect(try await spark.read.parquet(path).count() == 2) 76 | #expect(try await spark.read.parquet(path, path).count() == 4) 77 | await spark.stop() 78 | } 79 | 80 | @Test 81 | func table() async throws { 82 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 83 | let spark = try await SparkSession.builder.getOrCreate() 84 | try await SQLHelper.withTable(spark, tableName)({ 85 | try await spark.sql("CREATE TABLE \(tableName) USING ORC AS VALUES (1), (2)").count() 86 | #expect(try await spark.read.table(tableName).count() == 2) 87 | }) 88 | await spark.stop() 89 | } 90 | 91 | @Test 92 | func schema() async throws { 93 | let spark = try await SparkSession.builder.getOrCreate() 94 | let path = "../examples/src/main/resources/people.json" 95 | #expect(try await spark.read.schema("age SHORT").json(path).dtypes.count == 1) 96 | #expect(try await spark.read.schema("age SHORT").json(path).dtypes[0] == ("age", "smallint")) 97 | #expect( 98 | try await spark.read.schema("age SHORT, name STRING").json(path).dtypes[0] == ( 99 | "age", "smallint" 100 | )) 101 | #expect( 102 | try await spark.read.schema("age SHORT, name STRING").json(path).dtypes[1] == ( 103 | "name", "string" 104 | )) 105 | await spark.stop() 106 | } 107 | 108 | @Test 109 | func invalidSchema() async throws { 110 | let spark = try await SparkSession.builder.getOrCreate() 111 | await #expect(throws: SparkConnectError.InvalidType) { 112 | try await spark.read.schema("invalid-name SHORT") 113 | } 114 | await #expect(throws: SparkConnectError.InvalidType) { 115 | try await spark.read.schema("age UNKNOWN_TYPE") 116 | } 117 | await spark.stop() 118 | } 119 | 120 | @Test 121 | func inputFiles() async throws { 122 | let spark = try await SparkSession.builder.getOrCreate() 123 | let path = "../examples/src/main/resources/users.orc" 124 | let answer = try await spark.read.format("orc").load(path).inputFiles() 125 | #expect(answer.count == 1) 126 | #expect(answer[0].hasSuffix("users.orc")) 127 | await spark.stop() 128 | } 129 | } 130 | -------------------------------------------------------------------------------- /Sources/SparkConnect/DataFrameWriterV2.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | import Foundation 20 | 21 | /// Interface used to write a ``DataFrame`` to external storage using the v2 API. 22 | public actor DataFrameWriterV2: Sendable { 23 | 24 | let tableName: String 25 | 26 | let df: DataFrame 27 | 28 | var provider: String? = nil 29 | 30 | var extraOptions: CaseInsensitiveDictionary = CaseInsensitiveDictionary() 31 | 32 | var tableProperties: CaseInsensitiveDictionary = CaseInsensitiveDictionary() 33 | 34 | var partitioningColumns: [Spark_Connect_Expression] = [] 35 | 36 | var clusteringColumns: [String]? = nil 37 | 38 | init(_ table: String, _ df: DataFrame) { 39 | self.tableName = table 40 | self.df = df 41 | } 42 | 43 | /// Specifies a provider for the underlying output data source. Spark's default catalog supports 44 | /// "orc", "json", etc. 45 | /// - Parameter provider: <#provider description#> 46 | public func using(_ provider: String) -> DataFrameWriterV2 { 47 | self.provider = provider 48 | return self 49 | } 50 | 51 | /// Adds an output option for the underlying data source. 52 | /// - Parameters: 53 | /// - key: A key string. 54 | /// - value: A value string. 55 | /// - Returns: A `DataFrameWriter`. 56 | public func option(_ key: String, _ value: String) -> DataFrameWriterV2 { 57 | self.extraOptions[key] = value 58 | return self 59 | } 60 | 61 | /// Add a table property. 62 | /// - Parameters: 63 | /// - property: A property name. 64 | /// - value: A property value. 65 | public func tableProperty(property: String, value: String) -> DataFrameWriterV2 { 66 | self.tableProperties[property] = value 67 | return self 68 | } 69 | 70 | /// Partition the output table created by `create`, `createOrReplace`, or `replace` using the 71 | /// given columns or transforms. 72 | /// - Parameter columns: Columns to partition 73 | /// - Returns: A ``DataFrameWriterV2``. 74 | public func partitionBy(_ columns: String...) -> DataFrameWriterV2 { 75 | self.partitioningColumns = columns.map { $0.toExpression } 76 | return self 77 | } 78 | 79 | /// Clusters the output by the given columns on the storage. The rows with matching values in the 80 | /// specified clustering columns will be consolidated within the same group. 81 | /// - Parameter columns: Columns to cluster 82 | /// - Returns: A ``DataFrameWriterV2``. 83 | public func clusterBy(_ columns: String...) -> DataFrameWriterV2 { 84 | self.clusteringColumns = columns 85 | return self 86 | } 87 | 88 | /// Create a new table from the contents of the data frame. 89 | public func create() async throws { 90 | try await executeWriteOperation(.create) 91 | } 92 | 93 | /// Replace an existing table with the contents of the data frame. 94 | public func replace() async throws { 95 | try await executeWriteOperation(.replace) 96 | } 97 | 98 | /// Create a new table or replace an existing table with the contents of the data frame. 99 | public func createOrReplace() async throws { 100 | try await executeWriteOperation(.createOrReplace) 101 | } 102 | 103 | /// Append the contents of the data frame to the output table. 104 | public func append() async throws { 105 | try await executeWriteOperation(.append) 106 | } 107 | 108 | /// Overwrite rows matching the given filter condition with the contents of the ``DataFrame`` in the 109 | /// output table. 110 | /// - Parameter condition: A filter condition. 111 | public func overwrite(_ condition: String) async throws { 112 | try await executeWriteOperation(.overwrite, condition) 113 | } 114 | 115 | /// Overwrite all partition for which the ``DataFrame`` contains at least one row with the contents 116 | /// of the data frame in the output table. 117 | /// This operation is equivalent to Hive's `INSERT OVERWRITE ... PARTITION`, which replaces 118 | /// partitions dynamically depending on the contents of the ``DataFrame``. 119 | public func overwritePartitions() async throws { 120 | try await executeWriteOperation(.overwritePartitions) 121 | } 122 | 123 | private func executeWriteOperation( 124 | _ mode: WriteOperationV2.Mode, 125 | _ overwriteCondition: String? = nil 126 | ) async throws { 127 | var write = WriteOperationV2() 128 | 129 | let plan = await self.df.getPlan() as! Plan 130 | write.input = plan.root 131 | write.tableName = self.tableName 132 | if let provider = self.provider { 133 | write.provider = provider 134 | } 135 | write.partitioningColumns = self.partitioningColumns 136 | if let clusteringColumns = self.clusteringColumns { 137 | write.clusteringColumns = clusteringColumns 138 | } 139 | for option in self.extraOptions.toStringDictionary() { 140 | write.options[option.key] = option.value 141 | } 142 | for property in self.tableProperties.toStringDictionary() { 143 | write.tableProperties[property.key] = property.value 144 | } 145 | if let overwriteCondition { 146 | write.overwriteCondition = overwriteCondition.toExpression 147 | } 148 | write.mode = mode 149 | 150 | var command = Spark_Connect_Command() 151 | command.writeOperationV2 = write 152 | try await df.spark.client.execute(df.spark.sessionID, command) 153 | } 154 | } 155 | -------------------------------------------------------------------------------- /Tests/SparkConnectTests/DataFrameWriterTests.swift: -------------------------------------------------------------------------------- 1 | // 2 | // Licensed to the Apache Software Foundation (ASF) under one 3 | // or more contributor license agreements. See the NOTICE file 4 | // distributed with this work for additional information 5 | // regarding copyright ownership. The ASF licenses this file 6 | // to you under the Apache License, Version 2.0 (the 7 | // "License"); you may not use this file except in compliance 8 | // with the License. You may obtain a copy of the License at 9 | // 10 | // http://www.apache.org/licenses/LICENSE-2.0 11 | // 12 | // Unless required by applicable law or agreed to in writing, 13 | // software distributed under the License is distributed on an 14 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY 15 | // KIND, either express or implied. See the License for the 16 | // specific language governing permissions and limitations 17 | // under the License. 18 | // 19 | 20 | import Foundation 21 | import SparkConnect 22 | import Testing 23 | 24 | /// A test suite for `DataFrameWriter` 25 | @Suite(.serialized) 26 | struct DataFrameWriterTests { 27 | 28 | @Test 29 | func csv() async throws { 30 | let tmpDir = "/tmp/" + UUID().uuidString 31 | let spark = try await SparkSession.builder.getOrCreate() 32 | try await spark.range(2025).write.csv(tmpDir) 33 | #expect(try await spark.read.csv(tmpDir).count() == 2025) 34 | await spark.stop() 35 | } 36 | 37 | @Test 38 | func json() async throws { 39 | let tmpDir = "/tmp/" + UUID().uuidString 40 | let spark = try await SparkSession.builder.getOrCreate() 41 | try await spark.range(2025).write.json(tmpDir) 42 | #expect(try await spark.read.json(tmpDir).count() == 2025) 43 | await spark.stop() 44 | } 45 | 46 | @Test 47 | func xml() async throws { 48 | let tmpDir = "/tmp/" + UUID().uuidString 49 | let spark = try await SparkSession.builder.getOrCreate() 50 | if await spark.version >= "4.0.0" { 51 | try await spark.range(2025).write.option("rowTag", "person").xml(tmpDir) 52 | #expect(try await spark.read.option("rowTag", "person").xml(tmpDir).count() == 2025) 53 | } 54 | await spark.stop() 55 | } 56 | 57 | @Test 58 | func orc() async throws { 59 | let tmpDir = "/tmp/" + UUID().uuidString 60 | let spark = try await SparkSession.builder.getOrCreate() 61 | try await spark.range(2025).write.orc(tmpDir) 62 | #expect(try await spark.read.orc(tmpDir).count() == 2025) 63 | await spark.stop() 64 | } 65 | 66 | @Test 67 | func parquet() async throws { 68 | let tmpDir = "/tmp/" + UUID().uuidString 69 | let spark = try await SparkSession.builder.getOrCreate() 70 | try await spark.range(2025).write.parquet(tmpDir) 71 | #expect(try await spark.read.parquet(tmpDir).count() == 2025) 72 | await spark.stop() 73 | } 74 | 75 | @Test 76 | func pathAlreadyExist() async throws { 77 | let tmpDir = "/tmp/" + UUID().uuidString 78 | let spark = try await SparkSession.builder.getOrCreate() 79 | try await spark.range(2025).write.csv(tmpDir) 80 | try await #require(throws: Error.self) { 81 | try await spark.range(2025).write.csv(tmpDir) 82 | } 83 | await spark.stop() 84 | } 85 | 86 | @Test 87 | func overwrite() async throws { 88 | let tmpDir = "/tmp/" + UUID().uuidString 89 | let spark = try await SparkSession.builder.getOrCreate() 90 | try await spark.range(2025).write.csv(tmpDir) 91 | try await spark.range(2025).write.mode("overwrite").csv(tmpDir) 92 | await spark.stop() 93 | } 94 | 95 | @Test 96 | func save() async throws { 97 | let tmpDir = "/tmp/" + UUID().uuidString 98 | let spark = try await SparkSession.builder.getOrCreate() 99 | for format in ["csv", "json", "orc", "parquet"] { 100 | try await spark.range(2025).write.format(format).mode("overwrite").save(tmpDir) 101 | #expect(try await spark.read.format(format).load(tmpDir).count() == 2025) 102 | } 103 | await spark.stop() 104 | } 105 | 106 | @Test 107 | func saveAsTable() async throws { 108 | let spark = try await SparkSession.builder.getOrCreate() 109 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 110 | try await SQLHelper.withTable(spark, tableName)({ 111 | try await spark.range(1).write.saveAsTable(tableName) 112 | #expect(try await spark.read.table(tableName).count() == 1) 113 | 114 | try await #require(throws: SparkConnectError.TableOrViewAlreadyExists) { 115 | try await spark.range(1).write.saveAsTable(tableName) 116 | } 117 | 118 | try await spark.range(1).write.mode("overwrite").saveAsTable(tableName) 119 | #expect(try await spark.read.table(tableName).count() == 1) 120 | 121 | try await spark.range(1).write.mode("append").saveAsTable(tableName) 122 | #expect(try await spark.read.table(tableName).count() == 2) 123 | }) 124 | await spark.stop() 125 | } 126 | 127 | @Test 128 | func insertInto() async throws { 129 | let spark = try await SparkSession.builder.getOrCreate() 130 | let tableName = "TABLE_" + UUID().uuidString.replacingOccurrences(of: "-", with: "") 131 | try await SQLHelper.withTable(spark, tableName)({ 132 | try await #require(throws: SparkConnectError.TableOrViewNotFound) { 133 | try await spark.range(1).write.insertInto(tableName) 134 | } 135 | 136 | try await spark.range(1).write.saveAsTable(tableName) 137 | #expect(try await spark.read.table(tableName).count() == 1) 138 | 139 | try await spark.range(1).write.insertInto(tableName) 140 | #expect(try await spark.read.table(tableName).count() == 2) 141 | 142 | try await spark.range(1).write.insertInto(tableName) 143 | #expect(try await spark.read.table(tableName).count() == 3) 144 | }) 145 | await spark.stop() 146 | } 147 | 148 | @Test 149 | func partitionBy() async throws { 150 | let tmpDir = "/tmp/" + UUID().uuidString 151 | let spark = try await SparkSession.builder.getOrCreate() 152 | try await spark.sql("SELECT 1 col1, 2 col2").write.partitionBy("col2").csv(tmpDir) 153 | #expect(try await spark.read.csv("\(tmpDir)/col2=2").count() == 1) 154 | await spark.stop() 155 | } 156 | 157 | @Test 158 | func sortByBucketBy() async throws { 159 | let tmpDir = "/tmp/" + UUID().uuidString 160 | let spark = try await SparkSession.builder.getOrCreate() 161 | let df = try await spark.sql("SELECT 1 col1, 2 col2") 162 | try await #require(throws: Error.self) { 163 | try await df.write.sortBy("col2").csv(tmpDir) 164 | } 165 | try await #require(throws: Error.self) { 166 | try await df.write.sortBy("col2").bucketBy(numBuckets: 3, "col2").csv(tmpDir) 167 | } 168 | try await #require(throws: Error.self) { 169 | try await df.write.bucketBy(numBuckets: 3, "col2").csv(tmpDir) 170 | } 171 | await spark.stop() 172 | } 173 | } 174 | --------------------------------------------------------------------------------