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
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,13 @@ case class EventTimeWatermark(
.putLong(EventTimeWatermark.delayKey, delay.milliseconds)
.build()
a.withMetadata(updatedMetadata)
} else if (a.metadata.contains(EventTimeWatermark.delayKey)) {
// Remove existing watermark
val updatedMetadata = new MetadataBuilder()
.withMetadata(a.metadata)
.remove(EventTimeWatermark.delayKey)
.build()
a.withMetadata(updatedMetadata)
} else {
a
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import org.scalatest.BeforeAndAfter

import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.functions.{count, window}
import org.apache.spark.sql.streaming.OutputMode._
Expand Down Expand Up @@ -305,6 +306,19 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Loggin
)
}

test("the new watermark should override the old one") {
val df = MemoryStream[(Long, Long)].toDF()
.withColumn("first", $"_1".cast("timestamp"))
.withColumn("second", $"_2".cast("timestamp"))
.withWatermark("first", "1 minute")
.withWatermark("second", "2 minutes")

val eventTimeColumns = df.logicalPlan.output
.filter(_.metadata.contains(EventTimeWatermark.delayKey))
assert(eventTimeColumns.size === 1)
assert(eventTimeColumns(0).name === "second")
}

private def assertNumStateRows(numTotalRows: Long): AssertOnQuery = AssertOnQuery { q =>
val progressWithData = q.recentProgress.filter(_.numInputRows > 0).lastOption.get
assert(progressWithData.stateOperators(0).numRowsTotal === numTotalRows)
Expand Down