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 @@ -430,8 +430,11 @@ class MicroBatchExecution(
// Rewire the plan to use the new attributes that were returned by the source.
val newAttributePlan = newBatchesPlan transformAllExpressions {
case ct: CurrentTimestamp =>
// CurrentTimestamp is not TimeZoneAwareExpression while CurrentBatchTimestamp is.
// Without TimeZoneId, CurrentBatchTimestamp is unresolved. Here, we use an explicit
// dummy string to prevent UnresolvedException and to prevent to be used in the future.
CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs,
ct.dataType)
ct.dataType, Some("Dummy TimeZoneId"))
case cd: CurrentDate =>
CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs,
cd.dataType, cd.timeZoneId)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
import org.apache.spark.sql._
import org.apache.spark.sql.catalyst.plans.logical.Range
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes
import org.apache.spark.sql.catalyst.util.DateTimeUtils
import org.apache.spark.sql.execution.command.ExplainCommand
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProvider}
Expand Down Expand Up @@ -825,6 +826,47 @@ class StreamSuite extends StreamTest {
assert(query.exception.isEmpty)
}
}

test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " +
" to Dataset - use v2 sink") {
testCurrentTimestampOnStreamingQuery(useV2Sink = true)
}

test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " +
" to Dataset - use v1 sink") {
testCurrentTimestampOnStreamingQuery(useV2Sink = false)
}

private def testCurrentTimestampOnStreamingQuery(useV2Sink: Boolean): Unit = {
val input = MemoryStream[Int]
val df = input.toDS().withColumn("cur_timestamp", lit(current_timestamp()))

def assertBatchOutputAndUpdateLastTimestamp(
rows: Seq[Row],
curTimestamp: Long,
curDate: Int,
expectedValue: Int): Long = {
assert(rows.size === 1)
val row = rows.head
assert(row.getInt(0) === expectedValue)
assert(row.getTimestamp(1).getTime >= curTimestamp)
row.getTimestamp(1).getTime
}

var lastTimestamp = System.currentTimeMillis()
val currentDate = DateTimeUtils.millisToDays(lastTimestamp)
testStream(df, useV2Sink = useV2Sink) (
AddData(input, 1),
CheckLastBatch { rows: Seq[Row] =>
lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 1)
},
Execute { _ => Thread.sleep(1000) },
AddData(input, 2),
CheckLastBatch { rows: Seq[Row] =>
lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 2)
}
)
}
}

abstract class FakeSource extends StreamSourceProvider {
Expand Down