Skip to content
Closed
Original file line number Diff line number Diff line change
Expand Up @@ -674,7 +674,7 @@ object FunctionRegistry {
expression[WindowTime]("window_time"),
expression[MakeDate]("make_date"),
expression[MakeTime]("make_time"),
expression[MakeTimestamp]("make_timestamp"),
expressionBuilder("make_timestamp", MakeTimestampExpressionBuilder),
expression[TryMakeTimestamp]("try_make_timestamp"),
expression[MonthName]("monthname"),
// We keep the 2 expression builders below to have different function docs.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2793,6 +2793,10 @@ object TryMakeTimestampLTZExpressionBuilder extends ExpressionBuilder {
}
}

/**
* This make_timestamp overload creates a timestamp from:
* - a year, month, day, hour, min, sec and optional timezone.
*/
// scalastyle:off line.size.limit
@ExpressionDescription(
usage = "_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create timestamp from year, month, day, hour, min, sec and timezone fields. The result data type is consistent with the value of configuration `spark.sql.timestampType`. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.",
Expand Down Expand Up @@ -3079,6 +3083,168 @@ case class TryMakeTimestamp(
}
}

/**
* This make_timestamp overload creates a timestamp from:
* - a date, optional time, and optional timezone.
*/
// scalastyle:off line.size.limit
@ExpressionDescription(
usage = """
_FUNC_(date[, time[, timezone]]) - Create timestamp from date and time fields.
""",
arguments = """
Arguments:
* date - a date expression
* time - a time expression (optional)
* timezone - the time zone identifier (optional). For example, CET, UTC and etc.
""",
examples = """
Examples:
> SELECT _FUNC_(DATE'2014-12-28');
2014-12-27 16:00:00
> SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
2014-12-27 22:30:45.887
> SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET');
2014-12-27 21:30:45.887
""",
group = "datetime_funcs",
since = "4.1.0")
// scalastyle:on line.size.limit
case class MakeTimestampFromDateTime(
date: Expression,
time: Option[Expression] = None,
timezone: Option[Expression] = None)
extends Expression with RuntimeReplaceable with ExpectsInputTypes {

def this(date: Expression) =
this(date, None, None)

def this(date: Expression, time: Expression) =
this(date, Some(time), None)

def this(date: Expression, time: Expression, timezone: Expression) =
this(date, Some(time), Some(timezone))

override def children: Seq[Expression] = Seq(date) ++ time ++ timezone

override def inputTypes: Seq[AbstractDataType] = Seq(DateType, AnyTimeType) ++
timezone.map(_ => StringTypeWithCollation(supportsTrimCollation = true))

override def replacement: Expression = {
// If time is not provided, we use midnight, i.e. 00:00:00.
val timeExpr = time.getOrElse(Literal(0L, TimeType(0)))
// If timezone is not provided, we use UTC, i.e. +00:00.
val zoneIdExpr = timezone.getOrElse(Literal("+00:00"))
StaticInvoke(
classOf[DateTimeUtils.type],
TimestampType,
"makeTimestamp",
Seq(date, timeExpr, zoneIdExpr),
Seq(date.dataType, timeExpr.dataType, zoneIdExpr.dataType),
returnNullable = children.exists(_.nullable)
)
}

override def prettyName: String = "make_timestamp"

override protected def withNewChildrenInternal(
newChildren: IndexedSeq[Expression]): Expression = {
val timeOpt = if (time.isDefined) Some(newChildren(1)) else None
val timezoneOpt = if (timezone.isDefined) Some(newChildren(2)) else None
copy(date = newChildren(0), time = timeOpt, timezone = timezoneOpt)
}
}

/**
* This make_timestamp expression builder creates a timestamp from:
* - a year, month, day, hour, min, sec and optional timezone, or
* - a date, optional time, and optional timezone.
*/
// scalastyle:off line.size.limit
@ExpressionDescription(
usage = """
_FUNC_(year, month, day, hour, min, sec[, timezone]) - Create the current timestamp with local time zone from year, month, day, hour, min, sec and timezone fields. If the configuration `spark.sql.ansi.enabled` is false, the function returns NULL on invalid inputs. Otherwise, it will throw an error instead.

_FUNC_(date[, time[, timezone]]) - Create timestamp from date and time fields.
""",
arguments = """
Arguments:
* year - the year to represent, from 1 to 9999
* month - the month-of-year to represent, from 1 (January) to 12 (December)
* day - the day-of-month to represent, from 1 to 31
* hour - the hour-of-day to represent, from 0 to 23
* min - the minute-of-hour to represent, from 0 to 59
* sec - the second-of-minute and its micro-fraction to represent, from
0 to 60. If the sec argument equals to 60, the seconds field is set
to 0 and 1 minute is added to the final timestamp.
* date - a date expression
* time - a time expression (optional)
* timezone - the time zone identifier (optional). For example, CET, UTC and etc.
""",
examples = """
Examples:
> SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887);
2014-12-28 06:30:45.887
> SELECT _FUNC_(2014, 12, 28, 6, 30, 45.887, 'CET');
2014-12-27 21:30:45.887
> SELECT _FUNC_(DATE'2014-12-28');
2014-12-27 16:00:00
> SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887');
2014-12-27 22:30:45.887
> SELECT _FUNC_(DATE'2014-12-28', TIME'6:30:45.887', 'CET');
2014-12-27 21:30:45.887
""",
group = "datetime_funcs",
since = "4.1.0")
// scalastyle:on line.size.limit
object MakeTimestampExpressionBuilder extends ExpressionBuilder {
override def build(funcName: String, expressions: Seq[Expression]): Expression = {
val numArgs = expressions.length
if (numArgs == 1) {
// date
MakeTimestampFromDateTime(
expressions(0)
)
} else if (numArgs == 2) {
// date, time
MakeTimestampFromDateTime(
expressions(0),
Some(expressions(1))
)
} else if (numArgs == 3) {
// date, time, timezone
MakeTimestampFromDateTime(
expressions(0),
Some(expressions(1)),
Some(expressions(2))
)
} else if (numArgs == 6) {
// year, month, day, hour, min, sec
MakeTimestamp(
expressions(0),
expressions(1),
expressions(2),
expressions(3),
expressions(4),
expressions(5)
)
} else if (numArgs == 7) {
// year, month, day, hour, min, sec, timezone
MakeTimestamp(
expressions(0),
expressions(1),
expressions(2),
expressions(3),
expressions(4),
expressions(5),
Some(expressions(6))
)
} else {
throw QueryCompilationErrors.wrongNumArgsError(funcName, Seq(1, 2, 3, 6, 7), numArgs)
}
}
}

object DatePart {

def parseExtractField(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -839,15 +839,43 @@ object DateTimeUtils extends SparkDateTimeUtils {
/**
* Makes a timestamp without time zone from a date and a local time.
*
* @param days The number of days since the epoch. 1970-01-01.
* @param days The number of days since the epoch 1970-01-01.
* Negative numbers represent earlier days.
* @param nanos The number of nanoseconds within the day since the midnight.
* @return The number of microseconds since the epoch of 1970-01-01 00:00:00Z.
* @return The number of microseconds since the epoch 1970-01-01 00:00:00Z.
*/
def makeTimestampNTZ(days: Int, nanos: Long): Long = {
localDateTimeToMicros(LocalDateTime.of(daysToLocalDate(days), nanosToLocalTime(nanos)))
}

/**
* Makes a timestamp from a date and a local time.
*
* @param days The number of days since the epoch 1970-01-01.
* Negative numbers represent earlier days.
* @param nanos The number of nanoseconds within the day since the midnight.
* @param zoneId The time zone ID at which the operation is performed.
* @return The number of microseconds since the epoch 1970-01-01 00:00:00Z.
*/
def makeTimestamp(days: Int, nanos: Long, zoneId: ZoneId): Long = {
val ldt = LocalDateTime.of(daysToLocalDate(days), nanosToLocalTime(nanos))
instantToMicros(ldt.atZone(zoneId).toInstant)
}

/**
* Makes a timestamp from a date and a local time with timezone string.
*
* @param days The number of days since the epoch 1970-01-01.
* Negative numbers represent earlier days.
* @param nanos The number of nanoseconds within the day since the midnight.
* @param timezone The time zone string.
* @return The number of microseconds since the epoch 1970-01-01 00:00:00Z.
*/
def makeTimestamp(days: Int, nanos: Long, timezone: UTF8String): Long = {
val zoneId = getZoneId(timezone.toString)
makeTimestamp(days, nanos, zoneId)
}

/**
* Adds a day-time interval to a time.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2142,14 +2142,158 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
}
}

test("make timestamp_ntz from date and time") {
def dateLit(d: String): Expression = Literal(LocalDate.parse(d))
def timeLit(t: String): Expression = Literal(LocalTime.parse(t))
def tsNtz(s: String): Long = localDateTimeToMicros(LocalDateTime.parse(s), UTC)
/**
* Helper method to create a DATE literal from a string in date format.
*/
private def dateLit(date: String): Expression = Literal(LocalDate.parse(date))

/**
* Helper method to create a TIME literal from a string in time format.
*/
private def timeLit(time: String): Expression = Literal(LocalTime.parse(time))

/**
* Helper method to get the microseconds from a timestamp represented as a string.
*/
private def timestampToMicros(timestamp: String, zoneId: ZoneId): Long = {
localDateTimeToMicros(LocalDateTime.parse(timestamp), zoneId)
}

test("SPARK-51415: make timestamp from date") {
// Test with valid date.
checkEvaluation(
MakeTimestampFromDateTime(dateLit("2023-10-01")),
timestampToMicros("2023-10-01T00:00:00", UTC)
)

// Test with null date.
checkEvaluation(
MakeTimestampFromDateTime(Literal(null, DateType)),
null
)
}

test("SPARK-51415: make timestamp from date and time") {
// Test with valid date and time.
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456"))
),
timestampToMicros("2023-10-01T12:34:56.123456", UTC)
)

// Test with null date.
checkEvaluation(
MakeTimestampFromDateTime(
Literal(null, DateType),
Some(timeLit("12:34:56.123456"))
),
null
)
// Test with null time.
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(Literal(null, TimeType()))
),
null
)
// Test with null date and null time.
checkEvaluation(
MakeTimestampFromDateTime(
Literal(null, DateType),
Some(Literal(null, TimeType())
)),
null
)
}

test("SPARK-51415: make timestamp from date, time, and timezone") {
// Test with valid date, time, and timezone.
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal("-09:30"))
),
timestampToMicros("2023-10-01T12:34:56.123456", MIT)
)
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal("-08:00"))
),
timestampToMicros("2023-10-01T12:34:56.123456", PST)
)
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal("+00:00"))
),
timestampToMicros("2023-10-01T12:34:56.123456", UTC)
)
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal("+01:00"))
),
timestampToMicros("2023-10-01T12:34:56.123456", CET)
)
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal("+09:00"))
),
timestampToMicros("2023-10-01T12:34:56.123456", JST)
)

// Test with null date.
checkEvaluation(
MakeTimestampFromDateTime(
Literal(null, DateType),
Some(timeLit("12:34:56.123456")),
Some(Literal("+00:00"))
),
null
)
// Test with null time.
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(Literal(null, TimeType())),
Some(Literal("+00:00"))
),
null
)
// Test with null timezone.
checkEvaluation(
MakeTimestampFromDateTime(
dateLit("2023-10-01"),
Some(timeLit("12:34:56.123456")),
Some(Literal(null, StringType))
),
null
)
// Test with null date and null time.
checkEvaluation(
MakeTimestampFromDateTime(
Literal(null, DateType),
Some(Literal(null, TimeType())),
Some(Literal("+00:00"))
),
null
)
}

test("make timestamp_ntz from date and time") {
checkEvaluation(MakeTimestampNTZ(dateLit("1970-01-01"), timeLit("00:00:00")), 0L)
checkEvaluation(MakeTimestampNTZ(dateLit("2025-06-20"), timeLit("15:20:30.123456")),
tsNtz("2025-06-20T15:20:30.123456"))
timestampToMicros("2025-06-20T15:20:30.123456", UTC))
checkEvaluation(MakeTimestampNTZ(Literal(null, DateType), timeLit("15:20:30.123456")),
null)
checkEvaluation(MakeTimestampNTZ(dateLit("2025-06-20"), Literal(null, TimeType())),
Expand Down
Loading