diff --git a/python/pyspark/sql/connect/functions/builtin.py b/python/pyspark/sql/connect/functions/builtin.py index 2668b7a526fdd..aadbfde978768 100644 --- a/python/pyspark/sql/connect/functions/builtin.py +++ b/python/pyspark/sql/connect/functions/builtin.py @@ -3967,6 +3967,7 @@ def make_timestamp( make_timestamp.__doc__ = pysparkfuncs.make_timestamp.__doc__ +@overload def try_make_timestamp( years: "ColumnOrName", months: "ColumnOrName", @@ -3974,16 +3975,90 @@ def try_make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", +) -> Column: + ... + + +@overload +def try_make_timestamp( + years: "ColumnOrName", + months: "ColumnOrName", + days: "ColumnOrName", + hours: "ColumnOrName", + mins: "ColumnOrName", + secs: "ColumnOrName", + timezone: "ColumnOrName", +) -> Column: + ... + + +@overload +def try_make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: + ... + + +@overload +def try_make_timestamp( + *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" +) -> Column: + ... + + +def try_make_timestamp( + years: Optional["ColumnOrName"] = None, + months: Optional["ColumnOrName"] = None, + days: Optional["ColumnOrName"] = None, + hours: Optional["ColumnOrName"] = None, + mins: Optional["ColumnOrName"] = None, + secs: Optional["ColumnOrName"] = None, timezone: Optional["ColumnOrName"] = None, + date: Optional["ColumnOrName"] = None, + time: Optional["ColumnOrName"] = None, ) -> Column: - if timezone is not None: - return _invoke_function_over_columns( - "try_make_timestamp", years, months, days, hours, mins, secs, timezone - ) + if years is not None: + if any(arg is not None for arg in [date, time]): + raise PySparkValueError( + errorClass="CANNOT_SET_TOGETHER", + messageParameters={"arg_list": "years|months|days|hours|mins|secs and date|time"}, + ) + if timezone is not None: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", years), + cast("ColumnOrName", months), + cast("ColumnOrName", days), + cast("ColumnOrName", hours), + cast("ColumnOrName", mins), + cast("ColumnOrName", secs), + cast("ColumnOrName", timezone), + ) + else: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", years), + cast("ColumnOrName", months), + cast("ColumnOrName", days), + cast("ColumnOrName", hours), + cast("ColumnOrName", mins), + cast("ColumnOrName", secs), + ) else: - return _invoke_function_over_columns( - "try_make_timestamp", years, months, days, hours, mins, secs - ) + if any(arg is not None for arg in [years, months, days, hours, mins, secs]): + raise PySparkValueError( + errorClass="CANNOT_SET_TOGETHER", + messageParameters={"arg_list": "years|months|days|hours|mins|secs and date|time"}, + ) + if timezone is not None: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", date), + cast("ColumnOrName", time), + cast("ColumnOrName", timezone), + ) + else: + return _invoke_function_over_columns( + "try_make_timestamp", cast("ColumnOrName", date), cast("ColumnOrName", time) + ) try_make_timestamp.__doc__ = pysparkfuncs.try_make_timestamp.__doc__ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 99fb7ee1ec5e8..adc6fb1121642 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -24888,7 +24888,7 @@ def make_timestamp( ) -@_try_remote_functions +@overload def try_make_timestamp( years: "ColumnOrName", months: "ColumnOrName", @@ -24896,34 +24896,97 @@ def try_make_timestamp( hours: "ColumnOrName", mins: "ColumnOrName", secs: "ColumnOrName", +) -> Column: + ... + + +@overload +def try_make_timestamp( + years: "ColumnOrName", + months: "ColumnOrName", + days: "ColumnOrName", + hours: "ColumnOrName", + mins: "ColumnOrName", + secs: "ColumnOrName", + timezone: "ColumnOrName", +) -> Column: + ... + + +@overload +def try_make_timestamp(*, date: "ColumnOrName", time: "ColumnOrName") -> Column: + ... + + +@overload +def try_make_timestamp( + *, date: "ColumnOrName", time: "ColumnOrName", timezone: "ColumnOrName" +) -> Column: + ... + + +@_try_remote_functions +def try_make_timestamp( + years: Optional["ColumnOrName"] = None, + months: Optional["ColumnOrName"] = None, + days: Optional["ColumnOrName"] = None, + hours: Optional["ColumnOrName"] = None, + mins: Optional["ColumnOrName"] = None, + secs: Optional["ColumnOrName"] = None, timezone: Optional["ColumnOrName"] = None, + date: Optional["ColumnOrName"] = None, + time: Optional["ColumnOrName"] = None, ) -> Column: """ - Try to create timestamp from years, months, days, hours, mins, secs and timezone fields. + Try to create timestamp from years, months, days, hours, mins, secs and (optional) timezone + fields. Alternatively, try to create timestamp from date, time, and (optional) timezone fields. The result data type is consistent with the value of configuration `spark.sql.timestampType`. The function returns NULL on invalid inputs. .. versionadded:: 4.0.0 + .. versionchanged:: 4.1.0 + Added support for creating timestamps from date and time. + Parameters ---------- - years : :class:`~pyspark.sql.Column` or column name - The year to represent, from 1 to 9999 - months : :class:`~pyspark.sql.Column` or column name - The month-of-year to represent, from 1 (January) to 12 (December) - days : :class:`~pyspark.sql.Column` or column name - The day-of-month to represent, from 1 to 31 - hours : :class:`~pyspark.sql.Column` or column name - The hour-of-day to represent, from 0 to 23 - mins : :class:`~pyspark.sql.Column` or column name - The minute-of-hour to represent, from 0 to 59 - secs : :class:`~pyspark.sql.Column` or column name + years : :class:`~pyspark.sql.Column` or column name, optional + The year to represent, from 1 to 9999. + Required when creating timestamps from individual components. + Must be used with months, days, hours, mins, and secs. + months : :class:`~pyspark.sql.Column` or column name, optional + The month-of-year to represent, from 1 (January) to 12 (December). + Required when creating timestamps from individual components. + Must be used with years, days, hours, mins, and secs. + days : :class:`~pyspark.sql.Column` or column name, optional + The day-of-month to represent, from 1 to 31. + Required when creating timestamps from individual components. + Must be used with years, months, hours, mins, and secs. + hours : :class:`~pyspark.sql.Column` or column name, optional + The hour-of-day to represent, from 0 to 23. + Required when creating timestamps from individual components. + Must be used with years, months, days, mins, and secs. + mins : :class:`~pyspark.sql.Column` or column name, optional + The minute-of-hour to represent, from 0 to 59. + Required when creating timestamps from individual components. + Must be used with years, months, days, hours, and secs. + secs : :class:`~pyspark.sql.Column` or column name, optional The second-of-minute and its micro-fraction to represent, from 0 to 60. - The value can be either an integer like 13 , or a fraction like 13.123. + The value can be either an integer like 13, or a fraction like 13.123. If the sec argument equals to 60, the seconds field is set to 0 and 1 minute is added to the final timestamp. + Required when creating timestamps from individual components. + Must be used with years, months, days, hours, and mins. timezone : :class:`~pyspark.sql.Column` or column name, optional - The time zone identifier. For example, CET, UTC and etc. + The time zone identifier. For example, CET, UTC, and etc. + date : :class:`~pyspark.sql.Column` or column name, optional + The date to represent, in valid DATE format. + Required when creating timestamps from date and time components. + Must be used with time parameter only. + time : :class:`~pyspark.sql.Column` or column name, optional + The time to represent, in valid TIME format. + Required when creating timestamps from date and time components. + Must be used with date parameter only. Returns ------- @@ -24945,7 +25008,7 @@ def try_make_timestamp( -------- >>> spark.conf.set("spark.sql.session.timeZone", "America/Los_Angeles") - Example 1: Make timestamp from years, months, days, hours, mins and secs. + Example 1: Make timestamp from years, months, days, hours, mins, secs, and timezone. >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], @@ -24959,7 +25022,7 @@ def try_make_timestamp( |2014-12-27 21:30:45.887 | +----------------------------------------------------+ - Example 2: Make timestamp without timezone. + Example 2: Make timestamp from years, months, days, hours, mins, and secs (without timezone). >>> import pyspark.sql.functions as sf >>> df = spark.createDataFrame([[2014, 12, 28, 6, 30, 45.887, 'CET']], @@ -24972,7 +25035,6 @@ def try_make_timestamp( +----------------------------------------------------+ |2014-12-28 06:30:45.887 | +----------------------------------------------------+ - >>> spark.conf.unset("spark.sql.session.timeZone") Example 3: Make timestamp with invalid input. @@ -24988,16 +25050,85 @@ def try_make_timestamp( |NULL | +----------------------------------------------------+ + Example 4: Make timestamp from date, time, and timezone. + + >>> import pyspark.sql.functions as sf + >>> from datetime import date, time + >>> df = spark.range(1).select( + ... sf.lit(date(2014, 12, 28)).alias("date"), + ... sf.lit(time(6, 30, 45, 887000)).alias("time"), + ... sf.lit("CET").alias("tz") + ... ) + >>> df.select( + ... sf.try_make_timestamp(date=df.date, time=df.time, timezone=df.tz) + ... ).show(truncate=False) + +----------------------------------+ + |try_make_timestamp(date, time, tz)| + +----------------------------------+ + |2014-12-27 21:30:45.887 | + +----------------------------------+ + + Example 5: Make timestamp from date and time (without timezone). + + >>> import pyspark.sql.functions as sf + >>> from datetime import date, time + >>> df = spark.range(1).select( + ... sf.lit(date(2014, 12, 28)).alias("date"), + ... sf.lit(time(6, 30, 45, 887000)).alias("time") + ... ) + >>> df.select(sf.try_make_timestamp(date=df.date, time=df.time)).show(truncate=False) + +------------------------------+ + |try_make_timestamp(date, time)| + +------------------------------+ + |2014-12-28 06:30:45.887 | + +------------------------------+ + >>> spark.conf.unset("spark.sql.session.timeZone") """ - if timezone is not None: - return _invoke_function_over_columns( - "try_make_timestamp", years, months, days, hours, mins, secs, timezone - ) + if years is not None: + if any(arg is not None for arg in [date, time]): + raise PySparkValueError( + errorClass="CANNOT_SET_TOGETHER", + messageParameters={"arg_list": "years|months|days|hours|mins|secs and date|time"}, + ) + if timezone is not None: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", years), + cast("ColumnOrName", months), + cast("ColumnOrName", days), + cast("ColumnOrName", hours), + cast("ColumnOrName", mins), + cast("ColumnOrName", secs), + cast("ColumnOrName", timezone), + ) + else: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", years), + cast("ColumnOrName", months), + cast("ColumnOrName", days), + cast("ColumnOrName", hours), + cast("ColumnOrName", mins), + cast("ColumnOrName", secs), + ) else: - return _invoke_function_over_columns( - "try_make_timestamp", years, months, days, hours, mins, secs - ) + if any(arg is not None for arg in [years, months, days, hours, mins, secs]): + raise PySparkValueError( + errorClass="CANNOT_SET_TOGETHER", + messageParameters={"arg_list": "years|months|days|hours|mins|secs and date|time"}, + ) + if timezone is not None: + return _invoke_function_over_columns( + "try_make_timestamp", + cast("ColumnOrName", date), + cast("ColumnOrName", time), + cast("ColumnOrName", timezone), + ) + else: + return _invoke_function_over_columns( + "try_make_timestamp", cast("ColumnOrName", date), cast("ColumnOrName", time) + ) @_try_remote_functions diff --git a/python/pyspark/sql/tests/test_functions.py b/python/pyspark/sql/tests/test_functions.py index 41c07a61eb1e3..b7479b7b7f2dd 100644 --- a/python/pyspark/sql/tests/test_functions.py +++ b/python/pyspark/sql/tests/test_functions.py @@ -441,19 +441,247 @@ def test_try_parse_url(self): assertDataFrameEqual(actual, [Row(None)]) def test_try_make_timestamp(self): - data = [(2024, 5, 22, 10, 30, 0)] - df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) + """Comprehensive test cases for try_make_timestamp with various arguments.""" + + # Common input dataframe setup for multiple test cases (with various arguments). + df = self.spark.createDataFrame( + [(2024, 5, 22, 10, 30, 0, "CET")], + ["year", "month", "day", "hour", "minute", "second", "timezone"], + ) + df_frac = self.spark.createDataFrame( + [(2024, 5, 22, 10, 30, 45.123, "CET")], + ["year", "month", "day", "hour", "minute", "second", "timezone"], + ) + df_dt = self.spark.range(1).select( + F.lit(datetime.date(2024, 5, 22)).alias("date"), + F.lit(datetime.time(10, 30, 0)).alias("time"), + F.lit("CET").alias("timezone"), + ) + df_dt_frac = self.spark.range(1).select( + F.lit(datetime.date(2024, 5, 22)).alias("date"), + F.lit(datetime.time(10, 30, 45, 123000)).alias("time"), + F.lit("CET").alias("timezone"), + ) + # Expected results for comparison in different scenarios. + result_no_tz = datetime.datetime(2024, 5, 22, 10, 30) + result_with_tz = datetime.datetime(2024, 5, 22, 8, 30) + result_frac_no_tz = datetime.datetime(2024, 5, 22, 10, 30, 45, 123000) + result_frac_with_tz = datetime.datetime(2024, 5, 22, 8, 30, 45, 123000) + + # Test 1A: Basic 6 positional arguments (years, months, days, hours, mins, secs). actual = df.select( F.try_make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second) ) - assertDataFrameEqual(actual, [Row(datetime.datetime(2024, 5, 22, 10, 30))]) + assertDataFrameEqual(actual, [Row(result_no_tz)]) - data = [(2024, 13, 22, 10, 30, 0)] - df = self.spark.createDataFrame(data, ["year", "month", "day", "hour", "minute", "second"]) + # Test 1B: Basic 7 positional arguments (years, months, days, hours, mins, secs, timezone). actual = df.select( - F.try_make_timestamp(df.year, df.month, df.day, df.hour, df.minute, df.second) + F.try_make_timestamp( + df.year, df.month, df.day, df.hour, df.minute, df.second, df.timezone + ) ) - assertDataFrameEqual(actual, [Row(None)]) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Test 2A: Basic 6 keyword arguments (years, months, days, hours, mins, secs). + actual = df.select( + F.try_make_timestamp( + years=df.year, + months=df.month, + days=df.day, + hours=df.hour, + mins=df.minute, + secs=df.second, + ) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 2B: Basic 7 keyword arguments (years, months, days, hours, mins, secs, timezone). + actual = df.select( + F.try_make_timestamp( + years=df.year, + months=df.month, + days=df.day, + hours=df.hour, + mins=df.minute, + secs=df.second, + timezone=df.timezone, + ) + ) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Test 3A: Alternative 2 keyword arguments (date, time). + actual = df_dt.select(F.try_make_timestamp(date=df_dt.date, time=df_dt.time)) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 3B: Alternative 3 keyword arguments (date, time, timezone). + actual = df_dt.select( + F.try_make_timestamp(date=df_dt.date, time=df_dt.time, timezone=df_dt.timezone) + ) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Test 4A: Fractional seconds with positional arguments (without timezone). + actual = df_frac.select( + F.try_make_timestamp( + df_frac.year, + df_frac.month, + df_frac.day, + df_frac.hour, + df_frac.minute, + df_frac.second, + ) + ) + assertDataFrameEqual(actual, [Row(result_frac_no_tz)]) + + # Test 4B: Fractional seconds with positional arguments (with timezone). + actual = df_frac.select( + F.try_make_timestamp( + df_frac.year, + df_frac.month, + df_frac.day, + df_frac.hour, + df_frac.minute, + df_frac.second, + df_frac.timezone, + ) + ) + assertDataFrameEqual(actual, [Row(result_frac_with_tz)]) + + # Test 5A: Fractional seconds with keyword arguments (without timezone). + actual = df_frac.select( + F.try_make_timestamp( + years=df_frac.year, + months=df_frac.month, + days=df_frac.day, + hours=df_frac.hour, + mins=df_frac.minute, + secs=df_frac.second, + ) + ) + assertDataFrameEqual(actual, [Row(result_frac_no_tz)]) + + # Test 5B: Fractional seconds with keyword arguments (with timezone). + actual = df_frac.select( + F.try_make_timestamp( + years=df_frac.year, + months=df_frac.month, + days=df_frac.day, + hours=df_frac.hour, + mins=df_frac.minute, + secs=df_frac.second, + timezone=df_frac.timezone, + ) + ) + assertDataFrameEqual(actual, [Row(result_frac_with_tz)]) + + # Test 6A: Fractional seconds with date/time arguments (without timezone). + actual = df_dt_frac.select(F.try_make_timestamp(date=df_dt_frac.date, time=df_dt_frac.time)) + assertDataFrameEqual(actual, [Row(result_frac_no_tz)]) + + # Test 6B: Fractional seconds with date/time arguments (with timezone). + actual = df_dt_frac.select( + F.try_make_timestamp( + date=df_dt_frac.date, time=df_dt_frac.time, timezone=df_dt_frac.timezone + ) + ) + assertDataFrameEqual(actual, [Row(result_frac_with_tz)]) + + # Test 7: Edge case - February 29 in leap year. + df_leap = self.spark.createDataFrame( + [(2024, 2, 29, 0, 0, 0)], ["year", "month", "day", "hour", "minute", "second"] + ) + expected_leap = datetime.datetime(2024, 2, 29, 0, 0, 0) + actual = df_leap.select( + F.try_make_timestamp( + df_leap.year, + df_leap.month, + df_leap.day, + df_leap.hour, + df_leap.minute, + df_leap.second, + ) + ) + assertDataFrameEqual(actual, [Row(expected_leap)]) + + # Test 8: Mixed positional and keyword (should work for valid combinations). + actual = df.select( + F.try_make_timestamp( + df.year, df.month, df.day, hours=df.hour, mins=df.minute, secs=df.second + ) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 9A: Using literal values for positional arguments (without timezone). + actual = self.spark.range(1).select( + F.try_make_timestamp(F.lit(2024), F.lit(5), F.lit(22), F.lit(10), F.lit(30), F.lit(0)) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 9B: Using literal values for positional arguments (with timezone). + actual = self.spark.range(1).select( + F.try_make_timestamp( + F.lit(2024), F.lit(5), F.lit(22), F.lit(10), F.lit(30), F.lit(0), F.lit("CET") + ) + ) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Test 10A: Using literal values for date/time arguments (without timezone). + actual = self.spark.range(1).select( + F.try_make_timestamp( + date=F.lit(datetime.date(2024, 5, 22)), time=F.lit(datetime.time(10, 30, 0)) + ) + ) + assertDataFrameEqual(actual, [Row(result_no_tz)]) + + # Test 10B: Using literal values for date/time arguments (with timezone). + actual = self.spark.range(1).select( + F.try_make_timestamp( + date=F.lit(datetime.date(2024, 5, 22)), + time=F.lit(datetime.time(10, 30, 0)), + timezone=F.lit("CET"), + ) + ) + assertDataFrameEqual(actual, [Row(result_with_tz)]) + + # Error handling tests. + + # Test 11: Mixing timestamp and date/time keyword arguments - should raise Exception. + with self.assertRaises(PySparkValueError) as context: + df_dt.select( + F.try_make_timestamp(years=df.year, date=df_dt.date, time=df_dt.time) + ).collect() + error_msg = str(context.exception) + self.assertIn("CANNOT_SET_TOGETHER", error_msg) + self.assertIn("years|months|days|hours|mins|secs and date|time", error_msg) + + with self.assertRaises(PySparkValueError) as context: + df_dt.select( + F.try_make_timestamp(hours=df.hour, time=df_dt.time, timezone=df_dt.timezone) + ).collect() + error_msg = str(context.exception) + self.assertIn("CANNOT_SET_TOGETHER", error_msg) + self.assertIn("years|months|days|hours|mins|secs and date|time", error_msg) + + # Test 12: Incomplete keyword arguments - should raise Exception for None values. + with self.assertRaises(Exception): + F.try_make_timestamp(years=df.year) + with self.assertRaises(Exception): + F.try_make_timestamp(secs=df.second) + with self.assertRaises(Exception): + F.try_make_timestamp(years=df.year, months=df.month, days=df.day) + with self.assertRaises(Exception): + F.try_make_timestamp(days=df.day, timezone=df.timezone) + with self.assertRaises(Exception): + F.try_make_timestamp( + hours=df.hour, mins=df.minute, secs=df.second, timezone=df.timezone + ) + with self.assertRaises(Exception): + F.try_make_timestamp(date=df_dt.date) + with self.assertRaises(Exception): + F.try_make_timestamp(time=df_dt.time, timezone=df_dt.timezone) + with self.assertRaises(Exception): + F.try_make_timestamp(timezone=df.timezone) + with self.assertRaises(Exception): + F.try_make_timestamp(timezone=df_dt.timezone) def test_try_make_timestamp_ltz(self): # use local timezone here to avoid flakiness