Skip to content

[SPARK-51815] Add Row struct #63

New issue

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

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

Already on GitHub? Sign in to your account

Closed
wants to merge 2 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 5 additions & 5 deletions Sources/SparkConnect/Catalog.swift
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ public actor Catalog: Sendable {
catalog.catType = .currentCatalog(Spark_Connect_CurrentCatalog())
return catalog
})
return try await df.collect()[0][0]!
return try await df.collect()[0][0] as! String
}

/// Sets the current default catalog in this session.
Expand Down Expand Up @@ -130,7 +130,7 @@ public actor Catalog: Sendable {
return catalog
})
return try await df.collect().map {
CatalogMetadata(name: $0[0]!, description: $0[1])
try CatalogMetadata(name: $0[0] as! String, description: $0[1] as? String)
}
}

Expand All @@ -142,7 +142,7 @@ public actor Catalog: Sendable {
catalog.catType = .currentDatabase(Spark_Connect_CurrentDatabase())
return catalog
})
return try await df.collect()[0][0]!
return try await df.collect()[0][0] as! String
}

/// Sets the current default database in this session.
Expand Down Expand Up @@ -173,7 +173,7 @@ public actor Catalog: Sendable {
return catalog
})
return try await df.collect().map {
Database(name: $0[0]!, catalog: $0[1], description: $0[2], locationUri: $0[3]!)
try Database(name: $0[0] as! String, catalog: $0[1] as? String, description: $0[2] as? String, locationUri: $0[3] as! String)
}
}

Expand All @@ -189,7 +189,7 @@ public actor Catalog: Sendable {
return catalog
})
return try await df.collect().map {
Database(name: $0[0]!, catalog: $0[1], description: $0[2], locationUri: $0[3]!)
try Database(name: $0[0] as! String, catalog: $0[1] as? String, description: $0[2] as? String, locationUri: $0[3] as! String)
}.first!
}

Expand Down
20 changes: 10 additions & 10 deletions Sources/SparkConnect/DataFrame.swift
Original file line number Diff line number Diff line change
Expand Up @@ -197,15 +197,15 @@ public actor DataFrame: Sendable {
}
}

/// Execute the plan and return the result as ``[[String?]]``.
/// - Returns: ``[[String?]]``
public func collect() async throws -> [[String?]] {
/// Execute the plan and return the result as ``[Row]``.
/// - Returns: ``[Row]``
public func collect() async throws -> [Row] {
try await execute()

var result: [[String?]] = []
var result: [Row] = []
for batch in self.batches {
for i in 0..<batch.length {
var values: [String?] = []
var values: [Sendable?] = []
for column in batch.columns {
let str = column.array as! AsString
if column.data.isNull(i) {
Expand All @@ -217,7 +217,7 @@ public actor DataFrame: Sendable {
values.append(str.asString(i))
Copy link
Member

Choose a reason for hiding this comment

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

This is still stored the value as string? So we still cannot insert column values into Row in their types?

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes~ This is a part-one implementation to introduce Row struct like the note in the PR description.

Note that Row is added to support general type fields, but this PR replaces the existing API's [String?] signature into Row-based signature. The detailed one-to-one mapping among other types will be handled later.

Copy link
Member

Choose a reason for hiding this comment

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

Got it. Thank you.

}
}
result.append(values)
result.append(Row(valueArray: values))
}
}

Expand Down Expand Up @@ -377,15 +377,15 @@ public actor DataFrame: Sendable {

/// Returns the first `n` rows.
/// - Parameter n: The number of rows. (default: 1)
/// - Returns: ``[[String?]]``
public func head(_ n: Int32 = 1) async throws -> [[String?]] {
/// - Returns: ``[Row]``
public func head(_ n: Int32 = 1) async throws -> [Row] {
return try await limit(n).collect()
}

/// Returns the last `n` rows.
/// - Parameter n: The number of rows.
/// - Returns: ``[[String?]]``
public func tail(_ n: Int32) async throws -> [[String?]] {
/// - Returns: ``[Row]``
public func tail(_ n: Int32) async throws -> [Row] {
let lastN = DataFrame(spark:spark, plan: SparkConnectClient.getTail(self.plan.root, n))
return try await lastN.collect()
}
Expand Down
90 changes: 90 additions & 0 deletions Sources/SparkConnect/Row.swift
Original file line number Diff line number Diff line change
@@ -0,0 +1,90 @@
//
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements. See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership. The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License. You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied. See the License for the
// specific language governing permissions and limitations
// under the License.
//
import Foundation

public struct Row: Sendable, Equatable {
let values: [Sendable?]

public init(_ values: Sendable?...) {
self.values = values
}

public init(valueArray: [Sendable?]) {
self.values = valueArray
}

public static var empty: Row {
return Row()
}

public var size: Int { return length }

public var length: Int { return values.count }

subscript(index: Int) -> Sendable {
get throws {
return try get(index)
}
}

public func get(_ i: Int) throws -> Sendable {
if i < 0 || i >= self.length {
throw SparkConnectError.InvalidArgumentException
}
return values[i]
}

public static func == (lhs: Row, rhs: Row) -> Bool {
if lhs.values.count != rhs.values.count {
return false
}
return lhs.values.elementsEqual(rhs.values) { (x, y) in
if x == nil && y == nil {
return true
} else if let a = x as? Bool, let b = y as? Bool {
return a == b
} else if let a = x as? Int, let b = y as? Int {
return a == b
} else if let a = x as? Int8, let b = y as? Int8 {
return a == b
} else if let a = x as? Int16, let b = y as? Int16 {
return a == b
} else if let a = x as? Int32, let b = y as? Int32 {
return a == b
} else if let a = x as? Int64, let b = y as? Int64 {
return a == b
} else if let a = x as? Float, let b = y as? Float {
return a == b
} else if let a = x as? Double, let b = y as? Double {
return a == b
} else if let a = x as? String, let b = y as? String {
return a == b
} else {
return false
}
}
}

public func toString() -> String {
return "[\(self.values.map { "\($0 ?? "null")" }.joined(separator: ","))]"
}
}

extension Row {
}
20 changes: 10 additions & 10 deletions Tests/SparkConnectTests/DataFrameTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -249,15 +249,15 @@ struct DataFrameTests {
@Test
func sort() async throws {
let spark = try await SparkSession.builder.getOrCreate()
let expected = (1...10).map{ [String($0)] }
let expected = Array((1...10).map{ Row(String($0)) })
#expect(try await spark.range(10, 0, -1).sort("id").collect() == expected)
await spark.stop()
}

@Test
func orderBy() async throws {
let spark = try await SparkSession.builder.getOrCreate()
let expected = (1...10).map{ [String($0)] }
let expected = Array((1...10).map{ Row(String($0)) })
#expect(try await spark.range(10, 0, -1).orderBy("id").collect() == expected)
await spark.stop()
}
Expand All @@ -284,28 +284,28 @@ struct DataFrameTests {
#expect(
try await spark.sql(
"SELECT * FROM VALUES (1, true, 'abc'), (null, null, null), (3, false, 'def')"
).collect() == [["1", "true", "abc"], [nil, nil, nil], ["3", "false", "def"]])
).collect() == [Row("1", "true", "abc"), Row(nil, nil, nil), Row("3", "false", "def")])
await spark.stop()
}

@Test
func head() async throws {
let spark = try await SparkSession.builder.getOrCreate()
#expect(try await spark.range(0).head().isEmpty)
#expect(try await spark.range(2).sort("id").head() == [["0"]])
#expect(try await spark.range(2).sort("id").head(1) == [["0"]])
#expect(try await spark.range(2).sort("id").head(2) == [["0"], ["1"]])
#expect(try await spark.range(2).sort("id").head(3) == [["0"], ["1"]])
#expect(try await spark.range(2).sort("id").head() == [Row("0")])
#expect(try await spark.range(2).sort("id").head(1) == [Row("0")])
#expect(try await spark.range(2).sort("id").head(2) == [Row("0"), Row("1")])
#expect(try await spark.range(2).sort("id").head(3) == [Row("0"), Row("1")])
await spark.stop()
}

@Test
func tail() async throws {
let spark = try await SparkSession.builder.getOrCreate()
#expect(try await spark.range(0).tail(1).isEmpty)
#expect(try await spark.range(2).sort("id").tail(1) == [["1"]])
#expect(try await spark.range(2).sort("id").tail(2) == [["0"], ["1"]])
#expect(try await spark.range(2).sort("id").tail(3) == [["0"], ["1"]])
#expect(try await spark.range(2).sort("id").tail(1) == [Row("1")])
#expect(try await spark.range(2).sort("id").tail(2) == [Row("0"), Row("1")])
#expect(try await spark.range(2).sort("id").tail(3) == [Row("0"), Row("1")])
await spark.stop()
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[[61 62 63]]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/binary.sql.json

This file was deleted.

1 change: 1 addition & 0 deletions Tests/SparkConnectTests/Resources/queries/cache.sql.answer
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@

1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/cache.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@

This file was deleted.

1 change: 1 addition & 0 deletions Tests/SparkConnectTests/Resources/queries/date.sql.answer
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[2025-03-15 00:00:00 +0000]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/date.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
[Catalog Name,spark_catalog]
[Namespace Name,default]
[Comment,default database]
[Location,*]
[Owner,*]
Comment on lines +4 to +5
Copy link
Member

Choose a reason for hiding this comment

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

Hmm, why the the second elements of the two items ["Location","file:/opt/spark/work-dir/spark-warehouse"],["Owner","185"] are * now?

Copy link
Member Author

@dongjoon-hyun dongjoon-hyun Apr 16, 2025

Choose a reason for hiding this comment

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

During the Catalog PR, the cleaning methods were added previously. While regenerating answer files Today, I used the cleaned strings because the cleaned one is better in the GitHub repo.

private func cleanUp(_ str: String) -> String {
return removeOwner(removeID(removeLocation(str)))
}
private func removeID(_ str: String) -> String {
return str.replacing(regexPlanId, with: "plan_id=").replacing(regexID, with: "#")
}
private func removeLocation(_ str: String) -> String {
return str.replacing(regexLocation, with: "*")
}
private func removeOwner(_ str: String) -> String {
return str.replacing(regexOwner, with: "*")
}

Copy link
Member

Choose a reason for hiding this comment

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

I see. Thank you.

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
[Function: abs]
[Class: org.apache.spark.sql.catalyst.expressions.Abs]
[Usage: abs(expr) - Returns the absolute value of the numeric or interval value.]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,3 @@
[id,int,null]
[name,string,null]
[salary,double,null]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[col,int,null]

This file was deleted.

22 changes: 22 additions & 0 deletions Tests/SparkConnectTests/Resources/queries/explain.sql.answer
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
[== Parsed Logical Plan ==
'Aggregate ['k], ['k, unresolvedalias('sum('v))]
+- SubqueryAlias t
+- LocalRelation [k#, v#]

== Analyzed Logical Plan ==
k: int, sum(v): bigint
Aggregate [k#], [k#, sum(v#) AS sum(v)#]
+- SubqueryAlias t
+- LocalRelation [k#, v#]

== Optimized Logical Plan ==
Aggregate [k#], [k#, sum(v#) AS sum(v)#]
+- LocalRelation [k#, v#]

== Physical Plan ==
AdaptiveSparkPlan isFinalPlan=false
+- HashAggregate(keys=[k#], functions=[sum(v#)], output=[k#, sum(v)#])
+- Exchange hashpartitioning(k#, 200), ENSURE_REQUIREMENTS, [plan_id=]
+- HashAggregate(keys=[k#], functions=[partial_sum(v#)], output=[k#, sum#])
+- LocalTableScan [k#, v#]
]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/explain.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[1.0,-2.0,3.0,-4.0,inf,nan,inf,nan]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[127,-128,32767,-32768,2147483647,-2147483648,9223372036854775807,-9223372036854775808]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1,2 @@
[0,0]
[1,2]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[1]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/select.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[default]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[,testcache,true]

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[abc,def]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/string.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
[{1},{2,{3}}]
1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/struct.sql.json

This file was deleted.

Original file line number Diff line number Diff line change
@@ -0,0 +1 @@

1 change: 0 additions & 1 deletion Tests/SparkConnectTests/Resources/queries/uncache.sql.json

This file was deleted.

Loading
Loading