Skip to content
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
15 changes: 15 additions & 0 deletions Sources/SparkConnect/DataFrame.swift
Original file line number Diff line number Diff line change
Expand Up @@ -342,6 +342,21 @@ public actor DataFrame: Sendable {
return self
}

var storageLevel: StorageLevel {
Copy link
Member

Choose a reason for hiding this comment

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

I wonder why this is var instead of a function? E.g., in Dataset.scala, it is def storageLevel: StorageLevel.

get async throws {
try await withGRPCClient(
transport: .http2NIOPosix(
target: .dns(host: spark.client.host, port: spark.client.port),
transportSecurity: .plaintext
)
) { client in
let service = Spark_Connect_SparkConnectService.Client(wrapping: client)
return try await service
.analyzePlan(spark.client.getStorageLevel(spark.sessionID, plan)).getStorageLevel.storageLevel.toStorageLevel
}
}
}

public func explain() async throws {
try await explain("simple")
}
Expand Down
11 changes: 11 additions & 0 deletions Sources/SparkConnect/SparkConnectClient.swift
Original file line number Diff line number Diff line change
Expand Up @@ -282,6 +282,17 @@ public actor SparkConnectClient {
})
}

func getStorageLevel(_ sessionID: String, _ plan: Plan) async -> AnalyzePlanRequest
{
return analyze(
sessionID,
{
var level = AnalyzePlanRequest.GetStorageLevel()
level.relation = plan.root
return OneOf_Analyze.getStorageLevel(level)
})
}

func getExplain(_ sessionID: String, _ plan: Plan, _ mode: String) async -> AnalyzePlanRequest
{
return analyze(
Expand Down
18 changes: 18 additions & 0 deletions Sources/SparkConnect/StorageLevel.swift
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,12 @@ extension StorageLevel {
level.replication = self.replication
return level
}

public static func == (lhs: StorageLevel, rhs: StorageLevel) -> Bool {
return lhs.useDisk == rhs.useDisk && lhs.useMemory == rhs.useMemory
&& lhs.useOffHeap == rhs.useOffHeap && lhs.deserialized == rhs.deserialized
&& lhs.replication == rhs.replication
}
}

extension StorageLevel: CustomStringConvertible {
Expand All @@ -86,3 +92,15 @@ extension StorageLevel: CustomStringConvertible {
"StorageLevel(useDisk: \(useDisk), useMemory: \(useMemory), useOffHeap: \(useOffHeap), deserialized: \(deserialized), replication: \(replication))"
}
}

extension Spark_Connect_StorageLevel {
var toStorageLevel: StorageLevel {
return StorageLevel(
useDisk: self.useDisk,
useMemory: self.useMemory,
useOffHeap: self.useOffHeap,
deserialized: self.deserialized,
replication: self.replication
)
}
}
18 changes: 18 additions & 0 deletions Tests/SparkConnectTests/DataFrameTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -308,4 +308,22 @@ struct DataFrameTests {
await spark.stop()
}
#endif

@Test
func storageLevel() async throws {
let spark = try await SparkSession.builder.getOrCreate()
let df = try await spark.range(1)

_ = try await df.unpersist()
#expect(try await df.storageLevel == StorageLevel.NONE)
_ = try await df.persist()
#expect(try await df.storageLevel == StorageLevel.MEMORY_AND_DISK)

_ = try await df.unpersist()
#expect(try await df.storageLevel == StorageLevel.NONE)
_ = try await df.persist(storageLevel: StorageLevel.MEMORY_ONLY)
#expect(try await df.storageLevel == StorageLevel.MEMORY_ONLY)

await spark.stop()
}
}
Loading