Skip to content

[SPARK-52182] Support withWatermark for DataFrame #158

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 1 commit 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
14 changes: 14 additions & 0 deletions Sources/SparkConnect/DataFrame.swift
Original file line number Diff line number Diff line change
Expand Up @@ -117,6 +117,7 @@ import Synchronization
/// - ``transpose()``
/// - ``transpose(_:)``
/// - ``hint(_:_:)``
/// - ``withWatermark(_:_:)``
///
/// ### Join Operations
/// - ``join(_:)``
Expand Down Expand Up @@ -1444,6 +1445,19 @@ public actor DataFrame: Sendable {
try await checkpoint(eager, false, storageLevel)
}

/// Defines an event time watermark for this ``DataFrame``. A watermark tracks a point in time
/// before which we assume no more late data is going to arrive.
/// - Parameters:
/// - eventTime: the name of the column that contains the event time of the row.
/// - delayThreshold: the minimum delay to wait to data to arrive late, relative to
/// the latest record that has been processed in the form of an interval (e.g. "1 minute" or "5 hours").
/// NOTE: This should not be negative.
/// - Returns: A ``DataFrame`` instance.
public func withWatermark(_ eventTime: String, _ delayThreshold: String) -> DataFrame {
let plan = SparkConnectClient.getWithWatermark(self.plan.root, eventTime, delayThreshold)
return DataFrame(spark: self.spark, plan: plan)
}

/// Returns a ``DataFrameWriter`` that can be used to write non-streaming data.
public var write: DataFrameWriter {
get {
Expand Down
17 changes: 17 additions & 0 deletions Sources/SparkConnect/SparkConnectClient.swift
Original file line number Diff line number Diff line change
Expand Up @@ -1022,6 +1022,23 @@ public actor SparkConnectClient {
return plan
}

static func getWithWatermark(
_ child: Relation,
_ eventTime: String,
_ delayThreshold: String
) -> Plan {
var withWatermark = Spark_Connect_WithWatermark()
withWatermark.input = child
withWatermark.eventTime = eventTime
withWatermark.delayThreshold = delayThreshold

var relation = Relation()
relation.withWatermark = withWatermark
var plan = Plan()
plan.opType = .root(relation)
return plan
}

func createTempView(
_ child: Relation, _ viewName: String, replace: Bool, isGlobal: Bool
) async throws {
Expand Down
16 changes: 16 additions & 0 deletions Tests/SparkConnectTests/DataFrameTests.swift
Original file line number Diff line number Diff line change
Expand Up @@ -744,6 +744,22 @@ struct DataFrameTests {
await spark.stop()
}

@Test
func withWatermark() async throws {
let spark = try await SparkSession.builder.getOrCreate()
let df = try await spark
.sql("""
SELECT * FROM VALUES
(1, now()),
(1, now() - INTERVAL 1 HOUR),
(1, now() - INTERVAL 2 HOUR)
T(data, eventTime)
""")
.withWatermark("eventTime", "1 minute") // This tests only API for now
Copy link
Member Author

Choose a reason for hiding this comment

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

I manually checked from the UI.

Screenshot 2025-05-15 at 21 59 26

#expect(try await df.dropDuplicatesWithinWatermark("data").count() == 1)
await spark.stop()
}

@Test
func describe() async throws {
let spark = try await SparkSession.builder.getOrCreate()
Expand Down
Loading